Do not access model context multiple times
[netconf.git] / restconf / restconf-nb-rfc8040 / src / main / java / org / opendaylight / restconf / nb / rfc8040 / rests / services / impl / RestconfDataServiceImpl.java
index d0ebda4593cf91fd2f6874206a2e10bb7f4ac0cc..8257b6394e0cc986552805cadb9b8211ff8e2582 100644 (file)
@@ -8,8 +8,6 @@
 package org.opendaylight.restconf.nb.rfc8040.rests.services.impl;
 
 import static java.util.Objects.requireNonNull;
-import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant.PostPutQueryParameters.INSERT;
-import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant.PostPutQueryParameters.POINT;
 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.NOTIFICATION_STREAM;
 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAMS_PATH;
 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_ACCESS_PATH_PART;
@@ -17,43 +15,48 @@ import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsCo
 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_PATH;
 import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_PATH_PART;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.MoreExecutors;
 import java.net.URI;
 import java.time.Clock;
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
+import java.util.HashMap;
 import java.util.List;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Optional;
+import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import javax.ws.rs.Path;
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.UriInfo;
-import org.eclipse.jdt.annotation.NonNull;
-import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
+import org.opendaylight.mdsal.dom.api.DOMActionException;
 import org.opendaylight.mdsal.dom.api.DOMActionResult;
+import org.opendaylight.mdsal.dom.api.DOMActionService;
 import org.opendaylight.mdsal.dom.api.DOMDataBroker;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteOperations;
+import org.opendaylight.mdsal.dom.api.DOMDataTreeWriteTransaction;
 import org.opendaylight.mdsal.dom.api.DOMMountPoint;
-import org.opendaylight.netconf.dom.api.NetconfDataTreeService;
+import org.opendaylight.mdsal.dom.api.DOMMountPointService;
+import org.opendaylight.mdsal.dom.spi.SimpleDOMActionResult;
 import org.opendaylight.restconf.common.context.InstanceIdentifierContext;
-import org.opendaylight.restconf.common.context.NormalizedNodeContext;
-import org.opendaylight.restconf.common.context.WriterParameters;
 import org.opendaylight.restconf.common.errors.RestconfDocumentedException;
-import org.opendaylight.restconf.common.errors.RestconfError;
-import org.opendaylight.restconf.common.errors.RestconfError.ErrorTag;
-import org.opendaylight.restconf.common.errors.RestconfError.ErrorType;
 import org.opendaylight.restconf.common.patch.PatchContext;
 import org.opendaylight.restconf.common.patch.PatchStatusContext;
+import org.opendaylight.restconf.nb.rfc8040.ReadDataParams;
 import org.opendaylight.restconf.nb.rfc8040.Rfc8040;
-import org.opendaylight.restconf.nb.rfc8040.handlers.ActionServiceHandler;
-import org.opendaylight.restconf.nb.rfc8040.handlers.DOMMountPointServiceHandler;
+import org.opendaylight.restconf.nb.rfc8040.WriteDataParams;
+import org.opendaylight.restconf.nb.rfc8040.databind.jaxrs.QueryParams;
 import org.opendaylight.restconf.nb.rfc8040.handlers.SchemaContextHandler;
-import org.opendaylight.restconf.nb.rfc8040.handlers.TransactionChainHandler;
+import org.opendaylight.restconf.nb.rfc8040.legacy.NormalizedNodePayload;
+import org.opendaylight.restconf.nb.rfc8040.legacy.QueryParameters;
 import org.opendaylight.restconf.nb.rfc8040.rests.services.api.RestconfDataService;
 import org.opendaylight.restconf.nb.rfc8040.rests.services.api.RestconfStreamsSubscriptionService;
 import org.opendaylight.restconf.nb.rfc8040.rests.transactions.MdsalRestconfStrategy;
-import org.opendaylight.restconf.nb.rfc8040.rests.transactions.NetconfRestconfStrategy;
 import org.opendaylight.restconf.nb.rfc8040.rests.transactions.RestconfStrategy;
 import org.opendaylight.restconf.nb.rfc8040.rests.utils.DeleteDataTransactionUtil;
 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PatchDataTransactionUtil;
@@ -61,25 +64,29 @@ import org.opendaylight.restconf.nb.rfc8040.rests.utils.PlainPatchDataTransactio
 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PostDataTransactionUtil;
 import org.opendaylight.restconf.nb.rfc8040.rests.utils.PutDataTransactionUtil;
 import org.opendaylight.restconf.nb.rfc8040.rests.utils.ReadDataTransactionUtil;
-import org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfDataServiceConstant;
-import org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfInvokeOperationsUtil;
 import org.opendaylight.restconf.nb.rfc8040.streams.Configuration;
 import org.opendaylight.restconf.nb.rfc8040.streams.listeners.NotificationListenerAdapter;
 import org.opendaylight.restconf.nb.rfc8040.utils.mapping.RestconfMappingNodeUtil;
