Bug 6947 / Bug 6948 - implement point and insert query params
[netconf.git] / restconf / sal-rest-connector / src / main / java / org / opendaylight / netconf / sal / restconf / impl / BrokerFacade.java
index e2097310f2ce485d422941dc05237f665a4a069e..f0fa715f0f1567beb3fee759a9b950739354517e 100644 (file)
@@ -13,14 +13,18 @@ import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CountDownLatch;
+import javax.annotation.Nullable;
 import javax.ws.rs.core.Response.Status;
 import org.opendaylight.controller.md.sal.common.api.data.AsyncDataBroker.DataChangeScope;
 import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
+import org.opendaylight.controller.md.sal.common.api.data.ReadFailedException;
 import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataBroker;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataChangeListener;
@@ -28,6 +32,8 @@ import org.opendaylight.controller.md.sal.dom.api.DOMDataReadTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataReadWriteTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMDataWriteTransaction;
 import org.opendaylight.controller.md.sal.dom.api.DOMMountPoint;
+import org.opendaylight.controller.md.sal.dom.api.DOMNotificationListener;
+import org.opendaylight.controller.md.sal.dom.api.DOMNotificationService;
 import org.opendaylight.controller.md.sal.dom.api.DOMRpcException;
 import org.opendaylight.controller.md.sal.dom.api.DOMRpcResult;
 import org.opendaylight.controller.md.sal.dom.api.DOMRpcService;
@@ -35,13 +41,23 @@ import org.opendaylight.controller.sal.core.api.Broker.ConsumerSession;
 import org.opendaylight.netconf.sal.restconf.impl.RestconfError.ErrorTag;
 import org.opendaylight.netconf.sal.restconf.impl.RestconfError.ErrorType;
 import org.opendaylight.netconf.sal.streams.listeners.ListenerAdapter;
+import org.opendaylight.netconf.sal.streams.listeners.NotificationListenerAdapter;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
+import org.opendaylight.yangtools.yang.data.api.schema.LeafSetEntryNode;
+import org.opendaylight.yangtools.yang.data.api.schema.LeafSetNode;
 import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.MapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
+import org.opendaylight.yangtools.yang.data.api.schema.OrderedLeafSetNode;
+import org.opendaylight.yangtools.yang.data.api.schema.OrderedMapNode;
 import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
+import org.opendaylight.yangtools.yang.data.util.DataSchemaContextNode;
+import org.opendaylight.yangtools.yang.data.util.DataSchemaContextTree;
+import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
+import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaPath;
 import org.slf4j.Logger;
