Turn ApiPath into a record
[netconf.git] / restconf / restconf-nb / src / main / java / org / opendaylight / restconf / nb / rfc8040 / rests / transactions / RestconfStrategy.java
index 90d0d3729092255c2a59e35b58ccea6caf7ef96e..cb175b55d4fcd267f77e721f27b1386535161474 100644 (file)
@@ -9,29 +9,52 @@ package org.opendaylight.restconf.nb.rfc8040.rests.transactions;
 
 import static com.google.common.base.Verify.verifyNotNull;
 import static java.util.Objects.requireNonNull;
+import static org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes.fromInstanceId;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.io.CharSource;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
+import java.io.IOException;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 import java.util.Optional;
+import java.util.concurrent.CancellationException;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.NonNullByDefault;
 import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.mdsal.common.api.CommitInfo;
 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.DOMMountPoint;
+import org.opendaylight.mdsal.dom.api.DOMMountPointService;
 import org.opendaylight.mdsal.dom.api.DOMRpcResult;
 import org.opendaylight.mdsal.dom.api.DOMRpcService;
+import org.opendaylight.mdsal.dom.api.DOMSchemaService;
+import org.opendaylight.mdsal.dom.api.DOMSchemaService.YangTextSourceExtension;
 import org.opendaylight.mdsal.dom.api.DOMTransactionChain;
+import org.opendaylight.mdsal.dom.spi.SimpleDOMActionResult;
 import org.opendaylight.netconf.dom.api.NetconfDataTreeService;
+import org.opendaylight.restconf.api.ApiPath;
 import org.opendaylight.restconf.api.query.ContentParam;
 import org.opendaylight.restconf.api.query.WithDefaultsParam;
 import org.opendaylight.restconf.common.errors.RestconfDocumentedException;
@@ -42,12 +65,47 @@ 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.Insert;
+import org.opendaylight.restconf.nb.rfc8040.legacy.ErrorTags;
+import org.opendaylight.restconf.nb.rfc8040.legacy.NormalizedNodePayload;
+import org.opendaylight.restconf.nb.rfc8040.legacy.QueryParameters;
+import org.opendaylight.restconf.nb.rfc8040.utils.parser.YangInstanceIdentifierSerializer;
+import org.opendaylight.restconf.server.api.ChildBody;
+import org.opendaylight.restconf.server.api.ConfigurationMetadata;
+import org.opendaylight.restconf.server.api.DataGetParams;
+import org.opendaylight.restconf.server.api.DataGetResult;
+import org.opendaylight.restconf.server.api.DataPatchPath;
+import org.opendaylight.restconf.server.api.DataPatchResult;
+import org.opendaylight.restconf.server.api.DataPostBody;
+import org.opendaylight.restconf.server.api.DataPostPath;
+import org.opendaylight.restconf.server.api.DataPostResult;
+import org.opendaylight.restconf.server.api.DataPostResult.CreateResource;
+import org.opendaylight.restconf.server.api.DataPostResult.InvokeOperation;
+import org.opendaylight.restconf.server.api.DataPutPath;
+import org.opendaylight.restconf.server.api.DataPutResult;
+import org.opendaylight.restconf.server.api.DataYangPatchResult;
+import org.opendaylight.restconf.server.api.DatabindContext;
+import org.opendaylight.restconf.server.api.OperationInputBody;
+import org.opendaylight.restconf.server.api.OperationsGetResult;
+import org.opendaylight.restconf.server.api.OperationsPostPath;
+import org.opendaylight.restconf.server.api.OperationsPostResult;
+import org.opendaylight.restconf.server.api.PatchBody;
+import org.opendaylight.restconf.server.api.ResourceBody;
+import org.opendaylight.restconf.server.spi.ApiPathNormalizer;
+import org.opendaylight.restconf.server.spi.ApiPathNormalizer.InstanceReference;
+import org.opendaylight.restconf.server.spi.ApiPathNormalizer.Path.Action;
+import org.opendaylight.restconf.server.spi.ApiPathNormalizer.Path.Data;
+import org.opendaylight.restconf.server.spi.ApiPathNormalizer.Path.Rpc;
+import org.opendaylight.restconf.server.spi.OperationInput;
+import org.opendaylight.restconf.server.spi.RpcImplementation;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.netconf.with.defaults.rev110601.WithDefaultsMode;
 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.QNameModule;
+import org.opendaylight.yangtools.yang.common.Revision;
+import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
+import org.opendaylight.yangtools.yang.common.XMLNamespace;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifier;
 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier.NodeIdentifierWithPredicates;
@@ -70,16 +128,23 @@ import org.opendaylight.yangtools.yang.data.api.schema.UserMapNode;
 import org.opendaylight.yangtools.yang.data.api.schema.builder.CollectionNodeBuilder;
 import org.opendaylight.yangtools.yang.data.api.schema.builder.DataContainerNodeBuilder;
 import org.opendaylight.yangtools.yang.data.api.schema.builder.NormalizedNodeContainerBuilder;
-import org.opendaylight.yangtools.yang.data.impl.schema.Builders;
-import org.opendaylight.yangtools.yang.data.impl.schema.ImmutableNodes;
+import org.opendaylight.yangtools.yang.data.spi.node.ImmutableNodes;
 import org.opendaylight.yangtools.yang.data.util.DataSchemaContext;
-import org.opendaylight.yangtools.yang.data.util.DataSchemaContextTree;
 import org.opendaylight.yangtools.yang.model.api.DataSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
 import org.opendaylight.yangtools.yang.model.api.LeafListSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.LeafSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.ListSchemaNode;
 import org.opendaylight.yangtools.yang.model.api.RpcDefinition;