-import org.opendaylight.restconf.nb.rfc8040.utils.parser.IdentifierCodec;
 import org.opendaylight.restconf.nb.rfc8040.utils.parser.ParserIdentifier;
 import org.opendaylight.yang.gen.v1.urn.sal.restconf.event.subscription.rev140708.NotificationOutputTypeGrouping.NotificationOutputType;
-import org.opendaylight.yangtools.concepts.Immutable;
+import org.opendaylight.yangtools.yang.common.ErrorTag;
+import org.opendaylight.yangtools.yang.common.ErrorType;
 import org.opendaylight.yangtools.yang.common.QName;
 import org.opendaylight.yangtools.yang.common.Revision;
+import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.PathArgument;
 import org.opendaylight.yangtools.yang.data.api.schema.ContainerNode;
+import org.opendaylight.yangtools.yang.data.api.schema.MapEntryNode;
 import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode;
 import org.opendaylight.yangtools.yang.model.api.ActionDefinition;
 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
+import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.NotificationDefinition;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.SchemaNode;
-import org.opendaylight.yangtools.yang.model.api.SchemaPath;
+import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier.Absolute;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -88,58 +95,32 @@ import org.slf4j.LoggerFactory;
  */
 @Path("/")
 public class RestconfDataServiceImpl implements RestconfDataService {
-    private static final class QueryParams implements Immutable {
-        final @Nullable String point;
-        final @Nullable String insert;
-
-        QueryParams(final @Nullable String insert, final @Nullable String point) {
-            this.insert = insert;
-            this.point = point;
-        }
-    }
-
     private static final Logger LOG = LoggerFactory.getLogger(RestconfDataServiceImpl.class);
     private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MMM-dd HH:mm:ss");
+    private static final QName NETCONF_BASE_QNAME = SchemaContext.NAME;
 
     private final RestconfStreamsSubscriptionService delegRestconfSubscrService;
+    private final SchemaContextHandler schemaContextHandler;
+    private final MdsalRestconfStrategy restconfStrategy;
+    private final DOMMountPointService mountPointService;
     private final SubscribeToStreamUtil streamUtils;
-
-    // FIXME: evaluate thread-safety of updates (synchronized) vs. access (mostly unsynchronized) here
-    private SchemaContextHandler schemaContextHandler;
-    private TransactionChainHandler transactionChainHandler;
-    private DOMMountPointServiceHandler mountPointServiceHandler;
-    private volatile ActionServiceHandler actionServiceHandler;
+    private final DOMActionService actionService;
+    private final DOMDataBroker dataBroker;
 
     public RestconfDataServiceImpl(final SchemaContextHandler schemaContextHandler,
-            final TransactionChainHandler transactionChainHandler,
-            final DOMMountPointServiceHandler mountPointServiceHandler,
+            final DOMDataBroker dataBroker, final DOMMountPointService  mountPointService,
             final RestconfStreamsSubscriptionService delegRestconfSubscrService,
-            final ActionServiceHandler actionServiceHandler,
-            final Configuration configuration) {
-        this.actionServiceHandler = requireNonNull(actionServiceHandler);
+            final DOMActionService actionService, final Configuration configuration) {
         this.schemaContextHandler = requireNonNull(schemaContextHandler);
-        this.transactionChainHandler = requireNonNull(transactionChainHandler);
-        this.mountPointServiceHandler = requireNonNull(mountPointServiceHandler);
+        this.dataBroker = requireNonNull(dataBroker);
+        restconfStrategy = new MdsalRestconfStrategy(dataBroker);
+        this.mountPointService = requireNonNull(mountPointService);
         this.delegRestconfSubscrService = requireNonNull(delegRestconfSubscrService);
+        this.actionService = requireNonNull(actionService);
         streamUtils = configuration.isUseSSE() ? SubscribeToStreamUtil.serverSentEvents()
                 : SubscribeToStreamUtil.webSockets();
     }
 
-    @Override
-    public synchronized void updateHandlers(final Object... handlers) {
-        for (final Object object : handlers) {
-            if (object instanceof SchemaContextHandler) {
-                schemaContextHandler = (SchemaContextHandler) object;
-            } else if (object instanceof ActionServiceHandler) {
-                actionServiceHandler = (ActionServiceHandler) object;
-            } else if (object instanceof DOMMountPointServiceHandler) {
-                mountPointServiceHandler = (DOMMountPointServiceHandler) object;
-            } else if (object instanceof TransactionChainHandler) {
-                transactionChainHandler = (TransactionChainHandler) object;
-            }
-        }
-    }
-
     @Override
     public Response readData(final UriInfo uriInfo) {
         return readData(null, uriInfo);
@@ -147,216 +128,131 @@ public class RestconfDataServiceImpl implements RestconfDataService {
 
     @Override
     public Response readData(final String identifier, final UriInfo uriInfo) {
-        final EffectiveModelContext schemaContextRef = this.schemaContextHandler.get();
-        final InstanceIdentifierContext<?> instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
-                identifier, schemaContextRef, Optional.of(this.mountPointServiceHandler.get()));
-        final WriterParameters parameters = ReadDataTransactionUtil.parseUriParameters(instanceIdentifier, uriInfo);
+        final ReadDataParams readParams = QueryParams.newReadDataParams(uriInfo);
 
+        final EffectiveModelContext schemaContextRef = schemaContextHandler.get();
+        final InstanceIdentifierContext instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
+                identifier, schemaContextRef, Optional.of(mountPointService));
         final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
-        final RestconfStrategy strategy = getRestconfStrategy(instanceIdentifier, mountPoint);
-        final NormalizedNode<?, ?> node = readData(identifier, parameters.getContent(),
-                strategy, parameters.getWithDefault(), schemaContextRef, uriInfo);
+
+        // FIXME: this looks quite crazy, why do we even have it?
+        if (mountPoint == null && identifier != null && identifier.contains(STREAMS_PATH)
+            && !identifier.contains(STREAM_PATH_PART)) {
+            createAllYangNotificationStreams(schemaContextRef, uriInfo);
+        }
+
+        final QueryParameters queryParams = QueryParams.newQueryParameters(readParams, instanceIdentifier);
+        final List<YangInstanceIdentifier> fieldPaths = queryParams.fieldPaths();
+        final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
+        final NormalizedNode node;
+        if (fieldPaths != null && !fieldPaths.isEmpty()) {
+            node = ReadDataTransactionUtil.readData(readParams.content(), instanceIdentifier.getInstanceIdentifier(),
+                    strategy, readParams.withDefaults(), schemaContextRef, fieldPaths);
+        } else {
+            node = ReadDataTransactionUtil.readData(readParams.content(), instanceIdentifier.getInstanceIdentifier(),
+                    strategy, readParams.withDefaults(), schemaContextRef);
+        }
+
+        // FIXME: this is utter craziness, refactor it properly!
         if (identifier != null && identifier.contains(STREAM_PATH) && identifier.contains(STREAM_ACCESS_PATH_PART)
                 && identifier.contains(STREAM_LOCATION_PATH_PART)) {
-            final String value = (String) node.getValue();
+            final String value = (String) node.body();
             final String streamName = value.substring(value.indexOf(NOTIFICATION_STREAM + '/'));
-            this.delegRestconfSubscrService.subscribeToStream(streamName, uriInfo);
+            delegRestconfSubscrService.subscribeToStream(streamName, uriInfo);
         }
         if (node == null) {
             throw new RestconfDocumentedException(
                     "Request could not be completed because the relevant data model content does not exist",
-                    RestconfError.ErrorType.PROTOCOL,
-                    RestconfError.ErrorTag.DATA_MISSING);
+                    ErrorType.PROTOCOL, ErrorTag.DATA_MISSING);
         }
 
-        if (parameters.getContent().equals(RestconfDataServiceConstant.ReadData.ALL)
-                    || parameters.getContent().equals(RestconfDataServiceConstant.ReadData.CONFIG)) {
-            final QName type = node.getNodeType();
-            return Response.status(200)
-                    .entity(new NormalizedNodeContext(instanceIdentifier, node, parameters))
+        switch (readParams.content()) {
+            case ALL:
+            case CONFIG:
+                final QName type = node.getIdentifier().getNodeType();
+                return Response.status(Status.OK)
+                    .entity(NormalizedNodePayload.ofReadData(instanceIdentifier, node, queryParams))
                     .header("ETag", '"' + type.getModule().getRevision().map(Revision::toString).orElse(null)
                         + "-" + type.getLocalName() + '"')
                     .header("Last-Modified", FORMATTER.format(LocalDateTime.now(Clock.systemUTC())))
                     .build();
+            default:
+                return Response.status(Status.OK)
+                    .entity(NormalizedNodePayload.ofReadData(instanceIdentifier, node, queryParams))
+                    .build();
         }
-
-        return Response.status(200).entity(new NormalizedNodeContext(instanceIdentifier, node, parameters)).build();
-    }
-
-    /**
-     * Read specific type of data from data store via transaction and if identifier read data from
-     * streams then put streams from actual schema context to datastore.
-     *
-     * @param identifier    identifier of data to read
-     * @param content       type of data to read (config, state, all)
-     * @param strategy      {@link RestconfStrategy} - object that perform the actual DS operations
-     * @param withDefa      value of with-defaults parameter
-     * @param schemaContext schema context
-     * @param uriInfo       uri info
-     * @return {@link NormalizedNode}
-     */
-    private NormalizedNode<?, ?> readData(final String identifier, final String content,
-            final RestconfStrategy strategy, final String withDefa, final EffectiveModelContext schemaContext,
-            final UriInfo uriInfo) {
-        if (identifier != null && identifier.contains(STREAMS_PATH) && !identifier.contains(STREAM_PATH_PART)) {
-            createAllYangNotificationStreams(strategy, schemaContext, uriInfo);
-        }
-        return ReadDataTransactionUtil.readData(content, strategy, withDefa, schemaContext);
     }
 
-    private void createAllYangNotificationStreams(final RestconfStrategy strategy,
-            final EffectiveModelContext schemaContext, final UriInfo uriInfo) {
-        strategy.prepareReadWriteExecution();
-        final boolean exist = checkExist(schemaContext, strategy);
-
+    private void createAllYangNotificationStreams(final EffectiveModelContext schemaContext, final UriInfo uriInfo) {
+        final DOMDataTreeWriteTransaction transaction = dataBroker.newWriteOnlyTransaction();
         for (final NotificationDefinition notificationDefinition : schemaContext.getNotifications()) {
-            final NotificationListenerAdapter notifiStreamXML =
+            writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
                 CreateStreamUtil.createYangNotifiStream(notificationDefinition, schemaContext,
-                    NotificationOutputType.XML);
-            final NotificationListenerAdapter notifiStreamJSON =
+                    NotificationOutputType.XML));
+            writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
                 CreateStreamUtil.createYangNotifiStream(notificationDefinition, schemaContext,
-                    NotificationOutputType.JSON);
-            writeNotificationStreamToDatastore(schemaContext, uriInfo, strategy, exist, notifiStreamXML);
-            writeNotificationStreamToDatastore(schemaContext, uriInfo, strategy, exist, notifiStreamJSON);
+                    NotificationOutputType.JSON));
         }
         try {
-            strategy.commit().get();
+            transaction.commit().get();
         } catch (final InterruptedException | ExecutionException e) {
             throw new RestconfDocumentedException("Problem while putting data to DS.", e);
         }
     }
 
     private void writeNotificationStreamToDatastore(final EffectiveModelContext schemaContext,
-            final UriInfo uriInfo, final RestconfStrategy strategy, final boolean exist,
-            final NotificationListenerAdapter listener) {
+            final UriInfo uriInfo, final DOMDataTreeWriteOperations tx, final NotificationListenerAdapter listener) {
         final URI uri = streamUtils.prepareUriByStreamName(uriInfo, listener.getStreamName());
-        final NormalizedNode<?, ?> mapToStreams =
-            RestconfMappingNodeUtil.mapYangNotificationStreamByIetfRestconfMonitoring(
-                listener.getSchemaPath().getLastComponent(), schemaContext.getNotifications(), null,
-                listener.getOutputType(), uri, SubscribeToStreamUtil.getMonitoringModule(schemaContext), exist);
-        writeDataToDS(schemaContext,
-            listener.getSchemaPath().getLastComponent().getLocalName(), strategy, exist, mapToStreams);
-    }
+        final MapEntryNode mapToStreams = RestconfMappingNodeUtil.mapYangNotificationStreamByIetfRestconfMonitoring(
+                listener.getSchemaPath().lastNodeIdentifier(), schemaContext.getNotifications(), null,
+                listener.getOutputType(), uri);
 