@@ -54,12 +70,16 @@ public class BrokerFacade {
     private volatile DOMRpcService rpcService;
     private volatile ConsumerSession context;
     private DOMDataBroker domDataBroker;
+    private DOMNotificationService domNotification;
 
-    private BrokerFacade() {
-    }
+    private BrokerFacade() {}
 
     public void setRpcService(final DOMRpcService router) {
-        rpcService = router;
+        this.rpcService = router;
+    }
+
+    public void setDomNotificationService(final DOMNotificationService domNotification) {
+        this.domNotification = domNotification;
     }
 
     public void setContext(final ConsumerSession context) {
@@ -71,7 +91,7 @@ public class BrokerFacade {
     }
 
     private void checkPreconditions() {
-        if (context == null || domDataBroker == null) {
+        if ((this.context == null) || (this.domDataBroker == null)) {
             throw new RestconfDocumentedException(Status.SERVICE_UNAVAILABLE);
         }
     }
@@ -79,7 +99,7 @@ public class BrokerFacade {
     // READ configuration
     public NormalizedNode<?, ?> readConfigurationData(final YangInstanceIdentifier path) {
         checkPreconditions();
-        return readDataViaTransaction(domDataBroker.newReadOnlyTransaction(), CONFIGURATION, path);
+        return readDataViaTransaction(this.domDataBroker.newReadOnlyTransaction(), CONFIGURATION, path);
     }
 
     public NormalizedNode<?, ?> readConfigurationData(final DOMMountPoint mountPoint, final YangInstanceIdentifier path) {
@@ -95,7 +115,7 @@ public class BrokerFacade {
     // READ operational
     public NormalizedNode<?, ?> readOperationalData(final YangInstanceIdentifier path) {
         checkPreconditions();
-        return readDataViaTransaction(domDataBroker.newReadOnlyTransaction(), OPERATIONAL, path);
+        return readDataViaTransaction(this.domDataBroker.newReadOnlyTransaction(), OPERATIONAL, path);
     }
 
     public NormalizedNode<?, ?> readOperationalData(final DOMMountPoint mountPoint, final YangInstanceIdentifier path) {
@@ -108,120 +128,267 @@ public class BrokerFacade {
         throw new RestconfDocumentedException(errMsg);
     }
 
-    // PUT configuration
-    public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataPut(
-            final SchemaContext globalSchema, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload) {
+    /**
+     * <b>PUT configuration data</b>
+     *
+     * Prepare result(status) for PUT operation and PUT data via transaction.
+     * Return wrapped status and future from PUT.
+     *
+     * @param globalSchema
+     *            - used by merge parents (if contains list)
+     * @param path
+     *            - path of node
+     * @param payload
+     *            - input data
+     * @param point
+     * @param insert
+     * @return wrapper of status and future of PUT
+     */
+    public PutResult commitConfigurationDataPut(
+            final SchemaContext globalSchema, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final String insert, final String point) {
+        Preconditions.checkNotNull(globalSchema);
+        Preconditions.checkNotNull(path);
+        Preconditions.checkNotNull(payload);
+
         checkPreconditions();
-        return putDataViaTransaction(domDataBroker.newReadWriteTransaction(), CONFIGURATION, path, payload, globalSchema);
+
+        final DOMDataReadWriteTransaction newReadWriteTransaction = this.domDataBroker.newReadWriteTransaction();
+        final Status status = readDataViaTransaction(newReadWriteTransaction, CONFIGURATION, path) != null ? Status.OK
+                : Status.CREATED;
+        final CheckedFuture<Void, TransactionCommitFailedException> future = putDataViaTransaction(
+                newReadWriteTransaction, CONFIGURATION, path, payload, globalSchema, insert, point);
+        return new PutResult(status, future);
     }
 
-    public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataPut(
-            final DOMMountPoint mountPoint, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload) {
+    /**
+     * <b>PUT configuration data (Mount point)</b>
+     *
+     * Prepare result(status) for PUT operation and PUT data via transaction.
+     * Return wrapped status and future from PUT.
+     *
+     * @param mountPoint
+     *            - mount point for getting transaction for operation and schema
+     *            context for merging parents(if contains list)
+     * @param path
+     *            - path of node
+     * @param payload
+     *            - input data
+     * @param point
+     * @param insert
+     * @return wrapper of status and future of PUT
+     */
+    public PutResult commitMountPointDataPut(
+            final DOMMountPoint mountPoint, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final String insert, final String point) {
+        Preconditions.checkNotNull(mountPoint);
+        Preconditions.checkNotNull(path);
+        Preconditions.checkNotNull(payload);
+
         final Optional<DOMDataBroker> domDataBrokerService = mountPoint.getService(DOMDataBroker.class);
         if (domDataBrokerService.isPresent()) {
-            return putDataViaTransaction(domDataBrokerService.get().newReadWriteTransaction(), CONFIGURATION, path,
-                    payload, mountPoint.getSchemaContext());
+            final DOMDataReadWriteTransaction newReadWriteTransaction = domDataBrokerService.get().newReadWriteTransaction();
+            final Status status = readDataViaTransaction(newReadWriteTransaction, CONFIGURATION, path) != null
+                    ? Status.OK : Status.CREATED;
+            final CheckedFuture<Void, TransactionCommitFailedException> future = putDataViaTransaction(
+                    newReadWriteTransaction, CONFIGURATION, path, payload, mountPoint.getSchemaContext(), insert,
+                    point);
+            return new PutResult(status, future);
         }
         final String errMsg = "DOM data broker service isn't available for mount point " + path;
         LOG.warn(errMsg);
         throw new RestconfDocumentedException(errMsg);
     }
 
-    public PATCHStatusContext patchConfigurationDataWithinTransaction(final PATCHContext context,
-                                                                      final SchemaContext globalSchema) {
-        final DOMDataReadWriteTransaction patchTransaction = domDataBroker.newReadWriteTransaction();
-        List<PATCHStatusEntity> editCollection = new ArrayList<>();
+    public PATCHStatusContext patchConfigurationDataWithinTransaction(final PATCHContext patchContext) throws Exception {
+        final DOMMountPoint mountPoint = patchContext.getInstanceIdentifierContext().getMountPoint();
+
+        // get new transaction and schema context on server or on mounted device
+        final SchemaContext schemaContext;
+        final DOMDataReadWriteTransaction patchTransaction;
+        if (mountPoint == null) {
+            schemaContext = patchContext.getInstanceIdentifierContext().getSchemaContext();
+            patchTransaction = this.domDataBroker.newReadWriteTransaction();
+        } else {
+            schemaContext = mountPoint.getSchemaContext();
+
+            final Optional<DOMDataBroker> optional = mountPoint.getService(DOMDataBroker.class);
+
+            if (optional.isPresent()) {
+                patchTransaction = optional.get().newReadWriteTransaction();
+            } else {
+                // if mount point does not have broker it is not possible to continue and global error is reported
+                LOG.error("Http PATCH {} has failed - device {} does not support broker service",
+                        patchContext.getPatchId(), mountPoint.getIdentifier());
+                return new PATCHStatusContext(
+                        patchContext.getPatchId(),
+                        null,
+                        false,
+                        ImmutableList.of(new RestconfError(
+                                ErrorType.APPLICATION,
+                                ErrorTag.OPERATION_FAILED,
+                                "DOM data broker service isn't available for mount point "
+                                        + mountPoint.getIdentifier()))
+                );
+            }
+        }
+
+        final List<PATCHStatusEntity> editCollection = new ArrayList<>();
         List<RestconfError> editErrors;
-        List<RestconfError> globalErrors = null;
-        int errorCounter = 0;
+        boolean withoutError = true;
 
-        for (PATCHEntity patchEntity : context.getData()) {
+        for (final PATCHEntity patchEntity : patchContext.getData()) {
             final PATCHEditOperation operation = PATCHEditOperation.valueOf(patchEntity.getOperation().toUpperCase());
 
             switch (operation) {
                 case CREATE:
-                    if (errorCounter == 0) {
+                    if (withoutError) {
                         try {
                             postDataWithinTransaction(patchTransaction, CONFIGURATION, patchEntity.getTargetNode(),
-                                    patchEntity.getNode(), globalSchema);
+                                    patchEntity.getNode(), schemaContext);
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), true, null));
-                        } catch (RestconfDocumentedException e) {
+                        } catch (final RestconfDocumentedException e) {
+                            LOG.error("Error call http PATCH operation {} on target {}",
+                                    operation,
+                                    patchEntity.getTargetNode().toString());
+
                             editErrors = new ArrayList<>();
                             editErrors.addAll(e.getErrors());
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), false, editErrors));
-                            errorCounter++;
+                            withoutError = false;
                         }
                     }
                     break;
                 case REPLACE:
-                    if (errorCounter == 0) {
+                    if (withoutError) {
                         try {
                             putDataWithinTransaction(patchTransaction, CONFIGURATION, patchEntity
-                                    .getTargetNode(), patchEntity.getNode(), globalSchema);
+                                    .getTargetNode(), patchEntity.getNode(), schemaContext);
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), true, null));
-                        } catch (RestconfDocumentedException e) {
+                        } catch (final RestconfDocumentedException e) {
+                            LOG.error("Error call http PATCH operation {} on target {}",
+                                    operation,
+                                    patchEntity.getTargetNode().toString());
+
                             editErrors = new ArrayList<>();
                             editErrors.addAll(e.getErrors());
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), false, editErrors));
-                            errorCounter++;
+                            withoutError = false;
                         }
                     }
                     break;
                 case DELETE:
-                    if (errorCounter == 0) {
+                    if (withoutError) {
                         try {
                             deleteDataWithinTransaction(patchTransaction, CONFIGURATION, patchEntity
                                     .getTargetNode());
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), true, null));
-                        } catch (RestconfDocumentedException e) {
+                        } catch (final RestconfDocumentedException e) {
+                            LOG.error("Error call http PATCH operation {} on target {}",
+                                    operation,
+                                    patchEntity.getTargetNode().toString());
+
                             editErrors = new ArrayList<>();
                             editErrors.addAll(e.getErrors());
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), false, editErrors));
-                            errorCounter++;
+                            withoutError = false;
                         }
                     }
                     break;
                 case REMOVE:
