Introduce restconf.server.{api,spi,mdsal}
[netconf.git] / restconf / restconf-nb / src / main / java / org / opendaylight / restconf / nb / rfc8040 / rests / services / impl / RestconfDataServiceImpl.java
index 94878fee9918d5d2d385d22c467a42257628fbec..22e986cfd3657174ce82b1f475af494e24667a7a 100644 (file)
 package org.opendaylight.restconf.nb.rfc8040.rests.services.impl;
 
 import static java.util.Objects.requireNonNull;
-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;
-import static org.opendaylight.restconf.nb.rfc8040.rests.utils.RestconfStreamsConstants.STREAM_LOCATION_PATH_PART;
-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.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.MoreExecutors;
+import java.io.IOException;
+import java.io.InputStream;
 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;
-import java.util.Optional;
 import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.Encoded;
+import javax.ws.rs.GET;
+import javax.ws.rs.PATCH;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.container.AsyncResponse;
+import javax.ws.rs.container.Suspended;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
 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.DOMMountPoint;
-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.errors.RestconfDocumentedException;
+import org.opendaylight.restconf.common.errors.RestconfFuture;
+import org.opendaylight.restconf.common.errors.SettableRestconfFuture;
 import org.opendaylight.restconf.common.patch.PatchContext;
 import org.opendaylight.restconf.common.patch.PatchStatusContext;
+import org.opendaylight.restconf.common.patch.PatchStatusEntity;
+import org.opendaylight.restconf.nb.rfc8040.MediaTypes;
 import org.opendaylight.restconf.nb.rfc8040.ReadDataParams;
-import org.opendaylight.restconf.nb.rfc8040.WriteDataParams;
+import org.opendaylight.restconf.nb.rfc8040.databind.ChildBody;
 import org.opendaylight.restconf.nb.rfc8040.databind.DatabindProvider;
+import org.opendaylight.restconf.nb.rfc8040.databind.JsonChildBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.JsonOperationInputBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.JsonPatchBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.JsonResourceBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.OperationInputBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.PatchBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.ResourceBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.XmlChildBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.XmlOperationInputBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.XmlPatchBody;
+import org.opendaylight.restconf.nb.rfc8040.databind.XmlResourceBody;
 import org.opendaylight.restconf.nb.rfc8040.databind.jaxrs.QueryParams;
+import org.opendaylight.restconf.nb.rfc8040.legacy.ErrorTags;
+import org.opendaylight.restconf.nb.rfc8040.legacy.InstanceIdentifierContext;
 import org.opendaylight.restconf.nb.rfc8040.legacy.NormalizedNodePayload;
-import org.opendaylight.restconf.nb.rfc8040.legacy.QueryParameters;
-import org.opendaylight.restconf.nb.rfc8040.monitoring.RestconfStateStreams;
-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.RestconfStrategy;
-import org.opendaylight.restconf.nb.rfc8040.rests.utils.DeleteDataTransactionUtil;
-import org.opendaylight.restconf.nb.rfc8040.rests.utils.PatchDataTransactionUtil;
-import org.opendaylight.restconf.nb.rfc8040.rests.utils.PlainPatchDataTransactionUtil;
-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.RestconfStreamsConstants;
-import org.opendaylight.restconf.nb.rfc8040.streams.StreamsConfiguration;
-import org.opendaylight.restconf.nb.rfc8040.streams.listeners.ListenersBroker;
-import org.opendaylight.restconf.nb.rfc8040.streams.listeners.NotificationListenerAdapter;
-import org.opendaylight.restconf.nb.rfc8040.utils.parser.ParserIdentifier;
-import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.restconf.rev170126.restconf.restconf.Data;
-import org.opendaylight.yang.gen.v1.urn.sal.restconf.event.subscription.rev140708.NotificationOutputTypeGrouping.NotificationOutputType;
+import org.opendaylight.restconf.nb.rfc8040.rests.transactions.RestconfStrategy.CreateOrReplaceResult;
+import org.opendaylight.restconf.nb.rfc8040.utils.parser.IdentifierCodec;
+import org.opendaylight.yangtools.yang.common.Empty;
 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.MapNode;
 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.SchemaNode;
-import org.opendaylight.yangtools.yang.model.api.stmt.NotificationEffectiveStatement;
+import org.opendaylight.yangtools.yang.model.api.SchemaContext;
 import org.opendaylight.yangtools.yang.model.api.stmt.SchemaNodeIdentifier.Absolute;
+import org.opendaylight.yangtools.yang.model.util.SchemaInferenceStack.Inference;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Implementation of {@link RestconfDataService}.
+ * The "{+restconf}/data" subtree represents the datastore resource type, which is a collection of configuration data
+ * and state data nodes.
  */
 @Path("/")