-    private static boolean checkExist(final EffectiveModelContext schemaContext, final RestconfStrategy strategy) {
-        try {
-            return strategy.exists(LogicalDatastoreType.OPERATIONAL,
-                IdentifierCodec.deserialize(Rfc8040.MonitoringModule.PATH_TO_STREAMS, schemaContext)).get();
-        } catch (final InterruptedException | ExecutionException exception) {
-            throw new RestconfDocumentedException("Problem while checking data if exists", exception);
-        }
-    }
-
-    private static void writeDataToDS(final EffectiveModelContext schemaContext, final String name,
-                                      final RestconfStrategy strategy, final boolean exist,
-                                      final NormalizedNode<?, ?> mapToStreams) {
-        final String pathId;
-        if (exist) {
-            pathId = Rfc8040.MonitoringModule.PATH_TO_STREAM_WITHOUT_KEY + name;
-        } else {
-            pathId = Rfc8040.MonitoringModule.PATH_TO_STREAMS;
-        }
-        strategy.merge(LogicalDatastoreType.OPERATIONAL, IdentifierCodec.deserialize(pathId, schemaContext),
-            mapToStreams);
+        tx.merge(LogicalDatastoreType.OPERATIONAL,
+            Rfc8040.restconfStateStreamPath(mapToStreams.getIdentifier()), mapToStreams);
     }
 
     @Override