-                    if (errorCounter == 0) {
+                    if (withoutError) {
                         try {
                             deleteDataWithinTransaction(patchTransaction, CONFIGURATION, patchEntity
                                     .getTargetNode());
                             editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), true, null));
-                        } catch (RestconfDocumentedException e) {
-                            LOG.error("Error removing {} by {} operation", patchEntity.getTargetNode().toString(),
-                                    patchEntity.getEditId(), e);
+                        } catch (final RestconfDocumentedException e) {
+                            LOG.error("Error call http PATCH operation {} on target {}",
+                                    operation,
+                                    patchEntity.getTargetNode().toString());
+
+                            editErrors = new ArrayList<>();
+                            editErrors.addAll(e.getErrors());
+                            editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), false, editErrors));
+                            withoutError = false;
                         }
                     }
                     break;
+                case MERGE:
+                    if (withoutError) {
+                        try {
+                            mergeDataWithinTransaction(patchTransaction, CONFIGURATION, patchEntity.getTargetNode(),
+                                    patchEntity.getNode(), schemaContext);
+                            editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), true, null));
+                        } catch (final RestconfDocumentedException e) {
+                            LOG.error("Error call http PATCH operation {} on target {}",
+                                    operation,
+                                    patchEntity.getTargetNode().toString());
+
+                            editErrors = new ArrayList<>();
+                            editErrors.addAll(e.getErrors());
+                            editCollection.add(new PATCHStatusEntity(patchEntity.getEditId(), false, editErrors));
+                            withoutError = false;
+                        }
+                    }
+                    break;
+                default:
+                    LOG.error("Unsupported http PATCH operation {} on target {}",
+                            operation,
+                            patchEntity.getTargetNode().toString());
+                    break;
             }
         }
 
-        //TODO: make sure possible global errors are filled up correctly and decide transaction submission based on that
-        //globalErrors = new ArrayList<>();
-        if (errorCounter == 0) {
-            final CheckedFuture<Void, TransactionCommitFailedException> submit = patchTransaction.submit();
-            return new PATCHStatusContext(context.getPatchId(), ImmutableList.copyOf(editCollection), true,
-                    globalErrors);
-        } else {
+        // if errors then cancel transaction and return error status
+        if (!withoutError) {
             patchTransaction.cancel();
-            return new PATCHStatusContext(context.getPatchId(), ImmutableList.copyOf(editCollection), false,
-                    globalErrors);
+            return new PATCHStatusContext(patchContext.getPatchId(), ImmutableList.copyOf(editCollection), false, null);
         }
+
+        // if no errors commit transaction
+        final CountDownLatch waiter = new CountDownLatch(1);
+        final CheckedFuture<Void, TransactionCommitFailedException> future = patchTransaction.submit();
+        final PATCHStatusContextHelper status = new PATCHStatusContextHelper();
+
+        Futures.addCallback(future, new FutureCallback<Void>() {
+            @Override
+            public void onSuccess(@Nullable final Void result) {
+                status.setStatus(new PATCHStatusContext(patchContext.getPatchId(), ImmutableList.copyOf(editCollection),
+                        true, null));
+                waiter.countDown();
+            }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                // if commit failed it is global error
+                LOG.error("Http PATCH {} transaction commit has failed", patchContext.getPatchId());
+                status.setStatus(new PATCHStatusContext(patchContext.getPatchId(), ImmutableList.copyOf(editCollection),
+                        false, ImmutableList.of(
+                        new RestconfError(ErrorType.APPLICATION, ErrorTag.OPERATION_FAILED, t.getMessage()))));
+                waiter.countDown();
+            }
+        });
+
+        waiter.await();
+        return status.getStatus();
     }
 
     // POST configuration
     public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataPost(
-            final SchemaContext globalSchema, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload) {
+            final SchemaContext globalSchema, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final String insert, final String point) {
         checkPreconditions();
-        return postDataViaTransaction(domDataBroker.newReadWriteTransaction(), CONFIGURATION, path, payload, globalSchema);
+        return postDataViaTransaction(this.domDataBroker.newReadWriteTransaction(), CONFIGURATION, path, payload,
+                globalSchema, insert, point);
     }
 
     public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataPost(
-            final DOMMountPoint mountPoint, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload) {
+            final DOMMountPoint mountPoint, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final String insert, final String point) {
         final Optional<DOMDataBroker> domDataBrokerService = mountPoint.getService(DOMDataBroker.class);
         if (domDataBrokerService.isPresent()) {
             return postDataViaTransaction(domDataBrokerService.get().newReadWriteTransaction(), CONFIGURATION, path,
-                    payload, mountPoint.getSchemaContext());
+                    payload, mountPoint.getSchemaContext(), insert, point);
         }
         final String errMsg = "DOM data broker service isn't available for mount point " + path;
         LOG.warn(errMsg);