-public class RestconfDataServiceImpl implements RestconfDataService {
+public final class RestconfDataServiceImpl {
     private static final Logger LOG = LoggerFactory.getLogger(RestconfDataServiceImpl.class);
     private static final DateTimeFormatter FORMATTER = DateTimeFormatter.ofPattern("yyyy-MMM-dd HH:mm:ss");
 
-    private final RestconfStreamsSubscriptionService delegRestconfSubscrService;
     private final DatabindProvider databindProvider;
-    private final MdsalRestconfStrategy restconfStrategy;
-    private final DOMMountPointService mountPointService;
-    private final SubscribeToStreamUtil streamUtils;
     private final DOMActionService actionService;
-    private final DOMDataBroker dataBroker;
+    private final MdsalRestconfServer server;
 
-    public RestconfDataServiceImpl(final DatabindProvider databindProvider,
-            final DOMDataBroker dataBroker, final DOMMountPointService  mountPointService,
-            final RestconfStreamsSubscriptionService delegRestconfSubscrService,
-            final DOMActionService actionService, final StreamsConfiguration configuration) {
+    public RestconfDataServiceImpl(final DatabindProvider databindProvider, final MdsalRestconfServer server,
+            final DOMActionService actionService) {
         this.databindProvider = requireNonNull(databindProvider);
-        this.dataBroker = requireNonNull(dataBroker);
-        restconfStrategy = new MdsalRestconfStrategy(dataBroker);
-        this.mountPointService = requireNonNull(mountPointService);
-        this.delegRestconfSubscrService = requireNonNull(delegRestconfSubscrService);
+        this.server = requireNonNull(server);
         this.actionService = requireNonNull(actionService);
-        streamUtils = configuration.useSSE() ? SubscribeToStreamUtil.serverSentEvents()
-                : SubscribeToStreamUtil.webSockets();
     }
 
-    @Override
-    public Response readData(final UriInfo uriInfo) {
-        return readData(null, uriInfo);
+    /**
+     * Get target data resource from data root.
+     *
+     * @param uriInfo URI info
+     * @return {@link NormalizedNodePayload}
+     */
+    @GET
+    @Path("/data")
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_JSON,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public Response readData(@Context final UriInfo uriInfo) {
+        final var readParams = QueryParams.newReadDataParams(uriInfo);
+        return readData(server.bindRequestRoot(), readParams);
     }
 
-    @Override
-    public Response readData(final String identifier, final UriInfo uriInfo) {
-        final ReadDataParams readParams = QueryParams.newReadDataParams(uriInfo);
-
-        final EffectiveModelContext schemaContextRef = databindProvider.currentContext().modelContext();
-        // FIXME: go through
-        final InstanceIdentifierContext instanceIdentifier = ParserIdentifier.toInstanceIdentifier(
-                identifier, schemaContextRef, Optional.of(mountPointService));
-        final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
-
-        // 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);
-        }
+    /**
+     * Get target data resource.
+     *
+     * @param identifier path to target
+     * @param uriInfo URI info
+     * @return {@link NormalizedNodePayload}
+     */
+    @GET
+    @Path("/data/{identifier:.+}")
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_JSON,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public Response readData(@Encoded @PathParam("identifier") final String identifier,
+            @Context final UriInfo uriInfo) {
+        final var readParams = QueryParams.newReadDataParams(uriInfo);
+        return readData(server.bindRequestPath(identifier), readParams);
+    }
 
-        final QueryParameters queryParams = QueryParams.newQueryParameters(readParams, instanceIdentifier);
-        final List<YangInstanceIdentifier> fieldPaths = queryParams.fieldPaths();
-        final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
+    private Response readData(final InstanceIdentifierContext reqPath, final ReadDataParams readParams) {
+        final var queryParams = QueryParams.newQueryParameters(readParams, reqPath);
+        final var fieldPaths = queryParams.fieldPaths();
+        final var strategy = server.getRestconfStrategy(reqPath.getSchemaContext(), reqPath.getMountPoint());
         final NormalizedNode node;
         if (fieldPaths != null && !fieldPaths.isEmpty()) {
-            node = ReadDataTransactionUtil.readData(readParams.content(), instanceIdentifier.getInstanceIdentifier(),
-                    strategy, readParams.withDefaults(), schemaContextRef, fieldPaths);
+            node = strategy.readData(readParams.content(), reqPath.getInstanceIdentifier(),
+                readParams.withDefaults(), 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.body();
-            final String streamName = value.substring(value.indexOf(NOTIFICATION_STREAM + '/'));
-            delegRestconfSubscrService.subscribeToStream(streamName, uriInfo);
+            node = strategy.readData(readParams.content(), reqPath.getInstanceIdentifier(),
+                readParams.withDefaults());
         }
         if (node == null) {
             throw new RestconfDocumentedException(
-                    "Request could not be completed because the relevant data model content does not exist",
-                    ErrorType.PROTOCOL, ErrorTag.DATA_MISSING);
+                "Request could not be completed because the relevant data model content does not exist",
+                ErrorType.PROTOCOL, ErrorTag.DATA_MISSING);
         }
 
         return switch (readParams.content()) {
             case ALL, CONFIG -> {
                 final QName type = node.name().getNodeType();
                 yield Response.status(Status.OK)
-                    .entity(NormalizedNodePayload.ofReadData(instanceIdentifier, node, queryParams))
+                    .entity(new NormalizedNodePayload(reqPath.inference(), node, queryParams))
                     .header("ETag", '"' + type.getModule().getRevision().map(Revision::toString).orElse(null) + "-"
                         + type.getLocalName() + '"')
                     .header("Last-Modified", FORMATTER.format(LocalDateTime.now(Clock.systemUTC())))
                     .build();
             }
             case NONCONFIG -> Response.status(Status.OK)
-                .entity(NormalizedNodePayload.ofReadData(instanceIdentifier, node, queryParams))
+                .entity(new NormalizedNodePayload(reqPath.inference(), node, queryParams))
                 .build();
         };
     }
 
-    private void createAllYangNotificationStreams(final EffectiveModelContext schemaContext, final UriInfo uriInfo) {
-        final var transaction = dataBroker.newWriteOnlyTransaction();
-
-        for (var module : schemaContext.getModuleStatements().values()) {
-            final var moduleName = module.argument().getLocalName();
-            // Note: this handles only RFC6020 notifications
-            module.streamEffectiveSubstatements(NotificationEffectiveStatement.class).forEach(notification -> {
-                final var notifName = notification.argument();
-
-                writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
-                    createYangNotifiStream(moduleName, notifName, NotificationOutputType.XML));
-                writeNotificationStreamToDatastore(schemaContext, uriInfo, transaction,
-                    createYangNotifiStream(moduleName, notifName, NotificationOutputType.JSON));
-            });
+    /**
+     * Replace the data store.
+     *
+     * @param uriInfo request URI information
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PUT
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void putDataJSON(@Context final UriInfo uriInfo, final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonResourceBody(body)) {
+            putData(null, uriInfo, jsonBody, ar);
         }
+    }
 
-        try {
-            transaction.commit().get();
-        } catch (final InterruptedException | ExecutionException e) {
-            throw new RestconfDocumentedException("Problem while putting data to DS.", e);
+    /**
+     * Create or replace the target data resource.
+     *
+     * @param identifier path to target
+     * @param uriInfo request URI information
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PUT
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void putDataJSON(@Encoded @PathParam("identifier") final String identifier,
+            @Context final UriInfo uriInfo, final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonResourceBody(body)) {
+            putData(identifier, uriInfo, jsonBody, ar);
         }
     }
 
-    private static NotificationListenerAdapter createYangNotifiStream(final String moduleName, final QName notifName,
-            final NotificationOutputType outputType) {
-        final var streamName = createNotificationStreamName(moduleName, notifName.getLocalName(), outputType);
-        final var listenersBroker = ListenersBroker.getInstance();
-
-        final var existing = listenersBroker.notificationListenerFor(streamName);
-        return existing != null ? existing
-            : listenersBroker.registerNotificationListener(Absolute.of(notifName), streamName, outputType);
+    /**
+     * Replace the data store.
+     *
+     * @param uriInfo request URI information
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PUT
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void putDataXML(@Context final UriInfo uriInfo, final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlResourceBody(body)) {
+            putData(null, uriInfo, xmlBody, ar);
+        }
     }
 
-    private static String createNotificationStreamName(final String moduleName, final String notifName,
-            final NotificationOutputType outputType) {
-        final var sb = new StringBuilder()
-            .append(RestconfStreamsConstants.NOTIFICATION_STREAM)
-            .append('/').append(moduleName).append(':').append(notifName);
-        if (outputType != NotificationOutputType.XML) {
-            sb.append('/').append(outputType.getName());
+    /**
+     * Create or replace the target data resource.
+     *
+     * @param identifier path to target
+     * @param uriInfo request URI information
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PUT
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void putDataXML(@Encoded @PathParam("identifier") final String identifier,
+            @Context final UriInfo uriInfo, final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlResourceBody(body)) {
+            putData(identifier, uriInfo, xmlBody, ar);
         }
-        return sb.toString();
     }
 
-    private void writeNotificationStreamToDatastore(final EffectiveModelContext schemaContext,
-            final UriInfo uriInfo, final DOMDataTreeWriteOperations tx, final NotificationListenerAdapter listener) {
-        final URI uri = streamUtils.prepareUriByStreamName(uriInfo, listener.getStreamName());
-        final MapEntryNode mapToStreams = RestconfStateStreams.notificationStreamEntry(schemaContext,
-                listener.getSchemaPath().lastNodeIdentifier(), null, listener.getOutputType(), uri);
-
-        tx.merge(LogicalDatastoreType.OPERATIONAL,
-            RestconfStateStreams.restconfStateStreamPath(mapToStreams.name()), mapToStreams);
+    private void putData(final @Nullable String identifier, final UriInfo uriInfo, final ResourceBody body,
+            final AsyncResponse ar) {
+        final var reqPath = server.bindRequestPath(identifier);
+        final var insert = QueryParams.parseInsert(reqPath.getSchemaContext(), uriInfo);
+        final var req = bindResourceRequest(reqPath, body);
+
+        req.strategy().putData(req.path(), req.data(), insert).addCallback(new JaxRsRestconfCallback<>(ar) {
+            @Override
+            Response transform(final CreateOrReplaceResult result) {
+                return switch (result) {
+                    // Note: no Location header, as it matches the request path
+                    case CREATED -> Response.status(Status.CREATED).build();
+                    case REPLACED -> Response.noContent().build();
+                };
+            }
+        });
     }
 
-    @Override
-    public Response putData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
-        requireNonNull(payload);
-
-        final WriteDataParams params = QueryParams.newWriteDataParams(uriInfo);
-
-        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
-        final YangInstanceIdentifier path = iid.getInstanceIdentifier();
-
-        validInputData(iid.getSchemaNode() != null, payload);
-        validTopLevelNodeName(path, payload);
-        validateListKeysEqualityInPayloadAndUri(payload);
-
-        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
-        return PutDataTransactionUtil.putData(path, payload.getData(), iid.getSchemaContext(), strategy, params);
+    /**
+     * Create a top-level data resource.
+     *
+     * @param body data node for put to config DS
+     * @param uriInfo URI info
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @POST
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void postDataJSON(final InputStream body, @Context final UriInfo uriInfo,
+            @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonChildBody(body)) {
+            postData(jsonBody, uriInfo, ar);
+        }
     }
 
-    @Override
-    public Response postData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
-        return postData(payload, uriInfo);
+    /**
+     * Create a data resource in target.
+     *
+     * @param identifier path to target
+     * @param body data node for put to config DS
+     * @param uriInfo URI info
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @POST
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void postDataJSON(@Encoded @PathParam("identifier") final String identifier, final InputStream body,
+            @Context final UriInfo uriInfo, @Suspended final AsyncResponse ar) {
+        final var reqPath = server.bindRequestPath(identifier);
+        if (reqPath.getSchemaNode() instanceof ActionDefinition) {
+            try (var jsonBody = new JsonOperationInputBody(body)) {
+                invokeAction(reqPath, jsonBody, ar);
+            }
+        } else {
+            try (var jsonBody = new JsonChildBody(body)) {
+                postData(reqPath.inference(), reqPath.getInstanceIdentifier(), jsonBody, uriInfo,
+                    reqPath.getMountPoint(), ar);
+            }
+        }
     }
 
-    @Override
-    public Response postData(final NormalizedNodePayload payload, final UriInfo uriInfo) {
-        requireNonNull(payload);
-        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
-        if (iid.getSchemaNode() instanceof ActionDefinition) {
-            return invokeAction(payload);
+    /**
+     * Create a top-level data resource.
+     *
+     * @param body data node for put to config DS
+     * @param uriInfo URI info
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @POST
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void postDataXML(final InputStream body, @Context final UriInfo uriInfo, @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlChildBody(body)) {
+            postData(xmlBody, uriInfo, ar);
         }
-
-        final WriteDataParams params = QueryParams.newWriteDataParams(uriInfo);
-        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
-        return PostDataTransactionUtil.postData(uriInfo, iid.getInstanceIdentifier(), payload.getData(), strategy,
-            iid.getSchemaContext(), params);
     }
 
-    @Override
-    public Response deleteData(final String identifier) {
-        final InstanceIdentifierContext instanceIdentifier = ParserIdentifier.toInstanceIdentifier(identifier,
-            databindProvider.currentContext().modelContext(), Optional.of(mountPointService));
-
-        final DOMMountPoint mountPoint = instanceIdentifier.getMountPoint();
-        final RestconfStrategy strategy = getRestconfStrategy(mountPoint);
-        return DeleteDataTransactionUtil.deleteData(strategy, instanceIdentifier.getInstanceIdentifier());
+    /**
+     * Create a data resource in target.
+     *
+     * @param identifier path to target
+     * @param body data node for put to config DS
+     * @param uriInfo URI info
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @POST
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void postDataXML(@Encoded @PathParam("identifier") final String identifier, final InputStream body,
+            @Context final UriInfo uriInfo, @Suspended final AsyncResponse ar) {
+        final var reqPath = server.bindRequestPath(identifier);
+        if (reqPath.getSchemaNode() instanceof ActionDefinition) {
+            try (var xmlBody = new XmlOperationInputBody(body)) {
+                invokeAction(reqPath, xmlBody, ar);
+            }
+        } else {
+            try (var xmlBody = new XmlChildBody(body)) {
+                postData(reqPath.inference(), reqPath.getInstanceIdentifier(), xmlBody, uriInfo,
+                    reqPath.getMountPoint(), ar);
+            }
+        }
     }
 
-    @Override
-    public PatchStatusContext patchData(final String identifier, final PatchContext context, final UriInfo uriInfo) {
-        return patchData(context, uriInfo);
+    private void postData(final ChildBody body, final UriInfo uriInfo, final AsyncResponse ar) {
+        postData(Inference.ofDataTreePath(databindProvider.currentContext().modelContext()),
+            YangInstanceIdentifier.of(), body, uriInfo, null, ar);
     }
 
-    @Override
-    public PatchStatusContext patchData(final PatchContext context, final UriInfo uriInfo) {
-        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());
+    private void postData(final Inference inference, final YangInstanceIdentifier parentPath, final ChildBody body,
+            final UriInfo uriInfo, final @Nullable DOMMountPoint mountPoint, final AsyncResponse ar) {
+        final var modelContext = inference.getEffectiveModelContext();
+        final var insert = QueryParams.parseInsert(modelContext, uriInfo);
+        final var strategy = server.getRestconfStrategy(modelContext, mountPoint);
+        final var payload = body.toPayload(parentPath, inference);
+        final var data = payload.body();
+        final var path = concat(parentPath, payload.prefix());
+
+        strategy.postData(path, data, insert).addCallback(new JaxRsRestconfCallback<>(ar) {
+            @Override
+            Response transform(final Empty result) {
+                return Response.created(resolveLocation(uriInfo, path, modelContext, data)).build();
+            }
+        });
     }
 
-    @Override
-    public Response patchData(final String identifier, final NormalizedNodePayload payload, final UriInfo uriInfo) {
-        requireNonNull(payload);
-
-        final InstanceIdentifierContext iid = payload.getInstanceIdentifierContext();
-        final YangInstanceIdentifier path = iid.getInstanceIdentifier();
-        validInputData(iid.getSchemaNode() != null, payload);
-        validTopLevelNodeName(path, payload);
-        validateListKeysEqualityInPayloadAndUri(payload);
-
-        final RestconfStrategy strategy = getRestconfStrategy(iid.getMountPoint());
-        return PlainPatchDataTransactionUtil.patchData(path, payload.getData(), strategy, iid.getSchemaContext());
+    private static YangInstanceIdentifier concat(final YangInstanceIdentifier parent, final List<PathArgument> args) {
+        var ret = parent;
+        for (var arg : args) {
+            ret = ret.node(arg);
+        }
+        return ret;
     }
 
-    @VisibleForTesting
-    final RestconfStrategy getRestconfStrategy(final DOMMountPoint mountPoint) {
-        if (mountPoint == null) {
-            return restconfStrategy;
+    /**
+     * Get location from {@link YangInstanceIdentifier} and {@link UriInfo}.
+     *
+     * @param uriInfo       uri info
+     * @param initialPath   data path
+     * @param schemaContext reference to {@link SchemaContext}
+     * @return {@link URI}
+     */
+    private static URI resolveLocation(final UriInfo uriInfo, final YangInstanceIdentifier initialPath,
+                                       final EffectiveModelContext schemaContext, final NormalizedNode data) {
+        YangInstanceIdentifier path = initialPath;
+        if (data instanceof MapNode mapData) {
+            final var children = mapData.body();
+            if (!children.isEmpty()) {
+                path = path.node(children.iterator().next().name());
+            }
         }
 
-        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());
+        return uriInfo.getBaseUriBuilder().path("data").path(IdentifierCodec.serialize(path, schemaContext)).build();
+    }
+
+    /**
+     * Delete the target data resource.
+     *
+     * @param identifier path to target
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @DELETE
+    @Path("/data/{identifier:.+}")
+    public void deleteData(@Encoded @PathParam("identifier") final String identifier,
+            @Suspended final AsyncResponse ar) {
+        final var reqPath = server.bindRequestPath(identifier);
+        final var strategy = server.getRestconfStrategy(reqPath.getSchemaContext(), reqPath.getMountPoint());
+
+        strategy.delete(reqPath.getInstanceIdentifier()).addCallback(new JaxRsRestconfCallback<>(ar) {
+            @Override
+            Response transform(final Empty result) {
+                return Response.noContent().build();
+            }
         });
     }
 
     /**
-     * Invoke Action operation.
+     * Partially modify the target data store, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
      *
-     * @param payload {@link NormalizedNodePayload} - the body of the operation
-     * @return {@link NormalizedNodePayload} wrapped in {@link Response}
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
      */
-    public Response invokeAction(final NormalizedNodePayload payload) {
-        final InstanceIdentifierContext context = payload.getInstanceIdentifierContext();
-        final YangInstanceIdentifier yangIIdContext = context.getInstanceIdentifier();
-        final NormalizedNode data = payload.getData();
-
-        if (yangIIdContext.isEmpty() && !Data.QNAME.equals(data.name().getNodeType())) {
-            throw new RestconfDocumentedException("Instance identifier need to contain at least one path argument",
-                ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
+    @PATCH
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void plainPatchDataXML(final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlResourceBody(body)) {
+            plainPatchData(xmlBody, ar);
         }
+    }
 
-        final DOMMountPoint mountPoint = context.getMountPoint();
-        final Absolute schemaPath = context.inference().toSchemaInferenceStack().toSchemaNodeIdentifier();
-        final DOMActionResult response;
-        if (mountPoint != null) {
-            response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, mountPoint);
-        } else {
-            response = invokeAction((ContainerNode) data, schemaPath, yangIIdContext, actionService);
+    /**
+     * Partially modify the target data resource, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
+     *
+     * @param identifier path to target
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PATCH
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_XML,
+        MediaType.APPLICATION_XML,
+        MediaType.TEXT_XML
+    })
+    public void plainPatchDataXML(@Encoded @PathParam("identifier") final String identifier, final InputStream body,
+            @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlResourceBody(body)) {
+            plainPatchData(identifier, xmlBody, ar);
         }
-        final DOMActionResult result = checkActionResponse(response);
+    }
 
-        ContainerNode resultData = null;
-        if (result != null) {
-            resultData = result.getOutput().orElse(null);
+    /**
+     * Partially modify the target data store, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
+     *
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PATCH
+    @Path("/data")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void plainPatchDataJSON(final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonResourceBody(body)) {
+            plainPatchData(jsonBody, ar);
         }
+    }
 
-        if (resultData != null && resultData.isEmpty()) {
-            return Response.status(Status.NO_CONTENT).build();
+    /**
+     * Partially modify the target data resource, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
+     *
+     * @param identifier path to target
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    @PATCH
+    @Path("/data/{identifier:.+}")
+    @Consumes({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaType.APPLICATION_JSON,
+    })
+    public void plainPatchDataJSON(@Encoded @PathParam("identifier") final String identifier, final InputStream body,
+            @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonResourceBody(body)) {
+            plainPatchData(identifier, jsonBody, ar);
         }
+    }
 
-        return Response.status(Status.OK)
-            .entity(NormalizedNodePayload.ofNullable(context, resultData))
-            .build();
+    /**
+     * Partially modify the target data resource, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
+     *
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
+     */
+    private void plainPatchData(final ResourceBody body, final AsyncResponse ar) {
+        plainPatchData(server.bindRequestRoot(), body, ar);
     }
 
     /**
-     * Invoking Action via mount point.
+     * Partially modify the target data resource, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
      *
-     * @param mountPoint mount point
-     * @param data input data
-     * @param schemaPath schema path of data
-     * @return {@link DOMActionResult}
+     * @param identifier path to target
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
      */
-    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.")));
+    private void plainPatchData(final String identifier, final ResourceBody body, final AsyncResponse ar) {
+        plainPatchData(server.bindRequestPath(identifier), body, ar);
     }
 
     /**
-     * Invoke Action via ActionServiceHandler.
+     * Partially modify the target data resource, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040, section 4.6.1</a>.
      *
-     * @param data input data
-     * @param yangIId invocation context
-     * @param schemaPath schema path of data
-     * @param actionService action service to invoke action
-     * @return {@link DOMActionResult}
+     * @param reqPath path to target
+     * @param body data node for put to config DS
+     * @param ar {@link AsyncResponse} which needs to be completed
      */
-    // 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()));
+    private void plainPatchData(final InstanceIdentifierContext reqPath, final ResourceBody body,
+            final AsyncResponse ar) {
+        final var req = bindResourceRequest(reqPath, body);
+        req.strategy().merge(req.path(), req.data()).addCallback(new JaxRsRestconfCallback<>(ar) {
+            @Override
+            Response transform(final Empty result) {
+                return Response.ok().build();
+            }
+        });
+    }
+
+    private @NonNull ResourceRequest bindResourceRequest(final InstanceIdentifierContext reqPath,
+            final ResourceBody body) {
+        final var inference = reqPath.inference();
+        final var path = reqPath.getInstanceIdentifier();
+        final var data = body.toNormalizedNode(path, inference, reqPath.getSchemaNode());
+
+        return new ResourceRequest(
+            server.getRestconfStrategy(inference.getEffectiveModelContext(), reqPath.getMountPoint()),
+            path, data);
     }
 
     /**
-     * Check the validity of the result.
+     * Ordered list of edits that are applied to the target datastore by the server, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8072#section-2">RFC8072, section 2</a>.
      *
-     * @param response response of Action
-     * @return {@link DOMActionResult} result
+     * @param identifier path to target
+     * @param body YANG Patch body
+     * @param ar {@link AsyncResponse} which needs to be completed with a {@link PatchStatusContext}
      */
-    private static DOMActionResult checkActionResponse(final DOMActionResult response) {
-        if (response == null) {
-            return null;
+    @PATCH
+    @Path("/data/{identifier:.+}")
+    @Consumes(MediaTypes.APPLICATION_YANG_PATCH_XML)
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML
+    })
+    public void yangPatchDataXML(@Encoded @PathParam("identifier") final String identifier, final InputStream body,
+            @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlPatchBody(body)) {
+            yangPatchData(identifier, xmlBody, ar);
         }
+    }
 
-        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);
+    /**
+     * Ordered list of edits that are applied to the datastore by the server, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8072#section-2">RFC8072, section 2</a>.
+     *
+     * @param body YANG Patch body
+     * @param ar {@link AsyncResponse} which needs to be completed with a {@link PatchStatusContext}
+     */
+    @PATCH
+    @Path("/data")
+    @Consumes(MediaTypes.APPLICATION_YANG_PATCH_XML)
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML
+    })
+    public void yangPatchDataXML(final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var xmlBody = new XmlPatchBody(body)) {
+            yangPatchData(xmlBody, ar);
         }
     }
 
     /**
-     * Valid input data based on presence of a schema node.
+     * Ordered list of edits that are applied to the target datastore by the server, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8072#section-2">RFC8072, section 2</a>.
      *
-     * @param haveSchemaNode true if there is an underlying schema node
-     * @param payload    input data
+     * @param identifier path to target
+     * @param body YANG Patch body
+     * @param ar {@link AsyncResponse} which needs to be completed with a {@link PatchStatusContext}
      */
-    @VisibleForTesting
-    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);
+    @PATCH
+    @Path("/data/{identifier:.+}")
+    @Consumes(MediaTypes.APPLICATION_YANG_PATCH_JSON)
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML
+    })
+    public void yangPatchDataJSON(@Encoded @PathParam("identifier") final String identifier,
+            final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonPatchBody(body)) {
+            yangPatchData(identifier, jsonBody, ar);
         }
     }
 
     /**
-     * Valid top level node name.
+     * Ordered list of edits that are applied to the datastore by the server, as defined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8072#section-2">RFC8072, section 2</a>.
      *
-     * @param path    path of node
-     * @param payload data
+     * @param body YANG Patch body
+     * @param ar {@link AsyncResponse} which needs to be completed with a {@link PatchStatusContext}
      */