-    public Response putData(final String identifier, final NormalizedNodeContext payload, final UriInfo uriInfo) {
+    public Response putData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
         requireNonNull(payload);
 
-        final QueryParams checkedParms = checkQueryParameters(uriInfo);
-
-        final InstanceIdentifierContext<? extends SchemaNode> iid = payload.getInstanceIdentifierContext();
+        final WriteDataParams params = QueryParams.newWriteDataParams(uriInfo);
 
-        PutDataTransactionUtil.validInputData(iid.getSchemaNode(), payload);
-        PutDataTransactionUtil.validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
-        PutDataTransactionUtil.validateListKeysEqualityInPayloadAndUri(payload);
-
-        final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
-        final EffectiveModelContext ref = mountPoint == null
-                ? this.schemaContextHandler.get()
-                : mountPoint.getEffectiveModelContext();
-
-        final RestconfStrategy strategy = getRestconfStrategy(payload.getInstanceIdentifierContext(), mountPoint);
-        return PutDataTransactionUtil.putData(payload, ref, strategy, checkedParms.insert, checkedParms.point);
-    }
-
-    private static QueryParams checkQueryParameters(final UriInfo uriInfo) {
-        boolean insertUsed = false;
-        boolean pointUsed = false;
-        String insert = null;
-        String point = null;
-
-        for (final Entry<String, List<String>> entry : uriInfo.getQueryParameters().entrySet()) {
-            switch (entry.getKey()) {
-                case INSERT:
-                    if (!insertUsed) {
-                        insertUsed = true;
-                        insert = entry.getValue().get(0);
-                    } else {
-                        throw new RestconfDocumentedException("Insert parameter can be used only once.",
-                                RestconfError.ErrorType.PROTOCOL, RestconfError.ErrorTag.BAD_ELEMENT);
-                    }
-                    break;
-                case POINT:
-                    if (!pointUsed) {
-                        pointUsed = true;
-                        point = entry.getValue().get(0);
-                    } else {
-                        throw new RestconfDocumentedException("Point parameter can be used only once.",
-                                RestconfError.ErrorType.PROTOCOL, RestconfError.ErrorTag.BAD_ELEMENT);
-                    }
-                    break;
-                default:
-                    throw new RestconfDocumentedException("Bad parameter for post: " + entry.getKey(),
-                            RestconfError.ErrorType.PROTOCOL, RestconfError.ErrorTag.BAD_ELEMENT);
-            }
-        }
+        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
 
-        checkQueryParams(insertUsed, pointUsed, insert);
-        return new QueryParams(insert, point);
-    }
+        validInputData(iid.getSchemaNode() != null, payload);
+        validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
+        validateListKeysEqualityInPayloadAndUri(payload);
 