@@ -232,14 +399,14 @@ public class BrokerFacade {
     public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataDelete(
             final YangInstanceIdentifier path) {
         checkPreconditions();
-        return deleteDataViaTransaction(domDataBroker.newWriteOnlyTransaction(), CONFIGURATION, path);
+        return deleteDataViaTransaction(this.domDataBroker.newReadWriteTransaction(), CONFIGURATION, path);
     }
 
     public CheckedFuture<Void, TransactionCommitFailedException> commitConfigurationDataDelete(
             final DOMMountPoint mountPoint, final YangInstanceIdentifier path) {
         final Optional<DOMDataBroker> domDataBrokerService = mountPoint.getService(DOMDataBroker.class);
         if (domDataBrokerService.isPresent()) {
-            return deleteDataViaTransaction(domDataBrokerService.get().newWriteOnlyTransaction(), CONFIGURATION, path);
+            return deleteDataViaTransaction(domDataBrokerService.get().newReadWriteTransaction(), CONFIGURATION, path);
         }
         final String errMsg = "DOM data broker service isn't available for mount point " + path;
         LOG.warn(errMsg);
@@ -249,11 +416,11 @@ public class BrokerFacade {
     // RPC
     public CheckedFuture<DOMRpcResult, DOMRpcException> invokeRpc(final SchemaPath type, final NormalizedNode<?, ?> input) {
         checkPreconditions();
-        if (rpcService == null) {
+        if (this.rpcService == null) {
             throw new RestconfDocumentedException(Status.SERVICE_UNAVAILABLE);
         }
         LOG.trace("Invoke RPC {} with input: {}", type, input);
-        return rpcService.invokeRpc(type, input);
+        return this.rpcService.invokeRpc(type, input);
     }
 
     public void registerToListenDataChanges(final LogicalDatastoreType datastore, final DataChangeScope scope,
@@ -265,7 +432,7 @@ public class BrokerFacade {
         }
 
         final YangInstanceIdentifier path = listener.getPath();
-        final ListenerRegistration<DOMDataChangeListener> registration = domDataBroker.registerDataChangeListener(
+        final ListenerRegistration<DOMDataChangeListener> registration = this.domDataBroker.registerDataChangeListener(
                 datastore, path, listener, scope);
 
         listener.setRegistration(registration);
@@ -275,109 +442,557 @@ public class BrokerFacade {
             final LogicalDatastoreType datastore, final YangInstanceIdentifier path) {
         LOG.trace("Read {} via Restconf: {}", datastore.name(), path);
         final ListenableFuture<Optional<NormalizedNode<?, ?>>> listenableFuture = transaction.read(datastore, path);
-        if (listenableFuture != null) {
-            Optional<NormalizedNode<?, ?>> optional;
-            try {
-                LOG.debug("Reading result data from transaction.");
-                optional = listenableFuture.get();
-            } catch (InterruptedException | ExecutionException e) {
-                LOG.warn("Exception by reading {} via Restconf: {}", datastore.name(), path, e);
-                throw new RestconfDocumentedException("Problem to get data from transaction.", e.getCause());
+        final ReadDataResult<NormalizedNode<?, ?>> readData = new ReadDataResult<>();
+        final CountDownLatch responseWaiter = new CountDownLatch(1);
+
+        Futures.addCallback(listenableFuture, new FutureCallback<Optional<NormalizedNode<?, ?>>>() {
 
+            @Override
+            public void onSuccess(final Optional<NormalizedNode<?, ?>> result) {
+                handlingCallback(null, datastore, path, result, readData);
+                responseWaiter.countDown();
             }
-            if (optional != null) {
-                if (optional.isPresent()) {
-                    return optional.get();
-                }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                responseWaiter.countDown();
+                handlingCallback(t, datastore, path, null, null);
             }
+        });
+
+        try {
+            responseWaiter.await();
+        } catch (final Exception e) {
+            final String msg = "Problem while waiting for response";
+            LOG.warn(msg);
+            throw new RestconfDocumentedException(msg, e);
         }
-        return null;
+        return readData.getResult();
     }
 
+    /**
+     * POST data and submit transaction {@link DOMDataReadWriteTransaction}
+     */
     private CheckedFuture<Void, TransactionCommitFailedException> postDataViaTransaction(
             final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext,
+            final String insert, final String point) {
+        LOG.trace("POST {} via Restconf: {} with payload {}", datastore.name(), path, payload);
+        postData(rWTransaction, datastore, path, payload, schemaContext, insert, point);
+        return rWTransaction.submit();
+    }
+
+    /**
+     * POST data and do NOT submit transaction {@link DOMDataReadWriteTransaction}
+     */
+    private void postDataWithinTransaction(
+            final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
+        LOG.trace("POST {} within Restconf PATCH: {} with payload {}", datastore.name(), path, payload);
+        postData(rWTransaction, datastore, path, payload, schemaContext, null, null);
+    }
+
+    private void postData(final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
+                          final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final SchemaContext schemaContext, final String insert, final String point) {
+        if (insert == null) {
+            makeNormalPost(rWTransaction, datastore, path, payload, schemaContext);
+        } else {
+            final DataSchemaNode schemaNode = checkListAndOrderedType(schemaContext, path);
+            checkItemDoesNotExists(rWTransaction, datastore, path);
+            switch (insert) {
+                case "first":
+                    if(schemaNode instanceof ListSchemaNode){
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent().getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            rWTransaction.delete(datastore, path.getParent().getParent());
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                            makeNormalPost(rWTransaction, datastore, path.getParent().getParent(), readList,
+                                    schemaContext);
+                        }
+                    } else {
+                        final OrderedLeafSetNode readLeafList =
+                                (OrderedLeafSetNode) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            rWTransaction.delete(datastore, path.getParent());
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                            makeNormalPost(rWTransaction, datastore, path.getParent().getParent(), readLeafList,
+                                    schemaContext);
+                        }
+                    }
+                    break;
+                case "last":
+                    simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                    break;
+                case "before":
+                    if(schemaNode instanceof ListSchemaNode){
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent().getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            insertWithPointListPost(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readList,
+                                    true);
+                        }
+                    } else {
+                        final OrderedLeafSetNode<?> readLeafList =
+                                (OrderedLeafSetNode<?>) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            insertWithPointLeafListPost(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readLeafList, true);
+                        }
+                    }
+                    break;
+                case "after":
+                    if (schemaNode instanceof ListSchemaNode) {
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent().getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            insertWithPointListPost(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readList,
+                                    false);
+                        }
+                    } else {
+                        final OrderedLeafSetNode<?> readLeafList =
+                                (OrderedLeafSetNode<?>) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+                        } else {
+                            insertWithPointLeafListPost(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readLeafList, false);
+                        }
+                    }
+                    break;
+                default:
+                    throw new RestconfDocumentedException(
+                            "Used bad value of insert parameter. Possible values are first, last, before or after, "
+                                    + "but was: " + insert);
+            }
+        }
+    }
+
+    private void insertWithPointLeafListPost(final DOMDataReadWriteTransaction rWTransaction,
+            final LogicalDatastoreType datastore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final SchemaContext schemaContext, final String point, final OrderedLeafSetNode<?> readLeafList,
+            final boolean before) {
+        rWTransaction.delete(datastore, path.getParent().getParent());
+        final InstanceIdentifierContext<?> instanceIdentifier =
+                ControllerContext.getInstance().toInstanceIdentifier(point);
+        int p = 0;
+        for (final LeafSetEntryNode<?> nodeChild : readLeafList.getValue()) {
+            if (nodeChild.getIdentifier().equals(instanceIdentifier.getInstanceIdentifier().getLastPathArgument())) {
+                break;
+            }
+            p++;
+        }
+        if (!before) {
+            p++;
+        }
+        int h = 0;
+        final NormalizedNode<?, ?> emptySubtree =
+                ImmutableNodes.fromInstanceId(schemaContext, path.getParent().getParent());
+        rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
+        for (final LeafSetEntryNode<?> nodeChild : readLeafList.getValue()) {
+            if (h == p) {
+                checkItemDoesNotExists(rWTransaction, datastore, path);
+                simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+            }
+            final YangInstanceIdentifier childPath = path.getParent().getParent().node(nodeChild.getIdentifier());
+            checkItemDoesNotExists(rWTransaction, datastore, childPath);
+            rWTransaction.put(datastore, childPath, nodeChild);
+            h++;
+        }
+    }
+
+    private void insertWithPointListPost(final DOMDataReadWriteTransaction rWTransaction,
+            final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext,
+            final String point, final MapNode readList, final boolean before) {
+        rWTransaction.delete(datastore, path.getParent().getParent());
+        final InstanceIdentifierContext<?> instanceIdentifier =
+                ControllerContext.getInstance().toInstanceIdentifier(point);
+        int p = 0;
+        for (final MapEntryNode mapEntryNode : readList.getValue()) {
+            if (mapEntryNode.getIdentifier()
+                    .equals(instanceIdentifier.getInstanceIdentifier().getLastPathArgument())) {
+                break;
+            }
+            p++;
+        }
+        if (!before) {
+            p++;
+        }
+        int h = 0;
+        final NormalizedNode<?, ?> emptySubtree =
+                ImmutableNodes.fromInstanceId(schemaContext, path.getParent().getParent());
+        rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
+        for (final MapEntryNode mapEntryNode : readList.getValue()) {
+            if (h == p) {
+                checkItemDoesNotExists(rWTransaction, datastore, path);
+                simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
+            }
+            final YangInstanceIdentifier childPath = path.getParent().getParent().node(mapEntryNode.getIdentifier());
+            checkItemDoesNotExists(rWTransaction, datastore, childPath);
+            rWTransaction.put(datastore, childPath, mapEntryNode);
+            h++;
+        }
+    }
+
+    private DataSchemaNode checkListAndOrderedType(final SchemaContext ctx,
+            final YangInstanceIdentifier path) {
+        final YangInstanceIdentifier parent = path.getParent();
+        final DataSchemaContextNode<?> node = DataSchemaContextTree.from(ctx).getChild(parent);
+        final DataSchemaNode dataSchemaNode = node.getDataSchemaNode();
+
+        if (dataSchemaNode instanceof ListSchemaNode) {
+            if(!((ListSchemaNode) dataSchemaNode).isUserOrdered()) {
+                throw new RestconfDocumentedException("Insert parameter can be used only with ordered-by user list.");
+            }
+            return dataSchemaNode;
+        }
+        if (dataSchemaNode instanceof LeafListSchemaNode) {
+            if(!((LeafListSchemaNode) dataSchemaNode).isUserOrdered()) {
+                throw new RestconfDocumentedException("Insert parameter can be used only with ordered-by user leaf-list.");
+            }
+            return dataSchemaNode;
+        }
+        throw new RestconfDocumentedException("Insert parameter can be used only with list or leaf-list");
+    }
+
+    private void makeNormalPost(final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
             final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
-        // FIXME: This is doing correct post for container and list children
-        //        not sure if this will work for choice case
-        if(payload instanceof MapNode) {
-            LOG.trace("POST {} via Restconf: {} with payload {}", datastore.name(), path, payload);
+        if (payload instanceof MapNode) {
             final NormalizedNode<?, ?> emptySubtree = ImmutableNodes.fromInstanceId(schemaContext, path);
             rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
             ensureParentsByMerge(datastore, path, rWTransaction, schemaContext);
-            for(final MapEntryNode child : ((MapNode) payload).getValue()) {
+            for (final MapEntryNode child : ((MapNode) payload).getValue()) {
                 final YangInstanceIdentifier childPath = path.node(child.getIdentifier());
                 checkItemDoesNotExists(rWTransaction, datastore, childPath);
                 rWTransaction.put(datastore, childPath, child);
             }
-        } else {
-            checkItemDoesNotExists(rWTransaction,datastore, path);
-            ensureParentsByMerge(datastore, path, rWTransaction, schemaContext);
-            rWTransaction.put(datastore, path, payload);
-        }
-        return rWTransaction.submit();
-    }
-
-    private void postDataWithinTransaction(
-            final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
-            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
-        // FIXME: This is doing correct post for container and list children
-        //        not sure if this will work for choice case
-        if(payload instanceof MapNode) {
-            LOG.trace("POST {} within Restconf PATCH: {} with payload {}", datastore.name(), path, payload);
+        } else if (payload instanceof LeafSetNode) {
             final NormalizedNode<?, ?> emptySubtree = ImmutableNodes.fromInstanceId(schemaContext, path);
             rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
             ensureParentsByMerge(datastore, path, rWTransaction, schemaContext);
-            for(final MapEntryNode child : ((MapNode) payload).getValue()) {
+            for (final LeafSetEntryNode<?> child : ((LeafSetNode<?>) payload).getValue()) {
                 final YangInstanceIdentifier childPath = path.node(child.getIdentifier());
                 checkItemDoesNotExists(rWTransaction, datastore, childPath);
                 rWTransaction.put(datastore, childPath, child);
             }
         } else {
-            checkItemDoesNotExists(rWTransaction,datastore, path);
-            ensureParentsByMerge(datastore, path, rWTransaction, schemaContext);
-            rWTransaction.put(datastore, path, payload);
+            simplePostPut(rWTransaction, datastore, path, payload, schemaContext);
         }
     }
 
-    private void checkItemDoesNotExists(final DOMDataReadWriteTransaction rWTransaction,final LogicalDatastoreType store, final YangInstanceIdentifier path) {
-        final ListenableFuture<Boolean> futureDatastoreData = rWTransaction.exists(store, path);
+    private void simplePostPut(final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
+        checkItemDoesNotExists(rWTransaction, datastore, path);
+        ensureParentsByMerge(datastore, path, rWTransaction, schemaContext);
+        rWTransaction.put(datastore, path, payload);
+    }
+
+    /**
+     * Check if item already exists. Throws error if it does NOT already exist.
+     * @param rWTransaction Current transaction
+     * @param store Used datastore
+     * @param path Path to item to verify its existence
+     */
+    private void checkItemExists(final DOMDataReadWriteTransaction rWTransaction,
+                                 final LogicalDatastoreType store, final YangInstanceIdentifier path) {
+        final CountDownLatch responseWaiter = new CountDownLatch(1);
+        final ReadDataResult<Boolean> readData = new ReadDataResult<>();
+        final CheckedFuture<Boolean, ReadFailedException> future = rWTransaction.exists(store, path);
+
+        Futures.addCallback(future, new FutureCallback<Boolean>() {
+            @Override
+            public void onSuccess(@Nullable final Boolean result) {
+                handlingCallback(null, store, path, Optional.of(result), readData);
+                responseWaiter.countDown();
+            }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                responseWaiter.countDown();
+                handlingCallback(t, store, path, null, null);
+            }
+        });
+
         try {
-            if (futureDatastoreData.get()) {
-                final String errMsg = "Post Configuration via Restconf was not executed because data already exists";
-                LOG.trace("{}:{}", errMsg, path);
-                rWTransaction.cancel();
-                throw new RestconfDocumentedException("Data already exists for path: " + path, ErrorType.PROTOCOL,
-                        ErrorTag.DATA_EXISTS);
+            responseWaiter.await();
+        } catch (final Exception e) {
+            final String msg = "Problem while waiting for response";
+            LOG.warn(msg);
+            throw new RestconfDocumentedException(msg, e);
+        }
+
+        if ((readData.getResult() == null) || !readData.getResult()) {
+            final String errMsg = "Operation via Restconf was not executed because data does not exist";
+            LOG.trace("{}:{}", errMsg, path);
+            rWTransaction.cancel();
+            throw new RestconfDocumentedException("Data does not exist for path: " + path, ErrorType.PROTOCOL,
+                    ErrorTag.DATA_MISSING);
+        }
+    }
+
+    /**
+     * Check if item does NOT already exist. Throws error if it already exists.
+     * @param rWTransaction Current transaction
+     * @param store Used datastore
+     * @param path Path to item to verify its existence
+     */
+    private void checkItemDoesNotExists(final DOMDataReadWriteTransaction rWTransaction,
+                                        final LogicalDatastoreType store, final YangInstanceIdentifier path) {
+        final CountDownLatch responseWaiter = new CountDownLatch(1);
+        final ReadDataResult<Boolean> readData = new ReadDataResult<>();
+        final CheckedFuture<Boolean, ReadFailedException> future = rWTransaction.exists(store, path);
+
+        Futures.addCallback(future, new FutureCallback<Boolean>() {
+            @Override
+            public void onSuccess(@Nullable final Boolean result) {
+                handlingCallback(null, store, path, Optional.of(result), readData);
+                responseWaiter.countDown();
+            }
+
+            @Override
+            public void onFailure(final Throwable t) {
+                responseWaiter.countDown();
+                handlingCallback(t, store, path, null, null);
             }
-        } catch (InterruptedException | ExecutionException e) {
-            LOG.warn("It wasn't possible to get data loaded from datastore at path {}", path, e);
+        });
+
+        try {
+            responseWaiter.await();
+        } catch (final Exception e) {
+            final String msg = "Problem while waiting for response";
+            LOG.warn(msg);
+            throw new RestconfDocumentedException(msg, e);
         }
 
+        if (readData.getResult()) {
+            final String errMsg = "Operation via Restconf was not executed because data already exists";
+            LOG.trace("{}:{}", errMsg, path);
+            rWTransaction.cancel();
+            throw new RestconfDocumentedException("Data already exists for path: " + path, ErrorType.PROTOCOL,
+                    ErrorTag.DATA_EXISTS);
+        }
     }
 
+    /**
+     * PUT data and submit {@link DOMDataReadWriteTransaction}
+     *
+     * @param point
+     * @param insert
+     */
     private CheckedFuture<Void, TransactionCommitFailedException> putDataViaTransaction(
-            final DOMDataReadWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
-            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
+            final DOMDataReadWriteTransaction readWriteTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext,
+            final String insert, final String point) {
         LOG.trace("Put {} via Restconf: {} with payload {}", datastore.name(), path, payload);
-        ensureParentsByMerge(datastore, path, writeTransaction, schemaContext);
-        writeTransaction.put(datastore, path, payload);
-        return writeTransaction.submit();
+        putData(readWriteTransaction, datastore, path, payload, schemaContext, insert, point);
+        return readWriteTransaction.submit();
     }
 
+    /**
+     * PUT data and do NOT submit {@link DOMDataReadWriteTransaction}
+     *
+     * @param insert
+     * @param point
+     */
     private void putDataWithinTransaction(
             final DOMDataReadWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
             final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
         LOG.trace("Put {} within Restconf PATCH: {} with payload {}", datastore.name(), path, payload);
+        putData(writeTransaction, datastore, path, payload, schemaContext, null, null);
+    }
+
+    // FIXME: This is doing correct put for container and list children, not sure if this will work for choice case
+    private void putData(final DOMDataReadWriteTransaction rWTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext,
+            final String insert, final String point) {
+        if (insert == null) {
+            makePut(rWTransaction, datastore, path, payload, schemaContext);
+        } else {
+            final DataSchemaNode schemaNode = checkListAndOrderedType(schemaContext, path);
+            checkItemDoesNotExists(rWTransaction, datastore, path);
+            switch (insert) {
+                case "first":
+                    if (schemaNode instanceof ListSchemaNode) {
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            rWTransaction.delete(datastore, path.getParent());
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                            makePut(rWTransaction, datastore, path.getParent(), readList, schemaContext);
+                        }
+                    } else {
+                        final OrderedLeafSetNode<?> readLeafList =
+                                (OrderedLeafSetNode<?>) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            rWTransaction.delete(datastore, path.getParent());
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                            makePut(rWTransaction, datastore, path.getParent(), readLeafList,
+                                    schemaContext);
+                        }
+                    }
+                    break;
+                case "last":
+                    simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                    break;
+                case "before":
+                    if (schemaNode instanceof ListSchemaNode) {
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            insertWithPointListPut(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readList, true);
+                        }
+                    } else {
+                        final OrderedLeafSetNode<?> readLeafList =
+                                (OrderedLeafSetNode<?>) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            insertWithPointLeafListPut(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readLeafList, true);
+                        }
+                    }
+                    break;
+                case "after":
+                    if (schemaNode instanceof ListSchemaNode) {
+                        final OrderedMapNode readList =
+                                (OrderedMapNode) this.readConfigurationData(path.getParent());
+                        if ((readList == null) || readList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            insertWithPointListPut(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readList, false);
+                        }
+                    } else {
+                        final OrderedLeafSetNode<?> readLeafList =
+                                (OrderedLeafSetNode<?>) readConfigurationData(path.getParent());
+                        if ((readLeafList == null) || readLeafList.getValue().isEmpty()) {
+                            simplePut(datastore, path, rWTransaction, schemaContext, payload);
+                        } else {
+                            insertWithPointLeafListPut(rWTransaction, datastore, path, payload, schemaContext, point,
+                                    readLeafList, false);
+                        }
+                    }
+                    break;
+                default:
+                    throw new RestconfDocumentedException(
+                            "Used bad value of insert parameter. Possible values are first, last, before or after, "
+                                    + "but was: " + insert);
+            }
+        }
+    }
+
+    private void insertWithPointLeafListPut(final DOMDataReadWriteTransaction rWTransaction,
+            final LogicalDatastoreType datastore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final SchemaContext schemaContext, final String point, final OrderedLeafSetNode<?> readLeafList,
+            final boolean before) {
+        rWTransaction.delete(datastore, path.getParent());
+        final InstanceIdentifierContext<?> instanceIdentifier =
+                ControllerContext.getInstance().toInstanceIdentifier(point);
+        int p = 0;
+        for (final LeafSetEntryNode<?> nodeChild : readLeafList.getValue()) {
+            if (nodeChild.getIdentifier().equals(instanceIdentifier.getInstanceIdentifier().getLastPathArgument())) {
+                break;
+            }
+            p++;
+        }
+        if (!before) {
+            p++;
+        }
+        int h = 0;
+        final NormalizedNode<?, ?> emptySubtree =
+                ImmutableNodes.fromInstanceId(schemaContext, path.getParent());
+        rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
+        for (final LeafSetEntryNode<?> nodeChild : readLeafList.getValue()) {
+            if (h == p) {
+                simplePut(datastore, path, rWTransaction, schemaContext, payload);
+            }
+            final YangInstanceIdentifier childPath = path.getParent().node(nodeChild.getIdentifier());
+            rWTransaction.put(datastore, childPath, nodeChild);
+            h++;
+        }
+    }
+
+    private void insertWithPointListPut(final DOMDataReadWriteTransaction rWTransaction,
+            final LogicalDatastoreType datastore, final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload,
+            final SchemaContext schemaContext, final String point, final OrderedMapNode readList,
+            final boolean before) {
+        rWTransaction.delete(datastore, path.getParent());
+        final InstanceIdentifierContext<?> instanceIdentifier =
+                ControllerContext.getInstance().toInstanceIdentifier(point);
+        int p = 0;
+        for (final MapEntryNode mapEntryNode : readList.getValue()) {
+            if (mapEntryNode.getIdentifier().equals(instanceIdentifier.getInstanceIdentifier().getLastPathArgument())) {
+                break;
+            }
+            p++;
+        }
+        if (!before) {
+            p++;
+        }
+        int h = 0;
+        final NormalizedNode<?, ?> emptySubtree =
+                ImmutableNodes.fromInstanceId(schemaContext, path.getParent());
+        rWTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
+        for (final MapEntryNode mapEntryNode : readList.getValue()) {
+            if (h == p) {
+                simplePut(datastore, path, rWTransaction, schemaContext, payload);
+            }
+            final YangInstanceIdentifier childPath = path.getParent().node(mapEntryNode.getIdentifier());
+            rWTransaction.put(datastore, childPath, mapEntryNode);
+            h++;
+        }
+    }
+
+    private void makePut(final DOMDataReadWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
+        if (payload instanceof MapNode) {
+            final NormalizedNode<?, ?> emptySubtree = ImmutableNodes.fromInstanceId(schemaContext, path);
+            writeTransaction.merge(datastore, YangInstanceIdentifier.create(emptySubtree.getIdentifier()), emptySubtree);
+            ensureParentsByMerge(datastore, path, writeTransaction, schemaContext);
+            for (final MapEntryNode child : ((MapNode) payload).getValue()) {
+                final YangInstanceIdentifier childPath = path.node(child.getIdentifier());
+                writeTransaction.put(datastore, childPath, child);
+            }
+        } else {
+            simplePut(datastore, path, writeTransaction, schemaContext, payload);
+        }
+    }
+
+    private void simplePut(final LogicalDatastoreType datastore, final YangInstanceIdentifier path,
+            final DOMDataReadWriteTransaction writeTransaction, final SchemaContext schemaContext,
+            final NormalizedNode<?, ?> payload) {
         ensureParentsByMerge(datastore, path, writeTransaction, schemaContext);
         writeTransaction.put(datastore, path, payload);
     }
 
     private CheckedFuture<Void, TransactionCommitFailedException> deleteDataViaTransaction(
-            final DOMDataWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
+            final DOMDataReadWriteTransaction readWriteTransaction, final LogicalDatastoreType datastore,
             final YangInstanceIdentifier path) {
         LOG.trace("Delete {} via Restconf: {}", datastore.name(), path);
-        writeTransaction.delete(datastore, path);
-        return writeTransaction.submit();
+        checkItemExists(readWriteTransaction, datastore, path);
+        readWriteTransaction.delete(datastore, path);
+        return readWriteTransaction.submit();
     }
 
     private void deleteDataWithinTransaction(
@@ -387,38 +1002,117 @@ public class BrokerFacade {
         writeTransaction.delete(datastore, path);
     }
 
+    private void mergeDataWithinTransaction(
+            final DOMDataReadWriteTransaction writeTransaction, final LogicalDatastoreType datastore,
+            final YangInstanceIdentifier path, final NormalizedNode<?, ?> payload, final SchemaContext schemaContext) {
+        LOG.trace("Merge {} within Restconf PATCH: {} with payload {}", datastore.name(), path, payload);
+        ensureParentsByMerge(datastore, path, writeTransaction, schemaContext);
+
+        // merging is necessary only for lists otherwise we can call put method
+        if (payload instanceof MapNode) {
+            writeTransaction.merge(datastore, path, payload);
+        } else {
+            writeTransaction.put(datastore, path, payload);
+        }
+    }
+
     public void setDomDataBroker(final DOMDataBroker domDataBroker) {
         this.domDataBroker = domDataBroker;
     }
 
-    private void ensureParentsByMerge(final LogicalDatastoreType store,
-                                      final YangInstanceIdentifier normalizedPath, final DOMDataReadWriteTransaction rwTx, final SchemaContext schemaContext) {
+    /**
+     * Helper class for result of transaction commit callback.
+     * @param <T> Type of result
+     */
+    private final class ReadDataResult<T> {
+        T result = null;
+
+        T getResult() {
+            return this.result;
+        }
+
+        void setResult(final T result) {
+            this.result = result;
+        }
+    }
+
+    /**
+     * Set result from transaction commit callback.
+     * @param t Throwable if transaction commit failed
+     * @param datastore Datastore from which data are read
+     * @param path Path from which data are read
+     * @param result Result of read from {@code datastore}
+     * @param readData Result value which will be set
+     * @param <X> Result type
+     */
+    protected final static <X> void handlingCallback(final Throwable t, final LogicalDatastoreType datastore,
+                                                     final YangInstanceIdentifier path, final Optional<X> result,
+                                                     final ReadDataResult<X> readData) {
+        if (t != null) {
+            LOG.warn("Exception by reading {} via Restconf: {}", datastore.name(), path, t);
+            throw new RestconfDocumentedException("Problem to get data from transaction.", t);
+        } else {
+            LOG.debug("Reading result data from transaction.");
+            if (result != null) {
+                if (result.isPresent()) {
+                    readData.setResult(result.get());
+                }
+            }
+        }
+    }
+
+    public void registerToListenNotification(final NotificationListenerAdapter listener) {
+        checkPreconditions();
+
+        if (listener.isListening()) {
+            return;
+        }
+
+        final SchemaPath path = listener.getSchemaPath();
+        final ListenerRegistration<DOMNotificationListener> registration = this.domNotification
+                .registerNotificationListener(listener, path);
+
+        listener.setRegistration(registration);
+    }
+
+    private final class PATCHStatusContextHelper {
+        PATCHStatusContext status;
+
+        public PATCHStatusContext getStatus() {
+            return this.status;
+        }
+
+        public void setStatus(final PATCHStatusContext status) {
+            this.status = status;
+        }
+    }
+
+    private void ensureParentsByMerge(final LogicalDatastoreType store, final YangInstanceIdentifier normalizedPath,
+            final DOMDataReadWriteTransaction rwTx, final SchemaContext schemaContext) {
         final List<PathArgument> normalizedPathWithoutChildArgs = new ArrayList<>();
         YangInstanceIdentifier rootNormalizedPath = null;
 
         final Iterator<PathArgument> it = normalizedPath.getPathArguments().iterator();
 
-        while(it.hasNext()) {
+        while (it.hasNext()) {
             final PathArgument pathArgument = it.next();
-            if(rootNormalizedPath == null) {
+            if (rootNormalizedPath == null) {
                 rootNormalizedPath = YangInstanceIdentifier.create(pathArgument);
             }
 
-            // Skip last element, its not a parent
-            if(it.hasNext()) {
+            if (it.hasNext()) {
                 normalizedPathWithoutChildArgs.add(pathArgument);
             }
         }
 
-        // No parent structure involved, no need to ensure parents
-        if(normalizedPathWithoutChildArgs.isEmpty()) {
+        if (normalizedPathWithoutChildArgs.isEmpty()) {
             return;
         }
 
         Preconditions.checkArgument(rootNormalizedPath != null, "Empty path received");
 
-        final NormalizedNode<?, ?> parentStructure =
-                ImmutableNodes.fromInstanceId(schemaContext, YangInstanceIdentifier.create(normalizedPathWithoutChildArgs));
+        final NormalizedNode<?, ?> parentStructure = ImmutableNodes.fromInstanceId(schemaContext,
+                YangInstanceIdentifier.create(normalizedPathWithoutChildArgs));
         rwTx.merge(store, rootNormalizedPath, parentStructure);
     }
 }