+    @PATCH
+    @Path("/data")
+    @Consumes(MediaTypes.APPLICATION_YANG_PATCH_JSON)
+    @Produces({
+        MediaTypes.APPLICATION_YANG_DATA_JSON,
+        MediaTypes.APPLICATION_YANG_DATA_XML
+    })
+    public void yangPatchDataJSON(final InputStream body, @Suspended final AsyncResponse ar) {
+        try (var jsonBody = new JsonPatchBody(body)) {
+            yangPatchData(jsonBody, ar);
+        }
+    }
+
+    private void yangPatchData(final @NonNull PatchBody body, final AsyncResponse ar) {
+        final var context = server.bindRequestRoot().getSchemaContext();
+        yangPatchData(context, parsePatchBody(context, YangInstanceIdentifier.of(), body), null, ar);
+    }
+
+    private void yangPatchData(final String identifier, final @NonNull PatchBody body,
+            final AsyncResponse ar) {
+        final var reqPath = server.bindRequestPath(identifier);
+        final var modelContext = reqPath.getSchemaContext();
+        yangPatchData(modelContext, parsePatchBody(modelContext, reqPath.getInstanceIdentifier(), body),
+            reqPath.getMountPoint(), ar);
+    }
+
     @VisibleForTesting
-    static void validTopLevelNodeName(final YangInstanceIdentifier path, final NormalizedNodePayload payload) {
-        final QName dataNodeType = payload.getData().name().getNodeType();
-        if (path.isEmpty()) {
-            if (!Data.QNAME.equals(dataNodeType)) {
-                throw new RestconfDocumentedException("Instance identifier has to contain at least one path argument",
-                        ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE);
-            }
+    void yangPatchData(final @NonNull EffectiveModelContext modelContext,
+            final @NonNull PatchContext patch, final @Nullable DOMMountPoint mountPoint, final AsyncResponse ar) {
+        server.getRestconfStrategy(modelContext, mountPoint).patchData(patch)
+            .addCallback(new JaxRsRestconfCallback<>(ar) {
+                @Override
+                Response transform(final PatchStatusContext result) {
+                    return Response.status(getStatusCode(result)).entity(result).build();
+                }
+            });
+    }
+
+    private static Status getStatusCode(final PatchStatusContext result) {
+        if (result.ok()) {
+            return Status.OK;
+        } else if (result.globalErrors() == null || result.globalErrors().isEmpty()) {
+            return result.editCollection().stream()
+                .filter(patchStatus -> !patchStatus.isOk() && !patchStatus.getEditErrors().isEmpty())
+                .findFirst()
+                .map(PatchStatusEntity::getEditErrors)
+                .flatMap(errors -> errors.stream().findFirst())
+                .map(error -> ErrorTags.statusOf(error.getErrorTag()))
+                .orElse(Status.INTERNAL_SERVER_ERROR);
         } 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);
-            }
+            final var error = result.globalErrors().iterator().next();
+            return ErrorTags.statusOf(error.getErrorTag());
+        }
+    }
+
+    private static @NonNull PatchContext parsePatchBody(final @NonNull EffectiveModelContext context,
+            final @NonNull YangInstanceIdentifier urlPath, final @NonNull PatchBody body) {
+        try {
+            return body.toPatchContext(context, urlPath);
+        } catch (IOException e) {
+            LOG.debug("Error parsing YANG Patch input", e);
+            throw new RestconfDocumentedException("Error parsing input: " + e.getMessage(), ErrorType.PROTOCOL,
+                    ErrorTag.MALFORMED_MESSAGE, e);
         }
     }
 
     /**
-     * Validates whether keys in {@code payload} are equal to values of keys in
-     * {@code iiWithData} for list schema node.
+     * Invoke Action operation.
      *
-     * @throws RestconfDocumentedException if key values or key count in payload and URI isn't equal
+     * @param payload {@link NormalizedNodePayload} - the body of the operation
+     * @param ar {@link AsyncResponse} which needs to be completed with a NormalizedNodePayload
      */