+import org.opendaylight.yangtools.yang.model.api.source.SourceIdentifier;
+import org.opendaylight.yangtools.yang.model.api.source.SourceRepresentation;
+import org.opendaylight.yangtools.yang.model.api.source.YangTextSource;
+import org.opendaylight.yangtools.yang.model.api.source.YinTextSource;
+import org.opendaylight.yangtools.yang.model.api.stmt.ModuleEffectiveStatement;
+import org.opendaylight.yangtools.yang.model.api.stmt.RpcEffectiveStatement;
+import org.opendaylight.yangtools.yang.model.api.stmt.SubmoduleEffectiveStatement;
+import org.opendaylight.yangtools.yang.model.util.SchemaInferenceStack;
+import org.opendaylight.yangtools.yang.model.util.SchemaInferenceStack.Inference;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,57 +157,124 @@ import org.slf4j.LoggerFactory;
 // FIXME: it seems the first three operations deal with lifecycle of a transaction, while others invoke various
 //        operations. This should be handled through proper allocation indirection.
 public abstract class RestconfStrategy {
+    @NonNullByDefault
+    public record StrategyAndPath(RestconfStrategy strategy, Data path) {
+        public StrategyAndPath {
+            requireNonNull(strategy);
+            requireNonNull(path);
+        }
+    }
+
     /**
-     * Result of a {@code PUT} request as defined in
-     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.5">RFC8040 section 4.5</a>. The definition makes it
-     * clear that the logical operation is {@code create-or-replace}.
+     * Result of a partial {@link ApiPath} lookup for the purposes of supporting {@code yang-ext:mount}-delimited mount
+     * points with possible nesting.
+     *
+     * @param strategy the strategy to use
+     * @param tail the {@link ApiPath} tail to use with the strategy
      */
-    public enum CreateOrReplaceResult {
-        /**
-         * A new resource has been created.
-         */
-        CREATED,
-        /*
-         * An existing resources has been replaced.
-         */
-        REPLACED;
+    @NonNullByDefault
+    public record StrategyAndTail(RestconfStrategy strategy, ApiPath tail) {
+        public StrategyAndTail {
+            requireNonNull(strategy);
+            requireNonNull(tail);
+        }
     }
 
     private static final Logger LOG = LoggerFactory.getLogger(RestconfStrategy.class);
-
-    private final @NonNull EffectiveModelContext modelContext;
-    private final @Nullable DOMRpcService rpcService;
-
-    RestconfStrategy(final EffectiveModelContext modelContext, final @Nullable DOMRpcService rpcService) {
-        this.modelContext = requireNonNull(modelContext);
+    private static final @NonNull DataPutResult PUT_CREATED = new DataPutResult(true);
+    private static final @NonNull DataPutResult PUT_REPLACED = new DataPutResult(false);
+    private static final @NonNull DataPatchResult PATCH_EMPTY = new DataPatchResult();
+
+    private final @NonNull ImmutableMap<QName, RpcImplementation> localRpcs;
+    private final @NonNull ApiPathNormalizer pathNormalizer;
+    private final @NonNull DatabindContext databind;
+    private final YangTextSourceExtension sourceProvider;
+    private final DOMMountPointService mountPointService;
+    private final DOMActionService actionService;
+    private final DOMRpcService rpcService;
+
+    RestconfStrategy(final DatabindContext databind, final ImmutableMap<QName, RpcImplementation> localRpcs,
+            final @Nullable DOMRpcService rpcService, final @Nullable DOMActionService actionService,
+            final YangTextSourceExtension sourceProvider, final @Nullable DOMMountPointService mountPointService) {
+        this.databind = requireNonNull(databind);
+        this.localRpcs = requireNonNull(localRpcs);
         this.rpcService = rpcService;
+        this.actionService = actionService;
+        this.sourceProvider = sourceProvider;
+        this.mountPointService = mountPointService;
+        pathNormalizer = new ApiPathNormalizer(databind);
+    }
+
+    public final @NonNull StrategyAndPath resolveStrategyPath(final ApiPath path) {
+        final var andTail = resolveStrategy(path);
+        final var strategy = andTail.strategy();
+        return new StrategyAndPath(strategy, strategy.pathNormalizer.normalizeDataPath(andTail.tail()));
     }
 
     /**
-     * Look up the appropriate strategy for a particular mount point.
+     * Resolve any and all {@code yang-ext:mount} to the target {@link StrategyAndTail}.
      *
-     * @param modelContext {@link EffectiveModelContext} of target mount point
-     * @param mountPoint Target mount point
-     * @return A strategy, or null if the mount point does not expose a supported interface
-     * @throws NullPointerException if any argument is {@code null}
+     * @param path {@link ApiPath} to resolve
+     * @return A strategy and the remaining path
+     * @throws NullPointerException if {@code path} is {@code null}
      */
-    public static @Nullable RestconfStrategy forMountPoint(final EffectiveModelContext modelContext,
-            final DOMMountPoint mountPoint) {
+    public final @NonNull StrategyAndTail resolveStrategy(final ApiPath path) {
+        var mount = path.indexOf("yang-ext", "mount");
+        if (mount == -1) {
+            return new StrategyAndTail(this, path);
+        }
+        if (mountPointService == null) {
+            throw new RestconfDocumentedException("Mount point service is not available",
+                ErrorType.APPLICATION, ErrorTag.OPERATION_FAILED);
+        }
+        final var mountPath = path.subPath(0, mount);
+        final var dataPath = pathNormalizer.normalizeDataPath(path.subPath(0, mount));
+        final var mountPoint = mountPointService.getMountPoint(dataPath.instance())
+            .orElseThrow(() -> new RestconfDocumentedException("Mount point '" + mountPath + "' does not exist",
+                ErrorType.PROTOCOL, ErrorTags.RESOURCE_DENIED_TRANSPORT));
+
+        return createStrategy(mountPath, mountPoint).resolveStrategy(path.subPath(mount + 1));
+    }
+
+    private static @NonNull RestconfStrategy createStrategy(final ApiPath mountPath, final DOMMountPoint mountPoint) {
+        final var mountSchemaService = mountPoint.getService(DOMSchemaService.class)
+            .orElseThrow(() -> new RestconfDocumentedException(
+                "Mount point '" + mountPath + "' does not expose DOMSchemaService",
+                ErrorType.PROTOCOL, ErrorTags.RESOURCE_DENIED_TRANSPORT));
+        final var mountModelContext = mountSchemaService.getGlobalContext();
+        if (mountModelContext == null) {
+            throw new RestconfDocumentedException("Mount point '" + mountPath + "' does not have any models",
+                ErrorType.PROTOCOL, ErrorTags.RESOURCE_DENIED_TRANSPORT);
+        }
+        final var mountDatabind = DatabindContext.ofModel(mountModelContext);
+        final var mountPointService = mountPoint.getService(DOMMountPointService.class).orElse(null);
         final var rpcService = mountPoint.getService(DOMRpcService.class).orElse(null);
+        final var actionService = mountPoint.getService(DOMActionService.class).orElse(null);
+        final var sourceProvider = mountPoint.getService(DOMSchemaService.class)
+            .flatMap(schema -> Optional.ofNullable(schema.extension(YangTextSourceExtension.class)))
+            .orElse(null);
 
         final var netconfService = mountPoint.getService(NetconfDataTreeService.class);
         if (netconfService.isPresent()) {
-            return new NetconfRestconfStrategy(modelContext, netconfService.orElseThrow(), rpcService);
+            return new NetconfRestconfStrategy(mountDatabind, netconfService.orElseThrow(), rpcService, actionService,
+                sourceProvider, mountPointService);
         }
         final var dataBroker = mountPoint.getService(DOMDataBroker.class);
         if (dataBroker.isPresent()) {
-            return new MdsalRestconfStrategy(modelContext, dataBroker.orElseThrow(), rpcService);
+            return new MdsalRestconfStrategy(mountDatabind, dataBroker.orElseThrow(), rpcService, actionService,
+                sourceProvider, mountPointService);
         }
-        return null;
+        LOG.warn("Mount point {} does not expose a suitable access interface", mountPath);
+        throw new RestconfDocumentedException("Could not find a supported access interface in mount point",
+            ErrorType.APPLICATION, ErrorTag.OPERATION_FAILED, mountPoint.getIdentifier());
+    }
+
+    public final @NonNull DatabindContext databind() {
+        return databind;
     }
 
     public final @NonNull EffectiveModelContext modelContext() {
-        return modelContext;
+        return databind.modelContext();
     }
 
     /**
@@ -162,17 +294,6 @@ public abstract class RestconfStrategy {
      */
     abstract ListenableFuture<Optional<NormalizedNode>> read(LogicalDatastoreType store, YangInstanceIdentifier path);
 
-    /**
-     * Read data selected using fields from the datastore.
-     *
-     * @param store the logical data store which should be modified
-     * @param path the parent data object path
-     * @param fields paths to selected fields relative to parent path
-     * @return a ListenableFuture containing the result of the read
-     */
-    abstract ListenableFuture<Optional<NormalizedNode>> read(LogicalDatastoreType store, YangInstanceIdentifier path,
-        List<YangInstanceIdentifier> fields);
-
     /**
      * Check if data already exists in the configuration datastore.
      *
@@ -183,39 +304,15 @@ public abstract class RestconfStrategy {
     // FIXME: this method should only be needed in MdsalRestconfStrategy
     abstract ListenableFuture<Boolean> exists(YangInstanceIdentifier path);
 
-    /**
-     * Delete data from the configuration datastore. If the data does not exist, this operation will fail, as outlined
-     * in <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.7">RFC8040 section 4.7</a>
-     *
-     * @param path Path to delete
-     * @return A {@link RestconfFuture}
-     * @throws NullPointerException if {@code path} is {@code null}
-     */
-    public final @NonNull RestconfFuture<Empty> delete(final YangInstanceIdentifier path) {
-        final var ret = new SettableRestconfFuture<Empty>();
-        delete(ret, requireNonNull(path));
-        return ret;
-    }
-
-    abstract void delete(@NonNull SettableRestconfFuture<Empty> future, @NonNull YangInstanceIdentifier path);
-
-    /**
-     * Merge data into the configuration datastore, as outlined in
-     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040 section 4.6.1</a>.
-     *
-     * @param path Path to merge
-     * @param data Data to merge
-     * @return A {@link RestconfFuture}
-     * @throws NullPointerException if any argument is {@code null}
-     */
-    public final @NonNull RestconfFuture<Empty> merge(final YangInstanceIdentifier path, final NormalizedNode data) {
-        final var ret = new SettableRestconfFuture<Empty>();
+    @VisibleForTesting
+    final @NonNull RestconfFuture<DataPatchResult> merge(final YangInstanceIdentifier path, final NormalizedNode data) {
+        final var ret = new SettableRestconfFuture<DataPatchResult>();
         merge(ret, requireNonNull(path), requireNonNull(data));
         return ret;
     }
 
-    private void merge(final @NonNull SettableRestconfFuture<Empty> future, final @NonNull YangInstanceIdentifier path,
-            final @NonNull NormalizedNode data) {
+    private void merge(final @NonNull SettableRestconfFuture<DataPatchResult> future,
+            final @NonNull YangInstanceIdentifier path, final @NonNull NormalizedNode data) {
         final var tx = prepareWriteExecution();
         // FIXME: this method should be further specialized to eliminate this call -- it is only needed for MD-SAL
         tx.ensureParentsByMerge(path);
@@ -223,26 +320,52 @@ public abstract class RestconfStrategy {
         Futures.addCallback(tx.commit(), new FutureCallback<CommitInfo>() {
             @Override
             public void onSuccess(final CommitInfo result) {
-                future.set(Empty.value());
+                // TODO: extract details once CommitInfo can communicate them
+                future.set(PATCH_EMPTY);
             }
 
             @Override
             public void onFailure(final Throwable cause) {
-                future.setFailure(TransactionUtil.decodeException(cause, "MERGE", path));
+                future.setFailure(TransactionUtil.decodeException(cause, "MERGE", path, modelContext()));
             }
         }, MoreExecutors.directExecutor());
     }
 
+    public @NonNull RestconfFuture<DataPutResult> dataPUT(final ApiPath apiPath, final ResourceBody body,
+            final Map<String, String> queryParameters) {
+        final Data path;
+        try {
+            path = pathNormalizer.normalizeDataPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        final Insert insert;
+        try {
+            insert = Insert.ofQueryParameters(databind, queryParameters);
+        } catch (IllegalArgumentException e) {
+            return RestconfFuture.failed(new RestconfDocumentedException(e.getMessage(),
+                ErrorType.PROTOCOL, ErrorTag.INVALID_VALUE, e));
+        }
+        final NormalizedNode data;
+        try {
+            data = body.toNormalizedNode(new DataPutPath(databind, path.inference(), path.instance()));
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+        return putData(path.instance(), data, insert);
+    }
+
     /**
      * Check mount point and prepare variables for put data to DS.
      *
      * @param path    path of data
      * @param data    data
      * @param insert  {@link Insert}
-     * @return A {@link CreateOrReplaceResult}
+     * @return A {@link DataPutResult}
      */
-    public final @NonNull CreateOrReplaceResult putData(final YangInstanceIdentifier path, final NormalizedNode data,
-            final @Nullable Insert insert) {
+    public final @NonNull RestconfFuture<DataPutResult> putData(final YangInstanceIdentifier path,
+            final NormalizedNode data, final @Nullable Insert insert) {
         final var exists = TransactionUtil.syncAccess(exists(path), path);
 
         final ListenableFuture<? extends CommitInfo> commitFuture;
@@ -254,8 +377,21 @@ public abstract class RestconfStrategy {
             commitFuture = replaceAndCommit(prepareWriteExecution(), path, data);
         }
 
-        TransactionUtil.syncCommit(commitFuture, "PUT", path);
-        return exists ? CreateOrReplaceResult.REPLACED : CreateOrReplaceResult.CREATED;
+        final var ret = new SettableRestconfFuture<DataPutResult>();
+
+        Futures.addCallback(commitFuture, new FutureCallback<CommitInfo>() {
+            @Override
+            public void onSuccess(final CommitInfo result) {
+                ret.set(exists ? PUT_REPLACED : PUT_CREATED);
+            }
+
+            @Override
+            public void onFailure(final Throwable cause) {
+                ret.setFailure(TransactionUtil.decodeException(cause, "PUT", path, modelContext()));
+            }
+        }, MoreExecutors.directExecutor());
+
+        return ret;
     }
 
     private ListenableFuture<? extends CommitInfo> insertAndCommitPut(final YangInstanceIdentifier path,
@@ -310,7 +446,7 @@ public abstract class RestconfStrategy {
         }
 
         int lastInsertedPosition = 0;
-        final var emptySubtree = ImmutableNodes.fromInstanceId(modelContext, path.getParent());
+        final var emptySubtree = fromInstanceId(modelContext(), path.getParent());
         tx.merge(YangInstanceIdentifier.of(emptySubtree.name()), emptySubtree);
         for (var nodeChild : readList.body()) {
             if (lastInsertedPosition == lastItemPosition) {
@@ -320,6 +456,13 @@ public abstract class RestconfStrategy {
             tx.replace(childPath, nodeChild);
             lastInsertedPosition++;
         }
+
+        // In case we are inserting after last element
+        if (!before) {
+            if (lastInsertedPosition == lastItemPosition) {
+                tx.replace(path, data);
+            }
+        }
     }
 
     private static ListenableFuture<? extends CommitInfo> replaceAndCommit(final RestconfTransaction tx,
@@ -330,8 +473,7 @@ public abstract class RestconfStrategy {
 
     private DataSchemaNode checkListAndOrderedType(final YangInstanceIdentifier path) {
         // FIXME: we have this available in InstanceIdentifierContext
-        final var dataSchemaNode = DataSchemaContextTree.from(modelContext).findChild(path).orElseThrow()
-            .dataSchemaNode();
+        final var dataSchemaNode = databind.schemaTree().findChild(path).orElseThrow().dataSchemaNode();
 
         final String message;
         if (dataSchemaNode instanceof ListSchemaNode listSchema) {
@@ -356,69 +498,130 @@ public abstract class RestconfStrategy {
      * @param path    path
      * @param data    data
      * @param insert  {@link Insert}
+     * @return A {@link RestconfFuture}
      */
-    public final void postData(final YangInstanceIdentifier path, final NormalizedNode data,
-            final @Nullable Insert insert) {
+    public final @NonNull RestconfFuture<CreateResource> postData(final YangInstanceIdentifier path,
+            final NormalizedNode data, final @Nullable Insert insert) {
         final ListenableFuture<? extends CommitInfo> future;
         if (insert != null) {
-            final var parentPath = path.coerceParent();
-            checkListAndOrderedType(parentPath);
-            future = insertAndCommitPost(path, data, insert, parentPath);
+            checkListAndOrderedType(path);
+            future = insertAndCommitPost(path, data, insert);
         } else {
             future = createAndCommit(prepareWriteExecution(), path, data);
         }
-        TransactionUtil.syncCommit(future, "POST", path);
+
+        final var ret = new SettableRestconfFuture<CreateResource>();
+        Futures.addCallback(future, new FutureCallback<CommitInfo>() {
+            @Override
+            public void onSuccess(final CommitInfo result) {
+                ret.set(new CreateResource(new YangInstanceIdentifierSerializer(databind).serializePath(
+                    data instanceof MapNode mapData && !mapData.isEmpty()
+                        ? path.node(mapData.body().iterator().next().name()) : path)));
+            }
+
+            @Override
+            public void onFailure(final Throwable cause) {
+                ret.setFailure(TransactionUtil.decodeException(cause, "POST", path, modelContext()));
+            }
+
+        }, MoreExecutors.directExecutor());
+        return ret;
     }
 
     private ListenableFuture<? extends CommitInfo> insertAndCommitPost(final YangInstanceIdentifier path,
-            final NormalizedNode data, final @NonNull Insert insert, final YangInstanceIdentifier parent) {
-        final var grandParent = parent.coerceParent();
+            final NormalizedNode data, final @NonNull Insert insert) {
         final var tx = prepareWriteExecution();
 
         return switch (insert.insert()) {
             case FIRST -> {
-                final var readData = tx.readList(grandParent);
+                final var readData = tx.readList(path);
                 if (readData == null || readData.isEmpty()) {
                     tx.replace(path, data);
                 } else {
-                    checkItemDoesNotExists(exists(path), path);
-                    tx.remove(grandParent);
+                    checkListDataDoesNotExist(path, data);
+                    tx.remove(path);
                     tx.replace(path, data);
-                    tx.replace(grandParent, readData);
+                    tx.replace(path, readData);
                 }
                 yield tx.commit();
             }
             case LAST -> createAndCommit(tx, path, data);
             case BEFORE -> {
-                final var readData = tx.readList(grandParent);
+                final var readData = tx.readList(path);
                 if (readData == null || readData.isEmpty()) {
                     tx.replace(path, data);
                 } else {
-                    checkItemDoesNotExists(exists(path), path);
-                    insertWithPointPost(tx, path, data, verifyNotNull(insert.pointArg()), readData, grandParent, true);
+                    checkListDataDoesNotExist(path, data);
+                    insertWithPointPost(tx, path, data, verifyNotNull(insert.pointArg()), readData, true);
                 }
                 yield tx.commit();
             }
             case AFTER -> {
-                final var readData = tx.readList(grandParent);
+                final var readData = tx.readList(path);
                 if (readData == null || readData.isEmpty()) {
                     tx.replace(path, data);
                 } else {
-                    checkItemDoesNotExists(exists(path), path);
-                    insertWithPointPost(tx, path, data, verifyNotNull(insert.pointArg()), readData, grandParent, false);
+                    checkListDataDoesNotExist(path, data);
+                    insertWithPointPost(tx, path, data, verifyNotNull(insert.pointArg()), readData, false);
                 }
                 yield tx.commit();
             }
         };
     }
 
+    /**
+     * Merge data into the configuration datastore, as outlined in
+     * <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.6.1">RFC8040 section 4.6.1</a>.
+     *
+     * @param apiPath Path to merge
+     * @param body Data to merge
+     * @return A {@link RestconfFuture}
+     * @throws NullPointerException if any argument is {@code null}
+     */
+    public final @NonNull RestconfFuture<DataPatchResult> dataPATCH(final ApiPath apiPath, final ResourceBody body) {
+        final Data path;
+        try {
+            path = pathNormalizer.normalizeDataPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        final NormalizedNode data;
+        try {
+            data = body.toNormalizedNode(new DataPutPath(databind, path.inference(), path.instance()));
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        return merge(path.instance(), data);
+    }
+
+    public final @NonNull RestconfFuture<DataYangPatchResult> dataPATCH(final ApiPath apiPath, final PatchBody body) {
+        final Data path;
+        try {
+            path = pathNormalizer.normalizeDataPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        final PatchContext patch;
+        try {
+            patch = body.toPatchContext(new DataPatchPath(databind, path.instance()));
+        } catch (IOException e) {
+            LOG.debug("Error parsing YANG Patch input", e);
+            return RestconfFuture.failed(new RestconfDocumentedException("Error parsing input: " + e.getMessage(),
+                ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE, e));
+        }
+        return patchData(patch);
+    }
+
     /**
      * Process edit operations of one {@link PatchContext}.
      *
      * @param patch Patch context to be processed
      * @return {@link PatchStatusContext}
      */
-    public final @NonNull PatchStatusContext patchData(final PatchContext patch) {
+    public final @NonNull RestconfFuture<DataYangPatchResult> patchData(final PatchContext patch) {
         final var editCollection = new ArrayList<PatchStatusEntity>();
         final var tx = prepareWriteExecution();
 
@@ -487,27 +690,38 @@ public abstract class RestconfStrategy {
             }
         }
 
-        // if no errors then submit transaction, otherwise cancel
-        final var patchId = patch.patchId();
-        if (noError) {
-            try {
-                TransactionUtil.syncCommit(tx.commit(), "PATCH", null);
-            } catch (RestconfDocumentedException e) {
+        final var ret = new SettableRestconfFuture<DataYangPatchResult>();
+        // We have errors
+        if (!noError) {
+            tx.cancel();
+            ret.set(new DataYangPatchResult(
+                new PatchStatusContext(modelContext(), patch.patchId(), List.copyOf(editCollection), false, null)));
+            return ret;
+        }
+
+        Futures.addCallback(tx.commit(), new FutureCallback<CommitInfo>() {
+            @Override
+            public void onSuccess(final CommitInfo result) {
+                ret.set(new DataYangPatchResult(
+                    new PatchStatusContext(modelContext(), patch.patchId(), List.copyOf(editCollection), true, null)));
+            }
+
+            @Override
+            public void onFailure(final Throwable cause) {
                 // if errors occurred during transaction commit then patch failed and global errors are reported
-                return new PatchStatusContext(modelContext, patchId, List.copyOf(editCollection), false, e.getErrors());
+                ret.set(new DataYangPatchResult(
+                    new PatchStatusContext(modelContext(), patch.patchId(), List.copyOf(editCollection), false,
+                        TransactionUtil.decodeException(cause, "PATCH", null, modelContext()).getErrors())));
             }
+        }, MoreExecutors.directExecutor());
 
-            return new PatchStatusContext(modelContext, patchId, List.copyOf(editCollection), true, null);
-        } else {
-            tx.cancel();
-            return new PatchStatusContext(modelContext, patchId, List.copyOf(editCollection), false, null);
-        }
+        return ret;
     }
 
     private void insertWithPointPost(final RestconfTransaction tx, final YangInstanceIdentifier path,
             final NormalizedNode data, final PathArgument pointArg, final NormalizedNodeContainer<?> readList,
-            final YangInstanceIdentifier grandParentPath, final boolean before) {
-        tx.remove(grandParentPath);
+            final boolean before) {
+        tx.remove(path);
 
         int lastItemPosition = 0;
         for (var nodeChild : readList.body()) {
@@ -521,15 +735,20 @@ public abstract class RestconfStrategy {
         }
 
         int lastInsertedPosition = 0;
-        final var emptySubtree = ImmutableNodes.fromInstanceId(modelContext, grandParentPath);
-        tx.merge(YangInstanceIdentifier.of(emptySubtree.name()), emptySubtree);
         for (var nodeChild : readList.body()) {
             if (lastInsertedPosition == lastItemPosition) {
                 tx.replace(path, data);
             }
-            tx.replace(grandParentPath.node(nodeChild.name()), nodeChild);
+            tx.replace(path.node(nodeChild.name()), nodeChild);
             lastInsertedPosition++;
         }
+
+        // In case we are inserting after last element
+        if (!before) {
+            if (lastInsertedPosition == lastItemPosition) {
+                tx.replace(path, data);
+            }
+        }
     }
 
     private static ListenableFuture<? extends CommitInfo> createAndCommit(final RestconfTransaction tx,
@@ -545,6 +764,23 @@ public abstract class RestconfStrategy {
         return tx.commit();
     }
 
+    /**
+     * Check if child items do NOT already exists in List at specified {@code path}.
+     *
+     * @param data Data to be checked
+     * @param path Path to be checked
+     * @throws RestconfDocumentedException if data already exists.
+     */
+    private void checkListDataDoesNotExist(final YangInstanceIdentifier path, final NormalizedNode data) {
+        if (data instanceof NormalizedNodeContainer<?> dataNode) {
+            for (final var node : dataNode.body()) {
+                checkItemDoesNotExists(exists(path.node(node.name())), path.node(node.name()));
+            }
+        } else {
+            throw new RestconfDocumentedException("Unexpected node type: " + data.getClass().getName());
+        }
+    }
+
     /**
      * Check if items do NOT already exists at specified {@code path}.
      *
@@ -561,6 +797,58 @@ public abstract class RestconfStrategy {
         }
     }
 
+    /**
+     * Delete data from the configuration datastore. If the data does not exist, this operation will fail, as outlined
+     * in <a href="https://www.rfc-editor.org/rfc/rfc8040#section-4.7">RFC8040 section 4.7</a>
+     *
+     * @param apiPath Path to delete
+     * @return A {@link RestconfFuture}
+     * @throws NullPointerException if {@code apiPath} is {@code null}
+     */
+    @SuppressWarnings("checkstyle:abbreviationAsWordInName")
+    public final @NonNull RestconfFuture<Empty> dataDELETE(final ApiPath apiPath) {
+        final Data path;
+        try {
+            path = pathNormalizer.normalizeDataPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        // FIXME: reject empty YangInstanceIdentifier, as datastores may not be deleted
+        final var ret = new SettableRestconfFuture<Empty>();
+        delete(ret, path.instance());
+        return ret;
+    }
+
+    abstract void delete(@NonNull SettableRestconfFuture<Empty> future, @NonNull YangInstanceIdentifier path);
+
+    public final @NonNull RestconfFuture<DataGetResult> dataGET(final ApiPath apiPath,
+            final DataGetParams params) {
+        final Data path;
+        try {
+            path = pathNormalizer.normalizeDataPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+        return dataGET(path, params);
+    }
+
+    abstract @NonNull RestconfFuture<DataGetResult> dataGET(Data path, DataGetParams params);
+
+    static final @NonNull RestconfFuture<DataGetResult> completeDataGET(final Inference inference,
+            final QueryParameters queryParams, final @Nullable NormalizedNode node,
+            final @Nullable ConfigurationMetadata metadata) {
+        if (node == null) {
+            return RestconfFuture.failed(new RestconfDocumentedException(
+                "Request could not be completed because the relevant data model content does not exist",
+                ErrorType.PROTOCOL, ErrorTag.DATA_MISSING));
+        }
+
+        final var payload = new NormalizedNodePayload(inference, node, queryParams);
+        return RestconfFuture.of(metadata == null ? new DataGetResult(payload)
+            : new DataGetResult(payload, metadata.entityTag(), metadata.lastModified()));
+    }
+
     /**
      * Read specific type of data from data store via transaction. Close {@link DOMTransactionChain} if any
      * inside of object {@link RestconfStrategy} provided as a parameter.
@@ -570,7 +858,9 @@ public abstract class RestconfStrategy {
      * @param defaultsMode value of with-defaults parameter
      * @return {@link NormalizedNode}
      */
-    public @Nullable NormalizedNode readData(final @NonNull ContentParam content,
+    // FIXME: NETCONF-1155: this method should asynchronous
+    @VisibleForTesting
+    final @Nullable NormalizedNode readData(final @NonNull ContentParam content,
             final @NonNull YangInstanceIdentifier path, final WithDefaultsParam defaultsMode) {
         return switch (content) {
             case ALL -> {
@@ -591,59 +881,12 @@ public abstract class RestconfStrategy {
         };
     }
 
-    /**
-     * Read specific type of data from data store via transaction with specified subtrees that should only be read.
-     * Close {@link DOMTransactionChain} inside of object {@link RestconfStrategy} provided as a parameter.
-     *
-     * @param content  type of data to read (config, state, all)
-     * @param path     the parent path to read
-     * @param withDefa value of with-defaults parameter
-     * @param fields   paths to selected subtrees which should be read, relative to to the parent path
-     * @return {@link NormalizedNode}
-     */
-    public @Nullable NormalizedNode readData(final @NonNull ContentParam content,
-            final @NonNull YangInstanceIdentifier path, final @Nullable WithDefaultsParam withDefa,
-            final @NonNull List<YangInstanceIdentifier> fields) {
-        return switch (content) {
-            case ALL -> {
-                // PREPARE STATE DATA NODE
-                final var stateDataNode = readDataViaTransaction(LogicalDatastoreType.OPERATIONAL, path, fields);
-                // PREPARE CONFIG DATA NODE
-                final var configDataNode = readDataViaTransaction(LogicalDatastoreType.CONFIGURATION, path, fields);
-
-                yield mergeConfigAndSTateDataIfNeeded(stateDataNode, withDefa == null ? configDataNode
-                    : prepareDataByParamWithDef(configDataNode, path, withDefa.mode()));
-            }
-            case CONFIG -> {
-                final var read = readDataViaTransaction(LogicalDatastoreType.CONFIGURATION, path, fields);
-                yield withDefa == null ? read : prepareDataByParamWithDef(read, path, withDefa.mode());
-            }
-            case NONCONFIG -> readDataViaTransaction(LogicalDatastoreType.OPERATIONAL, path, fields);
-        };
-    }
-
     private @Nullable NormalizedNode readDataViaTransaction(final LogicalDatastoreType store,
             final YangInstanceIdentifier path) {
         return TransactionUtil.syncAccess(read(store, path), path).orElse(null);
     }
 
-    /**
-     * Read specific type of data {@link LogicalDatastoreType} via transaction in {@link RestconfStrategy} with
-     * specified subtrees that should only be read.
-     *
-     * @param store                 datastore type
-     * @param path                  parent path to selected fields
-     * @param closeTransactionChain if it is set to {@code true}, after transaction it will close transactionChain
-     *                              in {@link RestconfStrategy} if any
-     * @param fields                paths to selected subtrees which should be read, relative to to the parent path
-     * @return {@link NormalizedNode}
-     */
-    private @Nullable NormalizedNode readDataViaTransaction(final @NonNull LogicalDatastoreType store,
-            final @NonNull YangInstanceIdentifier path, final @NonNull List<YangInstanceIdentifier> fields) {
-        return TransactionUtil.syncAccess(read(store, path, fields), path).orElse(null);
-    }
-
-    private NormalizedNode prepareDataByParamWithDef(final NormalizedNode readData, final YangInstanceIdentifier path,
+    final NormalizedNode prepareDataByParamWithDef(final NormalizedNode readData, final YangInstanceIdentifier path,
             final WithDefaultsMode defaultsMode) {
         final boolean trim = switch (defaultsMode) {
             case Trim -> true;
@@ -653,9 +896,9 @@ public abstract class RestconfStrategy {
         };
 
         // FIXME: we have this readily available in InstanceIdentifierContext
-        final var ctxNode = DataSchemaContextTree.from(modelContext).findChild(path).orElseThrow();
+        final var ctxNode = databind.schemaTree().findChild(path).orElseThrow();
         if (readData instanceof ContainerNode container) {
-            final var builder = Builders.containerBuilder().withNodeIdentifier(container.name());
+            final var builder = ImmutableNodes.newContainerBuilder().withNodeIdentifier(container.name());
             buildCont(builder, container.body(), ctxNode, trim);
             return builder.build();
         } else if (readData instanceof MapEntryNode mapEntry) {
@@ -663,7 +906,7 @@ public abstract class RestconfStrategy {
                 throw new IllegalStateException("Input " + mapEntry + " does not match " + ctxNode);
             }
 
-            final var builder = Builders.mapEntryBuilder().withNodeIdentifier(mapEntry.name());
+            final var builder = ImmutableNodes.newMapEntryBuilder().withNodeIdentifier(mapEntry.name());
             buildMapEntryBuilder(builder, mapEntry.body(), ctxNode, trim, listSchema.getKeyDefinition());
             return builder.build();
         } else {
@@ -693,7 +936,7 @@ public abstract class RestconfStrategy {
 
     private static void appendContainer(final DataContainerNodeBuilder<?, ?> builder, final ContainerNode container,
             final DataSchemaContext ctxNode, final boolean trim) {
-        final var childBuilder = Builders.containerBuilder().withNodeIdentifier(container.name());
+        final var childBuilder = ImmutableNodes.newContainerBuilder().withNodeIdentifier(container.name());
         buildCont(childBuilder, container.body(), ctxNode, trim);
         builder.withChild(childBuilder.build());
     }
@@ -757,8 +1000,8 @@ public abstract class RestconfStrategy {
         }
 
         final var childBuilder = switch (map.ordering()) {
-            case SYSTEM -> Builders.mapBuilder();
-            case USER -> Builders.orderedMapBuilder();
+            case SYSTEM -> ImmutableNodes.newSystemMapBuilder();
+            case USER -> ImmutableNodes.newUserMapBuilder();
         };
         buildList(childBuilder.withNodeIdentifier(map.name()), map.body(), childCtx, trim,
             listSchema.getKeyDefinition());
@@ -770,7 +1013,7 @@ public abstract class RestconfStrategy {
             final List<@NonNull QName> keys) {
         for (var entry : entries) {
             final var childCtx = getChildContext(ctxNode, entry);
-            final var mapEntryBuilder = Builders.mapEntryBuilder().withNodeIdentifier(entry.name());
+            final var mapEntryBuilder = ImmutableNodes.newMapEntryBuilder().withNodeIdentifier(entry.name());
             buildMapEntryBuilder(mapEntryBuilder, entry.body(), childCtx, trim, keys);
             builder.withChild(mapEntryBuilder.build());
         }
@@ -801,24 +1044,17 @@ public abstract class RestconfStrategy {
         return childCtx;
     }
 
-    private static NormalizedNode mergeConfigAndSTateDataIfNeeded(final NormalizedNode stateDataNode,
-                                                                  final NormalizedNode configDataNode) {
-        // if no data exists
-        if (stateDataNode == null && configDataNode == null) {
-            return null;
-        }
-
-        // return config data
+    static final NormalizedNode mergeConfigAndSTateDataIfNeeded(final NormalizedNode stateDataNode,
+            final NormalizedNode configDataNode) {
         if (stateDataNode == null) {
+            // No state, return config
             return configDataNode;
         }
-
-        // return state data
         if (configDataNode == null) {
+            // No config, return state
             return stateDataNode;
         }
-
-        // merge data from config and state
+        // merge config and state
         return mergeStateAndConfigData(stateDataNode, configDataNode);
     }
 
@@ -864,7 +1100,7 @@ public abstract class RestconfStrategy {
      */
     private static @NonNull NormalizedNode prepareRpcData(final @NonNull NormalizedNode configDataNode,
                                                           final @NonNull NormalizedNode stateDataNode) {
-        final var mapEntryBuilder = Builders.mapEntryBuilder()
+        final var mapEntryBuilder = ImmutableNodes.newMapEntryBuilder()
             .withNodeIdentifier((NodeIdentifierWithPredicates) configDataNode.name());
 
         // MAP CONFIG DATA
@@ -872,7 +1108,7 @@ public abstract class RestconfStrategy {
         // MAP STATE DATA
         mapRpcDataNode(stateDataNode, mapEntryBuilder);
 
-        return Builders.mapBuilder()
+        return ImmutableNodes.newSystemMapBuilder()
             .withNodeIdentifier(NodeIdentifier.create(configDataNode.name().getNodeType()))
             .addChild(mapEntryBuilder.build())
             .build();
@@ -900,23 +1136,23 @@ public abstract class RestconfStrategy {
     private static @NonNull NormalizedNode prepareData(final @NonNull NormalizedNode configDataNode,
                                                        final @NonNull NormalizedNode stateDataNode) {
         if (configDataNode instanceof UserMapNode configMap) {
-            final var builder = Builders.orderedMapBuilder().withNodeIdentifier(configMap.name());
+            final var builder = ImmutableNodes.newUserMapBuilder().withNodeIdentifier(configMap.name());
             mapValueToBuilder(configMap.body(), ((UserMapNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof SystemMapNode configMap) {
-            final var builder = Builders.mapBuilder().withNodeIdentifier(configMap.name());
+            final var builder = ImmutableNodes.newSystemMapBuilder().withNodeIdentifier(configMap.name());
             mapValueToBuilder(configMap.body(), ((SystemMapNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof MapEntryNode configEntry) {
-            final var builder = Builders.mapEntryBuilder().withNodeIdentifier(configEntry.name());
+            final var builder = ImmutableNodes.newMapEntryBuilder().withNodeIdentifier(configEntry.name());
             mapValueToBuilder(configEntry.body(), ((MapEntryNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof ContainerNode configContaienr) {
-            final var builder = Builders.containerBuilder().withNodeIdentifier(configContaienr.name());
+            final var builder = ImmutableNodes.newContainerBuilder().withNodeIdentifier(configContaienr.name());
             mapValueToBuilder(configContaienr.body(), ((ContainerNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof ChoiceNode configChoice) {
-            final var builder = Builders.choiceBuilder().withNodeIdentifier(configChoice.name());
+            final var builder = ImmutableNodes.newChoiceBuilder().withNodeIdentifier(configChoice.name());
             mapValueToBuilder(configChoice.body(), ((ChoiceNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof LeafNode configLeaf) {
@@ -924,23 +1160,24 @@ public abstract class RestconfStrategy {
             return configLeaf;
         } else if (configDataNode instanceof UserLeafSetNode) {
             final var configLeafSet = (UserLeafSetNode<Object>) configDataNode;
-            final var builder = Builders.<Object>orderedLeafSetBuilder().withNodeIdentifier(configLeafSet.name());
+            final var builder = ImmutableNodes.<Object>newUserLeafSetBuilder().withNodeIdentifier(configLeafSet.name());
             mapValueToBuilder(configLeafSet.body(), ((UserLeafSetNode<Object>) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof SystemLeafSetNode) {
             final var configLeafSet = (SystemLeafSetNode<Object>) configDataNode;
-            final var builder = Builders.<Object>leafSetBuilder().withNodeIdentifier(configLeafSet.name());
+            final var builder = ImmutableNodes.<Object>newSystemLeafSetBuilder()
+                .withNodeIdentifier(configLeafSet.name());
             mapValueToBuilder(configLeafSet.body(), ((SystemLeafSetNode<Object>) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof LeafSetEntryNode<?> configEntry) {
             // config trumps oper
             return configEntry;
         } else if (configDataNode instanceof UnkeyedListNode configList) {
-            final var builder = Builders.unkeyedListBuilder().withNodeIdentifier(configList.name());
+            final var builder = ImmutableNodes.newUnkeyedListBuilder().withNodeIdentifier(configList.name());
             mapValueToBuilder(configList.body(), ((UnkeyedListNode) stateDataNode).body(), builder);
             return builder.build();
         } else if (configDataNode instanceof UnkeyedListEntryNode configEntry) {
-            final var builder = Builders.unkeyedListEntryBuilder().withNodeIdentifier(configEntry.name());
+            final var builder = ImmutableNodes.newUnkeyedListEntryBuilder().withNodeIdentifier(configEntry.name());
             mapValueToBuilder(configEntry.body(), ((UnkeyedListEntryNode) stateDataNode).body(), builder);
             return builder.build();
         } else {
@@ -1002,42 +1239,293 @@ public abstract class RestconfStrategy {
             y -> builder.addChild((T) prepareData(y.getValue(), stateMap.get(y.getKey()))));
     }
 
-    public @NonNull RestconfFuture<Optional<ContainerNode>> invokeRpc(final QName type, final ContainerNode input) {
-        final var ret = new SettableRestconfFuture<Optional<ContainerNode>>();
+    public @NonNull RestconfFuture<OperationsGetResult> operationsGET() {
+        final var modelContext = modelContext();
+        final var modules = modelContext.getModuleStatements();
+        if (modules.isEmpty()) {
+            // No modules, or defensive return empty content
+            return RestconfFuture.of(new OperationsGetResult.Container(modelContext, ImmutableSetMultimap.of()));
+        }
+
+        // RPC QNames by their XMLNamespace/Revision. This should be a Table, but Revision can be null, which wrecks us.
+        final var table = new HashMap<XMLNamespace, Map<Revision, ImmutableSet<QName>>>();
+        for (var entry : modules.entrySet()) {
+            final var module = entry.getValue();
+            final var rpcNames = module.streamEffectiveSubstatements(RpcEffectiveStatement.class)
+                .map(RpcEffectiveStatement::argument)
+                .collect(ImmutableSet.toImmutableSet());
+            if (!rpcNames.isEmpty()) {
+                final var namespace = entry.getKey();
+                table.computeIfAbsent(namespace.namespace(), ignored -> new HashMap<>())
+                    .put(namespace.revision(), rpcNames);
+            }
+        }
+
+        // Now pick the latest revision for each namespace
+        final var rpcs = ImmutableSetMultimap.<QNameModule, QName>builder();
+        for (var entry : table.entrySet()) {
+            entry.getValue().entrySet().stream()
+            .sorted(Comparator.comparing(Entry::getKey, (first, second) -> Revision.compare(second, first)))
+            .findFirst()
+            .ifPresent(row -> rpcs.putAll(QNameModule.of(entry.getKey(), row.getKey()), row.getValue()));
+        }
+        return RestconfFuture.of(new OperationsGetResult.Container(modelContext, rpcs.build()));
+    }
+
+    public @NonNull RestconfFuture<OperationsGetResult> operationsGET(final ApiPath apiPath) {
+        if (apiPath.steps().isEmpty()) {
+            return operationsGET();
+        }
+
+        final Rpc rpc;
+        try {
+            rpc = pathNormalizer.normalizeRpcPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+
+        return RestconfFuture.of(new OperationsGetResult.Leaf(rpc.inference().modelContext(), rpc.rpc().argument()));
+    }
+
+    public @NonNull RestconfFuture<OperationsPostResult> operationsPOST(final URI restconfURI, final ApiPath apiPath,
+            final OperationInputBody body) {
+        final Rpc path;
+        try {
+            path = pathNormalizer.normalizeRpcPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
 
-        final var local = rpcService;
+        final var postPath = new OperationsPostPath(databind, path.inference());
+        final ContainerNode data;
+        try {
+            data = body.toContainerNode(postPath);
+        } catch (IOException e) {
+            LOG.debug("Error reading input", e);
+            return RestconfFuture.failed(new RestconfDocumentedException("Error parsing input: " + e.getMessage(),
+                ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE, e));
+        }
+
+        final var type = path.rpc().argument();
+        final var local = localRpcs.get(type);
         if (local != null) {
-            Futures.addCallback(local.invokeRpc(requireNonNull(type), requireNonNull(input)),
-                new FutureCallback<DOMRpcResult>() {
+            return local.invoke(restconfURI, new OperationInput(databind, postPath.operation(), data));
+        }
+        if (rpcService == null) {
+            LOG.debug("RPC invocation is not available");
+            return RestconfFuture.failed(new RestconfDocumentedException("RPC invocation is not available",
+                ErrorType.PROTOCOL, ErrorTag.OPERATION_NOT_SUPPORTED));
+        }
+
+        final var ret = new SettableRestconfFuture<OperationsPostResult>();
+        Futures.addCallback(rpcService.invokeRpc(type, data), new FutureCallback<DOMRpcResult>() {
+            @Override
+            public void onSuccess(final DOMRpcResult response) {
+                final var errors = response.errors();
+                if (errors.isEmpty()) {
+                    ret.set(new OperationsPostResult(databind, postPath.operation(), response.value()));
+                } else {
+                    LOG.debug("RPC invocation reported {}", response.errors());
+                    ret.setFailure(new RestconfDocumentedException("RPC implementation reported errors", null,
+                        response.errors()));
+                }
+            }
+
+            @Override
+            public void onFailure(final Throwable cause) {
+                LOG.debug("RPC invocation failed, cause");
+                if (cause instanceof RestconfDocumentedException ex) {
+                    ret.setFailure(ex);
+                } else {
+                    // TODO: YangNetconfErrorAware if we ever get into a broader invocation scope
+                    ret.setFailure(new RestconfDocumentedException(cause,
+                        new RestconfError(ErrorType.RPC, ErrorTag.OPERATION_FAILED, cause.getMessage())));
+                }
+            }
+        }, MoreExecutors.directExecutor());
+        return ret;
+    }
+
+    public @NonNull RestconfFuture<CharSource> resolveSource(final SourceIdentifier source,
+            final Class<? extends SourceRepresentation> representation) {
+        final var src = requireNonNull(source);
+        if (YangTextSource.class.isAssignableFrom(representation)) {
+            if (sourceProvider != null) {
+                final var ret = new SettableRestconfFuture<CharSource>();
+                Futures.addCallback(sourceProvider.getYangTexttSource(src), new FutureCallback<>() {
                     @Override
-                    public void onSuccess(final DOMRpcResult response) {
-                        final var errors = response.errors();
-                        if (errors.isEmpty()) {
-                            ret.set(Optional.ofNullable(response.value()));
-                        } else {
-                            LOG.debug("RPC invocation reported {}", response.errors());
-                            ret.setFailure(new RestconfDocumentedException("RPC implementation reported errors", null,
-                                response.errors()));
-                        }
+                    public void onSuccess(final YangTextSource result) {
+                        ret.set(result);
                     }
 
                     @Override
                     public void onFailure(final Throwable cause) {
-                        LOG.debug("RPC invocation failed, cause");
-                        if (cause instanceof RestconfDocumentedException ex) {
-                            ret.setFailure(ex);
-                        } else {
-                            // TODO: YangNetconfErrorAware if we ever get into a broader invocation scope
-                            ret.setFailure(new RestconfDocumentedException(cause,
-                                new RestconfError(ErrorType.RPC, ErrorTag.OPERATION_FAILED, cause.getMessage())));
-                        }
+                        ret.setFailure(cause instanceof RestconfDocumentedException e ? e
+                            : new RestconfDocumentedException(cause.getMessage(), ErrorType.RPC,
+                                ErrorTag.OPERATION_FAILED, cause));
                     }
                 }, MoreExecutors.directExecutor());
-        } else {
-            LOG.debug("RPC invocation is not available");
-            ret.setFailure(new RestconfDocumentedException("RPC invocation is not available",
-                ErrorType.PROTOCOL, ErrorTag.OPERATION_NOT_SUPPORTED));
+                return ret;
+            }
+            return exportSource(modelContext(), src, YangCharSource::new, YangCharSource::new);
+        }
+        if (YinTextSource.class.isAssignableFrom(representation)) {
+            return exportSource(modelContext(), src, YinCharSource.OfModule::new, YinCharSource.OfSubmodule::new);
+        }
+        return RestconfFuture.failed(new RestconfDocumentedException(
+            "Unsupported source representation " + representation.getName()));
+    }
+
+    private static @NonNull RestconfFuture<CharSource> exportSource(final EffectiveModelContext modelContext,
+            final SourceIdentifier source, final Function<ModuleEffectiveStatement, CharSource> moduleCtor,
+            final BiFunction<ModuleEffectiveStatement, SubmoduleEffectiveStatement, CharSource> submoduleCtor) {
+        // If the source identifies a module, things are easy
+        final var name = source.name().getLocalName();
+        final var optRevision = Optional.ofNullable(source.revision());
+        final var optModule = modelContext.findModule(name, optRevision);
+        if (optModule.isPresent()) {
+            return RestconfFuture.of(moduleCtor.apply(optModule.orElseThrow().asEffectiveStatement()));
+        }
+
+        // The source could be a submodule, which we need to hunt down
+        for (var module : modelContext.getModules()) {
+            for (var submodule : module.getSubmodules()) {
+                if (name.equals(submodule.getName()) && optRevision.equals(submodule.getRevision())) {
+                    return RestconfFuture.of(submoduleCtor.apply(module.asEffectiveStatement(),
+                        submodule.asEffectiveStatement()));
+                }
+            }
+        }
+
+        final var sb = new StringBuilder().append("Source ").append(source.name().getLocalName());
+        optRevision.ifPresent(rev -> sb.append('@').append(rev));
+        sb.append(" not found");
+        return RestconfFuture.failed(new RestconfDocumentedException(sb.toString(),
+            ErrorType.APPLICATION, ErrorTag.DATA_MISSING));
+    }
+
+    public final @NonNull RestconfFuture<? extends DataPostResult> dataPOST(final ApiPath apiPath,
+            final DataPostBody body, final Map<String, String> queryParameters) {
+        if (apiPath.steps().isEmpty()) {
+            return dataCreatePOST(body.toResource(), queryParameters);
         }
+        final InstanceReference path;
+        try {
+            path = pathNormalizer.normalizeDataOrActionPath(apiPath);
+        } catch (RestconfDocumentedException e) {
+            return RestconfFuture.failed(e);
+        }
+        if (path instanceof Data dataPath) {
+            try (var resourceBody = body.toResource()) {
+                return dataCreatePOST(new DataPostPath(databind, dataPath.inference(), dataPath.instance()),
+                    resourceBody, queryParameters);
+            }
+        }
+        if (path instanceof Action actionPath) {
+            try (var inputBody = body.toOperationInput()) {
+                return dataInvokePOST(actionPath, inputBody);
+            }
+        }
+        // Note: this should never happen
+        // FIXME: we should be able to eliminate this path with Java 21+ pattern matching
+        return RestconfFuture.failed(new RestconfDocumentedException("Unhandled path " + path));
+    }
+
+    public @NonNull RestconfFuture<CreateResource> dataCreatePOST(final ChildBody body,
+            final Map<String, String> queryParameters) {
+        return dataCreatePOST(new DataPostPath(databind,
+            SchemaInferenceStack.of(databind.modelContext()).toInference(), YangInstanceIdentifier.of()), body,
+            queryParameters);
+    }
+
+    private @NonNull RestconfFuture<CreateResource> dataCreatePOST(final DataPostPath path, final ChildBody body,
+            final Map<String, String> queryParameters) {
+        final Insert insert;
+        try {
+            insert = Insert.ofQueryParameters(path.databind(), queryParameters);
+        } catch (IllegalArgumentException e) {
+            return RestconfFuture.failed(new RestconfDocumentedException(e.getMessage(),
+                ErrorType.PROTOCOL, ErrorTag.INVALID_VALUE, e));
+        }
+
+        final var payload = body.toPayload(path);
+        return postData(concat(path.instance(), payload.prefix()), payload.body(), insert);
+    }
+
+    private static YangInstanceIdentifier concat(final YangInstanceIdentifier parent, final List<PathArgument> args) {
+        var ret = parent;
+        for (var arg : args) {
+            ret = ret.node(arg);
+        }
+        return ret;
+    }
+
+    private @NonNull RestconfFuture<InvokeOperation> dataInvokePOST(final Action path, final OperationInputBody body) {
+        final var inference = path.inference();
+        final ContainerNode input;
+        try {
+            input = body.toContainerNode(new OperationsPostPath(databind, inference));
+        } catch (IOException e) {
+            LOG.debug("Error reading input", e);
+            return RestconfFuture.failed(new RestconfDocumentedException("Error parsing input: " + e.getMessage(),
+                ErrorType.PROTOCOL, ErrorTag.MALFORMED_MESSAGE, e));
+        }
+
+        if (actionService == null) {
+            return RestconfFuture.failed(new RestconfDocumentedException("DOMActionService is missing."));
+        }
+
+        final var future = dataInvokePOST(actionService, path, input);
+        return future.transform(result -> result.getOutput()
+            .flatMap(output -> output.isEmpty() ? Optional.empty()
+                : Optional.of(new InvokeOperation(new NormalizedNodePayload(inference, output))))
+            .orElse(InvokeOperation.EMPTY));
+    }
+
+    /**
+     * Invoke Action via ActionServiceHandler.
+     *
+     * @param input 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> dataInvokePOST(final DOMActionService actionService,
+            final Action path, final @NonNull ContainerNode input) {
+        final var ret = new SettableRestconfFuture<DOMActionResult>();
+
+        Futures.addCallback(actionService.invokeAction(
+            path.inference().toSchemaInferenceStack().toSchemaNodeIdentifier(),
+            DOMDataTreeIdentifier.of(LogicalDatastoreType.OPERATIONAL, path.instance()), input),
+            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;
     }
 }