-    private static void checkQueryParams(final boolean insertUsed, final boolean pointUsed, final String insert) {
-        if (pointUsed && !insertUsed) {
-            throw new RestconfDocumentedException("Point parameter can't be used without Insert parameter.",
-                    RestconfError.ErrorType.PROTOCOL, RestconfError.ErrorTag.BAD_ELEMENT);
-        }
-        if (pointUsed && (insert.equals("first") || insert.equals("last"))) {
-            throw new RestconfDocumentedException(
-                    "Point parameter can be used only with 'after' or 'before' values of Insert parameter.",
-                    RestconfError.ErrorType.PROTOCOL, RestconfError.ErrorTag.BAD_ELEMENT);
-        }
+        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
+        return PutDataTransactionUtil.putData(payload, iid.getSchemaContext(), strategy, params);
     }
 
     @Override
-    public Response postData(final String identifier, final NormalizedNodeContext payload, final UriInfo uriInfo) {
+    public Response postData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
         return postData(payload, uriInfo);
     }
 
     @Override
-    public Response postData(final NormalizedNodeContext payload, final UriInfo uriInfo) {
+    public Response postData(final NormalizedNodePayload payload, final UriInfo uriInfo) {
         requireNonNull(payload);
-        if (payload.getInstanceIdentifierContext().getSchemaNode() instanceof ActionDefinition) {
+        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
+        if (iid.getSchemaNode() instanceof ActionDefinition) {
             return invokeAction(payload);
         }
 
-        final QueryParams checkedParms = checkQueryParameters(uriInfo);
-        final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
-        final RestconfStrategy strategy = getRestconfStrategy(payload.getInstanceIdentifierContext(),
-                payload.getInstanceIdentifierContext().getMountPoint());
-        return PostDataTransactionUtil.postData(uriInfo, payload, strategy,
-                getSchemaContext(mountPoint), checkedParms.insert, checkedParms.point);
+        final WriteDataParams params = QueryParams.newWriteDataParams(uriInfo);
+        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
+        return PostDataTransactionUtil.postData(uriInfo, payload, strategy, iid.getSchemaContext(), params);
     }
 
     @Override
     public Response deleteData(final String identifier) {
-        final InstanceIdentifierContext<?> instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
-                identifier, this.schemaContextHandler.get(), Optional.of(this.mountPointServiceHandler.get()));
+        final InstanceIdentifierContext instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
+                identifier, schemaContextHandler.get(), Optional.of(mountPointService));
 
         final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
-        final RestconfStrategy strategy = getRestconfStrategy(instanceIdentifier, mountPoint);
-        return DeleteDataTransactionUtil.deleteData(strategy);
+        final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
+        return DeleteDataTransactionUtil.deleteData(strategy, instanceIdentifier.getInstanceIdentifier());
     }
 
     @Override