-    @VisibleForTesting
-    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 listSchema) {
-            final var keyDefinitions = listSchema.getKeyDefinition();
-            if (lastPathArgument instanceof NodeIdentifierWithPredicates && data instanceof MapEntryNode) {
-                final Map<QName, Object> uriKeyValues = ((NodeIdentifierWithPredicates) lastPathArgument).asMap();
-                isEqualUriAndPayloadKeyValues(uriKeyValues, (MapEntryNode) data, keyDefinitions);
-            }
+    private void invokeAction(final InstanceIdentifierContext reqPath, final OperationInputBody body,
+            final AsyncResponse ar) {
+        final var yangIIdContext = reqPath.getInstanceIdentifier();
+        final ContainerNode input;
+        try {
+            input = body.toContainerNode(reqPath.inference());
+        } catch (IOException e) {
+            LOG.debug("Error reading input", e);
+            throw new RestconfDocumentedException("Error parsing input: " + e.getMessage(), ErrorType.PROTOCOL,
+                    ErrorTag.MALFORMED_MESSAGE, e);
         }
-    }
 
-    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 var mountPoint = reqPath.getMountPoint();
+        final var inference = reqPath.inference();
+        final var schemaPath = inference.toSchemaInferenceStack().toSchemaNodeIdentifier();
+        final var response = mountPoint != null ? invokeAction(input, schemaPath, yangIIdContext, mountPoint)
+            : invokeAction(input, schemaPath, yangIIdContext, actionService);
+
+        response.addCallback(new JaxRsRestconfCallback<>(ar) {
+            @Override
+            Response transform(final DOMActionResult result) {
+                final var output = result.getOutput().orElse(null);
+                return output == null || output.isEmpty() ? Response.status(Status.NO_CONTENT).build()
+                    : Response.status(Status.OK).entity(new NormalizedNodePayload(inference, output)).build();
+            }
+        });
+    }
 