@@ -366,111 +262,219 @@ public class RestconfDataServiceImpl implements RestconfDataService {
 
     @Override
     public PatchStatusContext patchData(final PatchContext context, final UriInfo uriInfo) {
-        final DOMMountPoint mountPoint = requireNonNull(context).getInstanceIdentifierContext().getMountPoint();
-        final RestconfStrategy strategy = getRestconfStrategy(context.getInstanceIdentifierContext(), mountPoint);
-        return PatchDataTransactionUtil.patchData(context, strategy, getSchemaContext(mountPoint));
+        final InstanceIdentifierContext iid = RestconfDocumentedException.throwIfNull(context,
+            ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE, "No patch documented provided")
+            .getInstanceIdentifierContext();
+        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
+        return PatchDataTransactionUtil.patchData(context, strategy, iid.getSchemaContext());
     }
 
     @Override
-    public Response patchData(final String identifier, final NormalizedNodeContext payload, final UriInfo uriInfo) {
+    public Response patchData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
         requireNonNull(payload);
 
-        final InstanceIdentifierContext<? extends SchemaNode> iid = payload
-                .getInstanceIdentifierContext();
+        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
+        validInputData(iid.getSchemaNode() != null, payload);
+        validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
+        validateListKeysEqualityInPayloadAndUri(payload);
 
-        PutDataTransactionUtil.validInputData(iid.getSchemaNode(), payload);
-        PutDataTransactionUtil.validTopLevelNodeName(iid.getInstanceIdentifier(), payload);
-        PutDataTransactionUtil.validateListKeysEqualityInPayloadAndUri(payload);
-
-        final DOMMountPoint mountPoint = payload.getInstanceIdentifierContext().getMountPoint();
-        final EffectiveModelContext ref = mountPoint == null
-                ? this.schemaContextHandler.get()
-                : mountPoint.getEffectiveModelContext();
-        final RestconfStrategy strategy = getRestconfStrategy(payload.getInstanceIdentifierContext(), mountPoint);
-
-        return PlainPatchDataTransactionUtil.patchData(payload, strategy, ref);
-    }
-
-    private EffectiveModelContext getSchemaContext(final DOMMountPoint mountPoint) {
-        return mountPoint == null ? schemaContextHandler.get() : mountPoint.getEffectiveModelContext();
-    }
-
-    public synchronized RestconfStrategy getRestconfStrategy(final InstanceIdentifierContext<?> instanceIdentifier,
-                                                final DOMMountPoint mountPoint) {
-        if (mountPoint != null) {
-            final Optional<NetconfDataTreeService> service = mountPoint.getService(NetconfDataTreeService.class);
-            if (service.isPresent()) {
-                return new NetconfRestconfStrategy(service.get(), instanceIdentifier);
-            }
-        }
-        final TransactionChainHandler transactionChain = mountPoint == null
-                ? transactionChainHandler : transactionChainOfMountPoint(mountPoint);
-        return new MdsalRestconfStrategy(instanceIdentifier, transactionChain);
+        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
+        return PlainPatchDataTransactionUtil.patchData(payload, strategy, iid.getSchemaContext());
     }
 
-    /**
-     * Prepare transaction chain to access data of mount point.
-     *
-     * @param mountPoint mount point reference
-     * @return {@link TransactionChainHandler}
-     */
-    private static TransactionChainHandler transactionChainOfMountPoint(final @NonNull DOMMountPoint mountPoint) {
-        final Optional<DOMDataBroker> domDataBrokerService = mountPoint.getService(DOMDataBroker.class);
-        if (domDataBrokerService.isPresent()) {
-            return new TransactionChainHandler(domDataBrokerService.get());
+    // FIXME: why is this synchronized?
+    public synchronized RestconfStrategy getRestconfStrategy(final DOMMountPoint mountPoint) {
+        if (mountPoint == null) {
+            return restconfStrategy;
         }
 
-        final String errMsg = "DOM data broker service isn't available for mount point " + mountPoint.getIdentifier();
-        LOG.warn(errMsg);
-        throw new RestconfDocumentedException(errMsg);
+        return RestconfStrategy.forMountPoint(mountPoint).orElseThrow(() -> {
+            LOG.warn("Mount point {} does not expose a suitable access interface", mountPoint.getIdentifier());
+            return new RestconfDocumentedException("Could not find a supported access interface in mount point "
+                + mountPoint.getIdentifier());
+        });
     }
 
     /**
      * Invoke Action operation.
      *
-     * @param payload {@link NormalizedNodeContext} - the body of the operation
-     * @return {@link NormalizedNodeContext} wrapped in {@link Response}
+     * @param payload {@link NormalizedNodePayload} - the body of the operation
+     * @return {@link NormalizedNodePayload} wrapped in {@link Response}
      */
-    public Response invokeAction(final NormalizedNodeContext payload) {
-        final InstanceIdentifierContext<?> context = payload.getInstanceIdentifierContext();
-        final DOMMountPoint mountPoint = context.getMountPoint();
-        final SchemaPath schemaPath = context.getSchemaNode().getPath();
+    public Response invokeAction(final NormalizedNodePayload payload) {
+        final InstanceIdentifierContext context = payload.getInstanceIdentifierContext();
         final YangInstanceIdentifier yangIIdContext = context.getInstanceIdentifier();
-        final NormalizedNode<?, ?> data = payload.getData();
+        final NormalizedNode data = payload.getData();
 
-        if (yangIIdContext.isEmpty() && !RestconfDataServiceConstant.NETCONF_BASE_QNAME.equals(data.getNodeType())) {
+        if (yangIIdContext.isEmpty() && !NETCONF_BASE_QNAME.equals(data.getIdentifier().getNodeType())) {
             throw new RestconfDocumentedException("Instance identifier need to contain at least one path argument",
                 ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
         }
 
+        final DOMMountPoint mountPoint = context.getMountPoint();
+        final Absolute schemaPath = context.inference().toSchemaInferenceStack().toSchemaNodeIdentifier();
         final DOMActionResult response;
-        final EffectiveModelContext schemaContextRef;
         if (mountPoint != null) {
-            response = RestconfInvokeOperationsUtil.invokeActionViaMountPoint(mountPoint, (ContainerNode) data,
-                schemaPath, yangIIdContext);
-            schemaContextRef = mountPoint.getEffectiveModelContext();
+            response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, mountPoint);
         } else {
-            response = RestconfInvokeOperationsUtil.invokeAction((ContainerNode) data, schemaPath,
-                this.actionServiceHandler, yangIIdContext);
-            schemaContextRef = this.schemaContextHandler.get();
+            response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, actionService);
         }
-        final DOMActionResult result = RestconfInvokeOperationsUtil.checkActionResponse(response);
+        final DOMActionResult result = checkActionResponse(response);
 
-        ActionDefinition resultNodeSchema = null;
         ContainerNode resultData = null;
         if (result != null) {
-            final Optional<ContainerNode> optOutput = result.getOutput();
-            if (optOutput.isPresent()) {
-                resultData = optOutput.get();
-                resultNodeSchema = (ActionDefinition) context.getSchemaNode();
+            resultData = result.getOutput().orElse(null);
+        }
+
+        if (resultData != null && resultData.isEmpty()) {
+            return Response.status(Status.NO_CONTENT).build();
+        }
+
+        return Response.status(Status.OK)
+            .entity(NormalizedNodePayload.ofNullable(context, resultData))
+            .build();
+    }
+
+    /**
+     * Invoking Action via mount point.
+     *
+     * @param mountPoint mount point
+     * @param data input data
+     * @param schemaPath schema path of data
+     * @return {@link DOMActionResult}
+     */
+    private static DOMActionResult invokeAction(final ContainerNode data,
+            final Absolute schemaPath, final YangInstanceIdentifier yangIId, final DOMMountPoint mountPoint) {
+        return invokeAction(data, schemaPath, yangIId, mountPoint.getService(DOMActionService.class)
+            .orElseThrow(() -> new RestconfDocumentedException("DomAction service is missing.")));
+    }
+
+    /**
+     * Invoke Action via ActionServiceHandler.
+     *
+     * @param data input data
+     * @param yangIId invocation context
+     * @param schemaPath schema path of data
+     * @param actionService action service to invoke action
+     * @return {@link DOMActionResult}
+     */
+    // FIXME: NETCONF-718: we should be returning a future here
+    private static DOMActionResult invokeAction(final ContainerNode data, final Absolute schemaPath,
+            final YangInstanceIdentifier yangIId, final DOMActionService actionService) {
+        return RestconfInvokeOperationsServiceImpl.checkedGet(Futures.catching(actionService.invokeAction(
+            schemaPath, new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL, yangIId.getParent()), data),
+            DOMActionException.class,
+            cause -> new SimpleDOMActionResult(List.of(RpcResultBuilder.newError(
+                ErrorType.RPC, ErrorTag.OPERATION_FAILED, cause.getMessage()))),
+            MoreExecutors.directExecutor()));
+    }
+
+    /**
+     * Check the validity of the result.
+     *
+     * @param response response of Action
+     * @return {@link DOMActionResult} result
+     */
+    private static DOMActionResult checkActionResponse(final DOMActionResult response) {
+        if (response == null) {
+            return null;
+        }
+
+        try {
+            if (response.getErrors().isEmpty()) {
+                return response;
+            }
+            LOG.debug("InvokeAction Error Message {}", response.getErrors());
+            throw new RestconfDocumentedException("InvokeAction Error Message ", null, response.getErrors());
+        } catch (final CancellationException e) {
+            final String errMsg = "The Action Operation was cancelled while executing.";
+            LOG.debug("Cancel Execution: {}", errMsg, e);
+            throw new RestconfDocumentedException(errMsg, ErrorType.RPC, ErrorTag.PARTIAL_OPERATION, e);
+        }
+    }
+
+    /**
+     * Valid input data based on presence of a schema node.
+     *
+     * @param haveSchemaNode true if there is an underlying schema node
+     * @param payload    input data
+     */
+    @VisibleForTesting
+    public static void validInputData(final boolean haveSchemaNode, final NormalizedNodePayload payload) {
+        final boolean haveData = payload.getData() != null;
+        if (haveSchemaNode) {
+            if (!haveData) {
+                throw new RestconfDocumentedException("Input is required.", ErrorType.PROTOCOL,
+                    ErrorTag.MALFORMED_MESSAGE);
+            }
+        } else if (haveData) {
+            throw new RestconfDocumentedException("No input expected.", ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
+        }
+    }
+
+    /**
+     * Valid top level node name.
+     *
+     * @param path    path of node
+     * @param payload data
+     */
+    @VisibleForTesting
+    public static void validTopLevelNodeName(final YangInstanceIdentifier path, final NormalizedNodePayload payload) {
+        final QName dataNodeType = payload.getData().getIdentifier().getNodeType();
+        if (path.isEmpty()) {
+            if (!NETCONF_BASE_QNAME.equals(dataNodeType)) {
+                throw new RestconfDocumentedException("Instance identifier has to contain at least one path argument",
+                        ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
+            }
+        } else {
+            final String identifierName = path.getLastPathArgument().getNodeType().getLocalName();
+            final String payloadName = dataNodeType.getLocalName();
+            if (!payloadName.equals(identifierName)) {
+                throw new RestconfDocumentedException(
+                        "Payload name (" + payloadName + ") is different from identifier name (" + identifierName + ")",
+                        ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
             }
         }
+    }
+
 
-        if (resultData != null && resultData.getValue().isEmpty()) {
-            throw new WebApplicationException(Response.Status.NO_CONTENT);
+    /**
+     * Validates whether keys in {@code payload} are equal to values of keys in
+     * {@code iiWithData} for list schema node.
+     *
+     * @throws RestconfDocumentedException if key values or key count in payload and URI isn't equal
+     */
+    @VisibleForTesting
+    public static void validateListKeysEqualityInPayloadAndUri(final NormalizedNodePayload payload) {
+        final InstanceIdentifierContext iiWithData = payload.getInstanceIdentifierContext();
+        final PathArgument lastPathArgument = iiWithData.getInstanceIdentifier().getLastPathArgument();
+        final SchemaNode schemaNode = iiWithData.getSchemaNode();
+        final NormalizedNode data = payload.getData();
+        if (schemaNode instanceof ListSchemaNode) {
+            final List<QName> keyDefinitions = ((ListSchemaNode) schemaNode).getKeyDefinition();
+            if (lastPathArgument instanceof NodeIdentifierWithPredicates && data instanceof MapEntryNode) {
+                final Map<QName, Object> uriKeyValues = ((NodeIdentifierWithPredicates) lastPathArgument).asMap();
+                isEqualUriAndPayloadKeyValues(uriKeyValues, (MapEntryNode) data, keyDefinitions);
+            }
         }
+    }
 
-        return Response.status(200).entity(new NormalizedNodeContext(new InstanceIdentifierContext<>(yangIIdContext,
-                resultNodeSchema, mountPoint, schemaContextRef), resultData)).build();
+    private static void isEqualUriAndPayloadKeyValues(final Map<QName, Object> uriKeyValues, final MapEntryNode payload,
+            final List<QName> keyDefinitions) {
+        final Map<QName, Object> mutableCopyUriKeyValues = new HashMap<>(uriKeyValues);
+        for (final QName keyDefinition : keyDefinitions) {
+            final Object uriKeyValue = RestconfDocumentedException.throwIfNull(
+                    mutableCopyUriKeyValues.remove(keyDefinition), ErrorType.PROTOCOL, ErrorTag.DATA_MISSING,
+                    "Missing key %s in URI.", keyDefinition);
+
+            final Object dataKeyValue = payload.getIdentifier().getValue(keyDefinition);
+
+            if (!uriKeyValue.equals(dataKeyValue)) {
+                final String errMsg = "The value '" + uriKeyValue + "' for key '" + keyDefinition.getLocalName()
+                        + "' specified in the URI doesn't match the value '" + dataKeyValue
+                        + "' specified in the message body. ";
+                throw new RestconfDocumentedException(errMsg, ErrorType.PROTOCOL, ErrorTag.INVALID_VALUE);
+            }
+        }
     }
 }