-            final Object dataKeyValue = payload.name().getValue(keyDefinition);
+    /**
+     * Invoking Action via mount point.
+     *
+     * @param mountPoint mount point
+     * @param data input data
+     * @param schemaPath schema path of data
+     * @return {@link DOMActionResult}
+     */
+    private static RestconfFuture<DOMActionResult> invokeAction(final ContainerNode data,
+            final Absolute schemaPath, final YangInstanceIdentifier yangIId, final DOMMountPoint mountPoint) {
+        final var actionService = mountPoint.getService(DOMActionService.class);
+        return actionService.isPresent() ? invokeAction(data, schemaPath, yangIId, actionService.orElseThrow())
+            : RestconfFuture.failed(new RestconfDocumentedException("DOMActionService is missing."));
+    }
 
-            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);
-            }
-        }
+    /**
+     * 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}
+     */
+    private static RestconfFuture<DOMActionResult> invokeAction(final ContainerNode data, final Absolute schemaPath,
+            final YangInstanceIdentifier yangIId, final DOMActionService actionService) {
+        final var ret = new SettableRestconfFuture<DOMActionResult>();
+
+        Futures.addCallback(actionService.invokeAction(schemaPath,
+            new DOMDataTreeIdentifier(LogicalDatastoreType.OPERATIONAL, yangIId.getParent()), data),
+            new FutureCallback<DOMActionResult>() {
+                @Override
+                public void onSuccess(final DOMActionResult result) {
+                    final var errors = result.getErrors();
+                    LOG.debug("InvokeAction Error Message {}", errors);
+                    if (errors.isEmpty()) {
+                        ret.set(result);
+                    } else {
+                        ret.setFailure(new RestconfDocumentedException("InvokeAction Error Message ", null, errors));
+                    }
+                }
+
+                @Override
+                public void onFailure(final Throwable cause) {
+                    if (cause instanceof DOMActionException) {
+                        ret.set(new SimpleDOMActionResult(List.of(RpcResultBuilder.newError(
+                            ErrorType.RPC, ErrorTag.OPERATION_FAILED, cause.getMessage()))));
+                    } else if (cause instanceof RestconfDocumentedException e) {
+                        ret.setFailure(e);
+                    } else if (cause instanceof CancellationException) {
+                        ret.setFailure(new RestconfDocumentedException("Action cancelled while executing",
+                            ErrorType.RPC, ErrorTag.PARTIAL_OPERATION, cause));
+                    } else {
+                        ret.setFailure(new RestconfDocumentedException("Invocation failed", cause));
+                    }
+                }
+            }, MoreExecutors.directExecutor());
+
+        return ret;
     }
 }