You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2022/02/04 11:54:34 UTC

[jena] branch main updated: JENA-2272: Introduce DSP. Refactor GSP. Common base StoreProtocol.

This is an automated email from the ASF dual-hosted git repository.

andy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/jena.git


The following commit(s) were added to refs/heads/main by this push:
     new 73832e3  JENA-2272: Introduce DSP. Refactor GSP. Common base StoreProtocol.
     new d3cbbba  Merge pull request #1184 from afs/store-protocol
73832e3 is described below

commit 73832e3ca81275e65d6d2c9f9a4a3942dafeefb3
Author: Andy Seaborne <an...@apache.org>
AuthorDate: Thu Feb 3 13:45:48 2022 +0000

    JENA-2272: Introduce DSP. Refactor GSP. Common base StoreProtocol.
---
 .../java/org/apache/jena/query/ModelStore.java     |  37 ++-
 .../java/org/apache/jena/sparql/exec/http/DSP.java | 146 +++++++++
 .../java/org/apache/jena/sparql/exec/http/GSP.java | 364 ++++++---------------
 .../jena/sparql/exec/http/StoreProtocol.java       | 306 +++++++++++++++++
 .../jena/sparql/exec/http/TS_SparqlExecHttp.java   |   2 +
 .../org/apache/jena/sparql/exec/http/TestDSP.java  | 145 ++++++++
 .../org/apache/jena/sparql/exec/http/TestGSP.java  |  80 ++---
 .../jena/sparql/exec/http/TestModelStore.java      | 286 ++++++++++++++++
 .../java/org/apache/jena/rdflink/RDFLinkHTTP.java  |  15 +-
 9 files changed, 1044 insertions(+), 337 deletions(-)

diff --git a/jena-arq/src/main/java/org/apache/jena/query/ModelStore.java b/jena-arq/src/main/java/org/apache/jena/query/ModelStore.java
index eb2d63b..2ededca 100644
--- a/jena-arq/src/main/java/org/apache/jena/query/ModelStore.java
+++ b/jena-arq/src/main/java/org/apache/jena/query/ModelStore.java
@@ -26,6 +26,7 @@ import org.apache.jena.rdf.model.ModelFactory;
 import org.apache.jena.riot.Lang;
 import org.apache.jena.riot.RDFFormat;
 import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.exec.http.DSP;
 import org.apache.jena.sparql.exec.http.GSP;
 
 /**
@@ -101,13 +102,22 @@ public class ModelStore {
 
     /** Send request for the default graph (that is, {@code ?default}) */
     public ModelStore defaultModel() {
+        return defaultGraph();
+    }
+
+    /** Send request for the default graph (that is, {@code ?default}) */
+    public ModelStore defaultGraph() {
         gsp().defaultGraph();
         return this;
     }
 
-    /** Send request for the dataset. This is "no GSP naming". */
+    /**
+     * Send request for the dataset. This is "no GSP naming".
+     * @deprecated This call is a no-op.
+     */
+    @Deprecated
     public ModelStore dataset() {
-        gsp().dataset();
+        //gsp().dataset();
         return this;
     }
 
@@ -125,7 +135,7 @@ public class ModelStore {
 
     /**
      * Set the Content-type for a POST, PUT request of a file
-     * or serialization of a graph opf dataset is necessary.
+     * or serialization of a graph of dataset is necessary.
      * Optional; if not set, the file extension is used or the
      * system default RDF syntax encoding.
      */
@@ -136,7 +146,7 @@ public class ModelStore {
 
     /**
      * Set the Content-type for a POST, PUT request of a file
-     * or serialization of a graph opf dataset is necessary.
+     * or serialization of a graph of dataset is necessary.
      * Optional; if not set, the file extension is used or the
      * system default RDF syntax encoding.
      */
@@ -145,6 +155,8 @@ public class ModelStore {
         return this;
     }
 
+    // -- Model Operations
+
     /** Get a graph */
     public Model GET() {
         Graph graph = gsp().GET();
@@ -251,6 +263,7 @@ public class ModelStore {
         gsp().DELETE();
     }
 
+    // -- Dataset Operations
 
     /**
      * GET dataset.
@@ -259,10 +272,14 @@ public class ModelStore {
      * graph data in the default graph of the dataset.
      */
     public Dataset getDataset() {
-        DatasetGraph dsg = gsp().getDataset();
+        DatasetGraph dsg = dsp().GET();
         return DatasetFactory.wrap(dsg);
     }
 
+    private DSP dsp() {
+        return DSP.request().copySetup(gsp);
+    }
+
     /**
      * POST the contents of a file using the filename extension to determine the
      * Content-Type to use if not already set.
@@ -270,12 +287,12 @@ public class ModelStore {
      * This operation does not parse the file.
      */
     public void postDataset(String file) {
-        gsp().postDataset(file);
+        dsp().POST(file);
     }
 
     /** POST a dataset */
     public void postDataset(Dataset dataset) {
-        gsp().postDataset(dataset.asDatasetGraph());
+        dsp().POST(dataset.asDatasetGraph());
     }
 
     /**
@@ -285,16 +302,16 @@ public class ModelStore {
      * This operation does not parse the file.
      */
     public void putDataset(String file) {
-        gsp().putDataset(file);
+        dsp().PUT(file);
     }
 
     /** PUT a dataset */
     public void putDataset(Dataset dataset) {
-        gsp().putDataset(dataset.asDatasetGraph());
+        dsp().PUT(dataset.asDatasetGraph());
     }
 
     /** Clear - delete named graphs, empty the default graph - SPARQL "CLEAR ALL" */
     public void clearDataset() {
-        gsp().clearDataset();
+        dsp().clear();
     }
 }
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/DSP.java b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/DSP.java
new file mode 100644
index 0000000..1181836
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/DSP.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.exec.http;
+
+import java.net.http.HttpClient;
+import java.util.Map;
+
+import org.apache.jena.http.HttpEnv;
+import org.apache.jena.http.HttpRDF;
+import org.apache.jena.http.Push;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFFormat;
+import org.apache.jena.riot.RDFLanguages;
+import org.apache.jena.riot.WebContent;
+import org.apache.jena.riot.system.StreamRDFLib;
+import org.apache.jena.sparql.ARQException;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphFactory;
+import org.apache.jena.sparql.core.DatasetGraphZero;
+
+/**
+ * Client for dataset operations over HTTP.
+ * <p> for graph operations, see {@link GSP} which is the client-side of
+ * <a href="https://www.w3.org/TR/sparql11-http-rdf-update/">SPARQL 1.1 Graph Store Protocol</a>.
+ * <p>
+ * This class provided GET, POST, PUT and clear() on datasets.
+ * DELETE is not supported. HTTP DELETE means "remove resource", not "clear resource".
+ * <p>
+ * Examples:
+ * <pre>
+ *   // Get the dataset.
+ *   DatasetGraph graph = DSP.service("http://example/dataset").GET();
+ * </pre>
+ * <pre>
+ *   // POST (add) to a dataset
+ *   DatasetGraph myData = ...;
+ *   GSP.service("http://example/dataset").POST(myData);
+ * </pre>
+ */
+public class DSP extends StoreProtocol<DSP>{
+    /**
+     * Create a request to the remote service.
+     */
+    public static DSP service(String service) {
+        return new DSP().endpoint(service);
+    }
+
+    /**
+     * Create a request to the remote service (without GSP naming).
+     * Call {@link #endpoint} to set the target.
+     */
+    public static DSP request() {
+        return new DSP();
+    }
+
+    protected DSP() {}
+
+    @Override
+    protected DSP thisBuilder() { return this; }
+
+    /**
+     * GET dataset.
+     * <p>
+     * If the remote end is a graph, the result is a dataset with that
+     * graph data in the default graph of the dataset.
+     */
+    public DatasetGraph GET() {
+        ensureAcceptHeader(WebContent.defaultRDFAcceptHeader);
+        DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
+        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+        HttpRDF.httpGetToStream(hc, serviceEndpoint, httpHeaders, StreamRDFLib.dataset(dsg));
+        return dsg;
+    }
+
+    /**
+     * POST the contents of a file using the filename extension to determine the
+     * Content-Type to use if not already set.
+     * <p>
+     * This operation does not parse the file.
+     */
+    public void POST(String file) {
+        String fileExtContentType = contentTypeFromFilename(file);
+        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.POST);
+    }
+
+    /** POST a dataset */
+    public void POST(DatasetGraph dataset) {
+        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
+        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+        HttpRDF.httpPostDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+    }
+
+    /**
+     * PUT the contents of a file using the filename extension to determine the
+     * Content-Type to use if not already set.
+     * <p>
+     * This operation does not parse the file.
+     */
+    public void PUT(String file) {
+        String fileExtContentType = contentTypeFromFilename(file);
+        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.PUT);
+    }
+
+    /** PUT a dataset */
+    public void PUT(DatasetGraph dataset) {
+        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
+        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+        HttpRDF.httpPutDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+    }
+
+    private static DatasetGraph emptyDSG = DatasetGraphZero.create();
+    /** Clear - delete named graphs, empty the default graph - similar to SPARQL "CLEAR ALL" */
+    public void clear() {
+        // Without relying on SPARQL Update.
+        PUT(emptyDSG);
+    }
+
+    /**
+     * Send a file of quads to a URL. The Content-Type is inferred from the file
+     * extension.
+     */
+    private static void uploadQuads(HttpClient httpClient, String endpoint, String file, String fileExtContentType, Map<String, String> headers, Push mode) {
+        Lang lang = RDFLanguages.contentTypeToLang(fileExtContentType);
+        if ( !RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang) )
+            throw new ARQException("Not an RDF format: " + file + " (lang=" + lang + ")");
+        pushFile(httpClient, endpoint, file, fileExtContentType, headers, mode);
+    }
+}
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/GSP.java b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/GSP.java
index 1537378..eca92ba 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/GSP.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/GSP.java
@@ -18,33 +18,25 @@
 
 package org.apache.jena.sparql.exec.http;
 
-import java.io.FileNotFoundException;
 import java.net.http.HttpClient;
-import java.net.http.HttpRequest.BodyPublisher;
-import java.net.http.HttpRequest.BodyPublishers;
-import java.nio.file.Path;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.BiConsumer;
 
-import org.apache.jena.atlas.web.ContentType;
-import org.apache.jena.atlas.web.HttpException;
 import org.apache.jena.graph.Graph;
 import org.apache.jena.graph.Node;
 import org.apache.jena.http.HttpEnv;
 import org.apache.jena.http.HttpLib;
 import org.apache.jena.http.HttpRDF;
 import org.apache.jena.http.Push;
-import org.apache.jena.riot.*;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFFormat;
+import org.apache.jena.riot.RDFLanguages;
+import org.apache.jena.riot.WebContent;
 import org.apache.jena.riot.system.RiotLib;
 import org.apache.jena.riot.system.StreamRDFLib;
-import org.apache.jena.riot.system.StreamRDFWriter;
 import org.apache.jena.riot.web.HttpNames;
-import org.apache.jena.shared.NotFoundException;
 import org.apache.jena.sparql.ARQException;
 import org.apache.jena.sparql.core.DatasetGraph;
-import org.apache.jena.sparql.core.DatasetGraphFactory;
 import org.apache.jena.sparql.graph.GraphFactory;
 
 /**
@@ -67,33 +59,36 @@ import org.apache.jena.sparql.graph.GraphFactory;
  *   Graph myData = ...;
  *   GSP.service("http://example/dataset").namedGraph("http://my/graph").POST(myData);
  * </pre>
+ * <p>
+ * See {@link DSP} for operations on datasets.
  */
-public class GSP {
-    private String              serviceEndpoint = null;
-    // Need to keep this separately from contentType because it affects the choice of writer.
-    private RDFFormat           rdfFormat       = null;
-    private HttpClient          httpClient      = null;
-    private Map<String, String> httpHeaders     = new HashMap<>();
-
-    // One, and only one of these three, must be set at the point the terminating operation is called.
+public class GSP extends StoreProtocol<GSP> {
+
+    // One, and only one of these two, must be set at the point the terminating operation is called.
     // 1 - Graph operation, GSP naming, default graph
     private boolean             defaultGraph    = false;
     // 2 - Graph operation, GSP naming, graph name.
     private String              graphName       = null;
+
+    // Legacy, deprecated.
     // 3 - Dataset operation without ?default or ?graph=
     private boolean             datasetGraph    = false;
 
-    /** Create a request to the remote service (without GSP naming).
-     *  Call {@link #defaultGraph()} or {@link #graphName(String)} to select the target graph.
+    /**
+     * Create a request to the remote serviceURL (without a URL query string).
+     * Call {@link #defaultGraph()} or {@link #graphName(String)} to select the target graph.
+     * See {@link DSP} for dataset operations.
      * @param service
      */
     public static GSP service(String service) {
         return new GSP().endpoint(service);
     }
 
-    /** Create a request to the remote service (without GSP naming).
-     *  Call {@link #endpoint} to set the target.
-     *  Call {@link #defaultGraph()} or {@link #graphName(String)} to select the target graph.
+    /**
+     * Create a request to the remote service (without GSP naming).
+     * Call {@link #endpoint} to set the target.
+     * Call {@link #defaultGraph()} or {@link #graphName(String)} to select the target graph.
+     * See {@link DSP} for dataset operations.
      */
     public static GSP request() {
         return new GSP();
@@ -101,42 +96,8 @@ public class GSP {
 
     protected GSP() {}
 
-    /**
-     * Set the URL of the query endpoint. This replaces any value set in the
-     * {@link #service(String)} call.
-     */
-    public GSP endpoint(String serviceURL) {
-        this.serviceEndpoint = Objects.requireNonNull(serviceURL);
-        return this;
-    }
-
-    public GSP httpClient(HttpClient httpClient) {
-        Objects.requireNonNull(httpClient, "HttpClient");
-        this.httpClient = httpClient;
-        return this;
-    }
-
-    /**
-     * Set an HTTP header that is added to the request.
-     * See {@link #accept}, {@link #acceptHeader} and {@link #contentType(RDFFormat)}.
-     * for specific handling of {@code Accept:} and {@code Content-Type}.
-     */
-    public GSP httpHeader(String headerName, String headerValue) {
-        Objects.requireNonNull(headerName);
-        Objects.requireNonNull(headerValue);
-        if ( httpHeaders == null )
-            httpHeaders = new HashMap<>();
-        httpHeaders.put(headerName, headerValue);
-        return this;
-    }
-
-    // Private - no getters.
-    private String httpHeader(String header) {
-        Objects.requireNonNull(header);
-        if ( httpHeaders == null )
-            return null;
-        return httpHeaders.get(header);
-    }
+    @Override
+    protected GSP thisBuilder() { return this; }
 
     /** Send request for a named graph (that is, {@code ?graph=}) */
     public GSP graphName(String graphName) {
@@ -155,7 +116,6 @@ public class GSP {
         Node gn = RiotLib.blankNodeToIri(graphName);
         this.graphName = gn.getURI();
         this.defaultGraph = false;
-        this.datasetGraph = false;
         return this;
     }
 
@@ -166,110 +126,25 @@ public class GSP {
         return this;
     }
 
-    /** Send request for the dataset. This is "no GSP naming". */
+    /**
+     * Send request for the dataset. This is "no GSP naming".
+     * @deprecated Use {@link DSP}.
+     */
+    @Deprecated
     public GSP dataset() {
         clearOperation();
-        this.datasetGraph = true;
         return this;
     }
 
     private void clearOperation() {
         this.defaultGraph = false;
-        this.datasetGraph = false;
         this.graphName = null;
     }
 
-    /** Set the accept header on GET requests. Optional; if not set, a system default is used. */
-    public GSP acceptHeader(String acceptHeader) {
-        httpHeader(HttpNames.hAccept, acceptHeader);
-        return this;
-    }
-
-    // No getters.
-    private String acceptHeader() {
-        return httpHeader(HttpNames.hAccept);
-    }
-
-    /** Set the accept header on GET requests. Optional; if not set, a system default is used. */
-    public GSP accept(Lang lang) {
-        String acceptHeader = (lang != null ) ? lang.getContentType().getContentTypeStr() : null;
-        httpHeader(HttpNames.hAccept, acceptHeader);
-        return this;
-    }
-
-    /**
-     * Set the Content-type for a POST, PUT request of a file
-     * or serialization of a graph opf dataset is necessary.
-     * Optional; if not set, the file extension is used or the
-     * system default RDF syntax encoding.
-     */
-    public GSP contentTypeHeader(String contentType) {
-        httpHeader(HttpNames.hContentType, contentType);
-        return this;
-    }
-
-    // No getters.
-    private String contentType() {
-        return httpHeader(HttpNames.hContentType);
-    }
-
-    /**
-     * Set the Content-type for a POST, PUT request of a file
-     * or serialization of a graph opf dataset is necessary.
-     * Optional; if not set, the file extension is used or the
-     * system default RDF syntax encoding.
-     */
-    public GSP contentType(RDFFormat rdfFormat) {
-        this.rdfFormat = rdfFormat;
-        String contentType = rdfFormat.getLang().getContentType().getContentTypeStr();
-        httpHeader(HttpNames.hContentType, contentType);
-        return this;
-    }
-
     final protected void validateGraphOperation() {
         Objects.requireNonNull(serviceEndpoint);
         if ( ! defaultGraph && graphName == null )
             throw exception("Need either default graph or a graph name");
-        if ( datasetGraph )
-            throw exception("Dataset request specified for graph operation");
-    }
-
-    final protected void internalDataset() {
-        // Set as dataset request.
-        // Checking is done by validateDatasetOperation.
-        // The dataset operations have "Dataset" in the name, so less point having
-        // required dataset(). We can't use GET() because the return type
-        // would be "Graph or DatasetGraph"
-        // Reconsider if graph synonyms provided.
-        this.datasetGraph = true;
-    }
-
-    final protected void validateDatasetOperation() {
-        Objects.requireNonNull(serviceEndpoint);
-        if ( defaultGraph )
-            throw exception("Default graph specified for dataset operation");
-        if ( graphName != null )
-            throw exception("A graph name specified for dataset operation");
-        if ( ! datasetGraph )
-            throw exception("Dataset request not specified for dataset operation");
-    }
-
-    /**
-     * Choose the HttpClient to use.
-     * The requestURL includes the query string (for graph GSP operations).
-     * If explicit set with {@link #httpClient(HttpClient)}, use that;
-     * other use the system registry and default {@code HttpClient} settings
-     * in {@link HttpEnv}.
-     */
-    private HttpClient requestHttpClient(String serviceURL, String requestURL) {
-        if ( httpClient != null )
-            return httpClient;
-        return HttpEnv.getHttpClient(serviceURL, httpClient);
-    }
-
-    // Setup problems.
-    private static RuntimeException exception(String msg) {
-        return new HttpException(msg);
     }
 
     // Synonyms mirror the dataset names, so getGraph/getDataset
@@ -413,10 +288,6 @@ public class GSP {
 //        DELETE();
 //    }
 
-    private String graphRequestURL() {
-        return HttpLib.requestURL(serviceEndpoint, queryStringForGraph(graphName));
-    }
-
     /**
      * Return the query string for a graph using the
      * <a href="https://www.w3.org/TR/sparql11-http-rdf-update/">SPARQL 1.1 Graph Store Protocol</a>.
@@ -445,92 +316,101 @@ public class GSP {
     // Expose access for subclasses. "final" to ensure that this class controls constraints and expectations.
     // Only valid when the request has correctly been setup.
 
-    final protected String graphName() { return graphName; }
-    final protected String service() { return serviceEndpoint; }
+    final protected String graphName()           { return graphName; }
+    final protected boolean isDefaultGraph()     { return graphName == null; }
+    final protected boolean isGraphOperation()   { return defaultGraph || graphName != null; }
+
+    
+
+    
+
+    //    /**
+    //     * Delete a graph.
+    //     * <p>
+    //     * Synonym for {@link #DELETE()}.
+    //     */
+    //    public void deleteGraph() {
+    //        // Synonym
+    //        DELETE();
+    //    }
+    
+        private String graphRequestURL() {
+            return HttpLib.requestURL(serviceEndpoint, queryStringForGraph(graphName));
+        }
 
-    final protected boolean isDefaultGraph() { return graphName == null; }
-    final protected boolean isGraphOperation() { return defaultGraph || graphName != null; }
-    final protected boolean isDatasetOperation() { return datasetGraph; }
+    final protected void internalDataset() {
+            // Set as dataset request.
+            // Checking is done by validateDatasetOperation.
+            // The dataset operations have "Dataset" in the name, so less point having
+            // required dataset(). We can't use GET() because the return type
+            // would be "Graph or DatasetGraph"
+            // Reconsider if graph synonyms provided.
+            this.datasetGraph = true;
+        }
 
-    final protected HttpClient httpClient() { return httpClient; }
-    final protected void httpHeaders(BiConsumer<String, String> action) { httpHeaders.forEach(action); }
+    final protected void validateDatasetOperation() {
+            Objects.requireNonNull(serviceEndpoint);
+            if ( defaultGraph )
+                throw exception("Default graph specified for dataset operation");
+            if ( graphName != null )
+                throw exception("A graph name specified for dataset operation");
+            if ( ! datasetGraph )
+                throw exception("Dataset request not specified for dataset operation");
+        }
 
-    /**
-     * GET dataset.
-     * <p>
-     * If the remote end is a graph, the result is a dataset with that
-     * graph data in the default graph of the dataset.
-     */
+    // Redirect
+    /** @deprecated Use {@link DSP#GET()} */
+    @Deprecated
     public DatasetGraph getDataset() {
         internalDataset();
         validateDatasetOperation();
-        ensureAcceptHeader(WebContent.defaultRDFAcceptHeader);
-        DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
-        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
-        HttpRDF.httpGetToStream(hc, serviceEndpoint, httpHeaders, StreamRDFLib.dataset(dsg));
-        return dsg;
-    }
-
-    private void ensureAcceptHeader(String dftAcceptheader) {
-        String requestAccept = header(acceptHeader(), WebContent.defaultRDFAcceptHeader);
-        acceptHeader(requestAccept);
+        return newDSP().GET();
     }
 
-    /**
-     * POST the contents of a file using the filename extension to determine the
-     * Content-Type to use if not already set.
-     * <p>
-     * This operation does not parse the file.
-     */
+    /** @deprecated Use {@link DSP#POST(String)} */
+    @Deprecated
     public void postDataset(String file) {
         internalDataset();
         validateDatasetOperation();
-        String fileExtContentType = contentTypeFromFilename(file);
-        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
-        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.POST);
+        newDSP().POST(file);
     }
 
-    /** POST a dataset */
+    /** @deprecated Use {@link DSP#POST(DatasetGraph)} */
+    @Deprecated
     public void postDataset(DatasetGraph dataset) {
         internalDataset();
         validateDatasetOperation();
-        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
-        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
-        HttpRDF.httpPostDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+        newDSP().POST(dataset);
     }
 
-    /**
-     * PUT the contents of a file using the filename extension to determine the
-     * Content-Type to use if not already set.
-     * <p>
-     * This operation does not parse the file.
-     */
+    /** @deprecated Use {@link DSP#PUT(String)} */
+    @Deprecated
     public void putDataset(String file) {
         internalDataset();
         validateDatasetOperation();
-        String fileExtContentType = contentTypeFromFilename(file);
-        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
-        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.PUT);
+        newDSP().PUT(file);
     }
 
-    /** PUT a dataset */
+    /** @deprecated Use {@link DSP#PUT(DatasetGraph)} */
+    @Deprecated
     public void putDataset(DatasetGraph dataset) {
         internalDataset();
         validateDatasetOperation();
-        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
-        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
-        HttpRDF.httpPutDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+        newDSP().PUT(dataset);
     }
 
-    /** Clear - delete named graphs, empty the default graph - SPARQL "CLEAR ALL" */
+    /** @deprecated Use {@link DSP#clear()} */
+    @Deprecated
     public void clearDataset() {
         internalDataset();
         validateDatasetOperation();
-        // DELETE on a dataset URL is not supported in Fuseki.
-        // HTTP DELETE means "remove resource", not "clear resource".
-//        String url = serviceEndpoint;
-//        HttpOp.httpDelete(url);
-        UpdateExecHTTP.service(serviceEndpoint).update("CLEAR ALL").execute();
+        newDSP().clear();
+    }
+
+    private DSP newDSP() {
+        DSP dsp = new DSP();
+        dsp.copySetup(this);
+        return dsp;
     }
 
     /** Send a file of triples to a URL. */
@@ -545,66 +425,4 @@ public class GSP {
             throw new ARQException("Not an RDF format: "+file+" (lang="+lang+")");
         pushFile(httpClient, gspUrl, file, fileExtContentType, headers, mode);
     }
-
-    /**
-     * Send a file of quads to a URL. The Content-Type is inferred from the file
-     * extension.
-     */
-    private static void uploadQuads(HttpClient httpClient, String endpoint, String file, String fileExtContentType, Map<String, String> headers, Push mode) {
-        Lang lang = RDFLanguages.contentTypeToLang(fileExtContentType);
-        if ( !RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang) )
-            throw new ARQException("Not an RDF format: " + file + " (lang=" + lang + ")");
-        pushFile(httpClient, endpoint, file, fileExtContentType, headers, mode);
-    }
-
-    /** Header string or default value. */
-    private static String header(String choice, String dftString) {
-        return choice != null ? choice : dftString;
-    }
-
-    /** Choose the format to write in.
-     * <ol>
-     * <li> {@code rdfFormat}
-     * <li> {@code contentType} setting, choosing streaming
-     * <li> {@code contentType} setting, choosing pretty
-     * <li> HttpEnv.dftTriplesFormat / HttpEnv.dftQuadsFormat /
-     * </ol>
-     */
-    private RDFFormat rdfFormat(RDFFormat dftFormat) {
-        if ( rdfFormat != null )
-            return rdfFormat;
-
-        if ( contentType() == null )
-            return dftFormat;
-
-        Lang lang = RDFLanguages.contentTypeToLang(contentType());
-        RDFFormat streamFormat = StreamRDFWriter.defaultSerialization(null);
-        if ( streamFormat != null )
-            return streamFormat;
-        return RDFWriterRegistry.defaultSerialization(lang);
-    }
-
-    /** Choose the Content-Type header for sending a file unless overridden. */
-    private String contentTypeFromFilename(String filename) {
-        String ctx = contentType();
-        if ( ctx != null )
-            return ctx;
-        ContentType ct = RDFLanguages.guessContentType(filename);
-        return ct == null ? null : ct.getContentTypeStr();
-    }
-
-    /** Send a file. fileContentType takes precedence over this.contentType.*/
-    protected static void pushFile(HttpClient httpClient, String endpoint, String file, String fileContentType,
-                                   Map<String, String> httpHeaders, Push style) {
-        try {
-            Path path = Path.of(file);
-            if ( fileContentType != null )
-            //if ( ! httpHeaders.containsKey(HttpNames.hContentType) )
-                httpHeaders.put(HttpNames.hContentType, fileContentType);
-            BodyPublisher body = BodyPublishers.ofFile(path);
-            HttpLib.httpPushData(httpClient, style, endpoint, HttpLib.setHeaders(httpHeaders), body);
-        } catch (FileNotFoundException ex) {
-            throw new NotFoundException(file);
-        }
-    }
 }
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/StoreProtocol.java b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/StoreProtocol.java
new file mode 100644
index 0000000..cd07d49
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/exec/http/StoreProtocol.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.exec.http;
+
+import java.io.FileNotFoundException;
+import java.net.http.HttpClient;
+import java.net.http.HttpRequest.BodyPublisher;
+import java.net.http.HttpRequest.BodyPublishers;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.BiConsumer;
+
+import org.apache.jena.atlas.web.ContentType;
+import org.apache.jena.atlas.web.HttpException;
+import org.apache.jena.http.HttpEnv;
+import org.apache.jena.http.HttpLib;
+import org.apache.jena.http.Push;
+import org.apache.jena.riot.*;
+import org.apache.jena.riot.system.StreamRDFWriter;
+import org.apache.jena.riot.web.HttpNames;
+import org.apache.jena.shared.NotFoundException;
+
+/**
+ * State and settings management for {@link GSP Graph Store Protocol}
+ * and {@link DSP Dataset Store Protocol} clients.
+ */
+public abstract class StoreProtocol<X extends StoreProtocol<X>> {
+    protected String              serviceEndpoint = null;
+    // Need to keep this separately from contentType because it affects the choice of writer.
+    protected RDFFormat           rdfFormat       = null;
+    protected HttpClient          httpClient      = null;
+    protected Map<String, String> httpHeaders     = new HashMap<>();
+
+    protected StoreProtocol() {}
+
+    protected abstract X thisBuilder();
+
+    /**
+     * Set the URL of the query endpoint.
+     */
+    public X endpoint(String serviceURL) {
+        this.serviceEndpoint = Objects.requireNonNull(serviceURL);
+        return thisBuilder();
+    }
+
+    protected String endpoint() {
+        return serviceEndpoint;
+    }
+
+    public X httpClient(HttpClient httpClient) {
+        Objects.requireNonNull(httpClient, "HttpClient");
+        this.httpClient = httpClient;
+        return thisBuilder();
+    }
+
+    /**
+     * Set an HTTP header that is added to the request.
+     * See {@link #accept}, {@link #acceptHeader} and {@link #contentType(RDFFormat)}.
+     * for specific handling of {@code Accept:} and {@code Content-Type}.
+     */
+    public X httpHeader(String headerName, String headerValue) {
+        Objects.requireNonNull(headerName);
+        Objects.requireNonNull(headerValue);
+        if ( httpHeaders == null )
+            httpHeaders = new HashMap<>();
+        httpHeaders.put(headerName, headerValue);
+        return thisBuilder();
+    }
+
+    // Protected - no public getters.
+    protected String httpHeader(String header) {
+        Objects.requireNonNull(header);
+        if ( httpHeaders == null )
+            return null;
+        return httpHeaders.get(header);
+    }
+
+    /** Set the accept header on GET requests. Optional; if not set, a system default is used. */
+    public X acceptHeader(String acceptHeader) {
+        httpHeader(HttpNames.hAccept, acceptHeader);
+        return thisBuilder();
+    }
+
+    // No getters.
+    protected String acceptHeader() {
+        return httpHeader(HttpNames.hAccept);
+    }
+
+    /** Set the accept header on GET requests. Optional; if not set, a system default is used. */
+    public X accept(Lang lang) {
+        String acceptHeader = (lang != null ) ? lang.getContentType().getContentTypeStr() : null;
+        httpHeader(HttpNames.hAccept, acceptHeader);
+        return thisBuilder();
+    }
+
+    /**
+     * Set the Content-type for a POST, PUT request of a file
+     * or serialization of a graph of dataset is necessary.
+     * Optional; if not set, the file extension is used or the
+     * system default RDF syntax encoding.
+     */
+    public X contentTypeHeader(String contentType) {
+        httpHeader(HttpNames.hContentType, contentType);
+        return thisBuilder();
+    }
+
+    // No getters.
+    protected String contentType() {
+        return httpHeader(HttpNames.hContentType);
+    }
+
+    /**
+     * Set the Content-type for a POST, PUT request of a file
+     * or serialization of a graph of dataset is necessary.
+     * Optional; if not set, the file extension is used or the
+     * system default RDF syntax encoding.
+     */
+    public X contentType(RDFFormat rdfFormat) {
+        this.rdfFormat = rdfFormat;
+        String contentType = rdfFormat.getLang().getContentType().getContentTypeStr();
+        httpHeader(HttpNames.hContentType, contentType);
+        return thisBuilder();
+    }
+
+    protected RDFFormat rdfFormat() { return rdfFormat; }
+
+    /** Copy the state (endpoint, HttpClient, HTTP headers, RDFFormat) of one StoreProtocol into this one.
+     * Any old setup on tghis object is lost.
+     * */
+    public X copySetup(StoreProtocol<?> other) {
+        clearSetup();
+        other.httpHeaders(this::httpHeader);
+        this.endpoint(other.endpoint());
+        if (other.httpClient() != null )
+            this.httpClient(httpClient());
+        if ( other.rdfFormat() != null )
+            this.rdfFormat(other.rdfFormat());
+        return thisBuilder();
+    }
+
+    private void clearSetup() {
+        serviceEndpoint = null;
+        rdfFormat       = null;
+        httpClient      = null;
+        httpHeaders     = new HashMap<>();
+    }
+
+    /**
+     * Choose the HttpClient to use.
+     * The requestURL includes the query string (for graph GSP operations).
+     * If explicit set with {@link #httpClient(HttpClient)}, use that;
+     * other use the system registry and default {@code HttpClient} settings
+     * in {@link HttpEnv}.
+     */
+    protected HttpClient requestHttpClient(String serviceURL, String requestURL) {
+        if ( httpClient != null )
+            return httpClient;
+        return HttpEnv.getHttpClient(serviceURL, httpClient);
+    }
+
+    // Setup problems.
+    protected static RuntimeException exception(String msg) {
+        return new HttpException(msg);
+    }
+
+    final protected String service() { return serviceEndpoint; }
+    final protected HttpClient httpClient() { return httpClient; }
+    final protected void httpHeaders(BiConsumer<String, String> action) { httpHeaders.forEach(action); }
+
+    protected void ensureAcceptHeader(String dftAcceptheader) {
+        String requestAccept = header(acceptHeader(), WebContent.defaultRDFAcceptHeader);
+        acceptHeader(requestAccept);
+    }
+
+//    /**
+//     * POST the contents of a file using the filename extension to determine the
+//     * Content-Type to use if not already set.
+//     * <p>
+//     * This operation does not parse the file.
+//     */
+//    public void POST(String file) {
+//        String fileExtContentType = contentTypeFromFilename(file);
+//        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+//        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.POST);
+//    }
+//
+//    /** POST a dataset */
+//    public void POST(DatasetGraph dataset) {
+//        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
+//        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+//        HttpRDF.httpPostDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+//    }
+//
+//    /**
+//     * PUT the contents of a file using the filename extension to determine the
+//     * Content-Type to use if not already set.
+//     * <p>
+//     * This operation does not parse the file.
+//     */
+//    public void PUT(String file) {
+//        String fileExtContentType = contentTypeFromFilename(file);
+//        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+//        uploadQuads(hc, serviceEndpoint, file, fileExtContentType, httpHeaders, Push.PUT);
+//    }
+//
+//    /** PUT a dataset */
+//    public void PUT(DatasetGraph dataset) {
+//        RDFFormat requestFmt = rdfFormat(HttpEnv.defaultQuadsFormat);
+//        HttpClient hc = requestHttpClient(serviceEndpoint, serviceEndpoint);
+//        HttpRDF.httpPutDataset(hc, serviceEndpoint, dataset, requestFmt, httpHeaders);
+//    }
+//
+////    /** Send a file of triples to a URL. */
+////    private static void uploadTriples(HttpClient httpClient, String gspUrl, String file, String fileExtContentType,
+////                                      Map<String, String> headers, Push mode) {
+////        Lang lang = RDFLanguages.contentTypeToLang(fileExtContentType);
+////        if ( lang == null )
+////            throw new ARQException("Not a recognized as an RDF format: "+fileExtContentType);
+////        if ( RDFLanguages.isQuads(lang) && ! RDFLanguages.isTriples(lang) )
+////            throw new ARQException("Can't load quads into a graph");
+////        if ( ! RDFLanguages.isTriples(lang) )
+////            throw new ARQException("Not an RDF format: "+file+" (lang="+lang+")");
+////        pushFile(httpClient, gspUrl, file, fileExtContentType, headers, mode);
+////    }
+//
+//    /**
+//     * Send a file of quads to a URL. The Content-Type is inferred from the file
+//     * extension.
+//     */
+//    private static void uploadQuads(HttpClient httpClient, String endpoint, String file, String fileExtContentType, Map<String, String> headers, Push mode) {
+//        Lang lang = RDFLanguages.contentTypeToLang(fileExtContentType);
+//        if ( !RDFLanguages.isQuads(lang) && !RDFLanguages.isTriples(lang) )
+//            throw new ARQException("Not an RDF format: " + file + " (lang=" + lang + ")");
+//        pushFile(httpClient, endpoint, file, fileExtContentType, headers, mode);
+//    }
+
+    /** Header string or default value. */
+    private static String header(String choice, String dftString) {
+        return choice != null ? choice : dftString;
+    }
+
+    /** Choose the format to write in.
+     * <ol>
+     * <li> {@code rdfFormat}
+     * <li> {@code contentType} setting, choosing streaming
+     * <li> {@code contentType} setting, choosing pretty
+     * <li> HttpEnv.dftTriplesFormat / HttpEnv.dftQuadsFormat /
+     * </ol>
+     */
+    protected RDFFormat rdfFormat(RDFFormat dftFormat) {
+        if ( rdfFormat != null )
+            return rdfFormat;
+
+        if ( contentType() == null )
+            return dftFormat;
+
+        Lang lang = RDFLanguages.contentTypeToLang(contentType());
+        RDFFormat streamFormat = StreamRDFWriter.defaultSerialization(null);
+        if ( streamFormat != null )
+            return streamFormat;
+        return RDFWriterRegistry.defaultSerialization(lang);
+    }
+
+    /** Choose the Content-Type header for sending a file unless overridden. */
+    protected String contentTypeFromFilename(String filename) {
+        String ctx = contentType();
+        if ( ctx != null )
+            return ctx;
+        ContentType ct = RDFLanguages.guessContentType(filename);
+        return ct == null ? null : ct.getContentTypeStr();
+    }
+
+    /** Send a file. fileContentType takes precedence over this.contentType.*/
+    protected static void pushFile(HttpClient httpClient, String endpoint, String file, String fileContentType,
+                                   Map<String, String> httpHeaders, Push style) {
+        try {
+            Path path = Path.of(file);
+            if ( fileContentType != null )
+            //if ( ! httpHeaders.containsKey(HttpNames.hContentType) )
+                httpHeaders.put(HttpNames.hContentType, fileContentType);
+            BodyPublisher body = BodyPublishers.ofFile(path);
+            HttpLib.httpPushData(httpClient, style, endpoint, HttpLib.setHeaders(httpHeaders), body);
+        } catch (FileNotFoundException ex) {
+            throw new NotFoundException(file);
+        }
+    }
+
+}
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TS_SparqlExecHttp.java b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TS_SparqlExecHttp.java
index 1d6493b..79cc1fd 100644
--- a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TS_SparqlExecHttp.java
+++ b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TS_SparqlExecHttp.java
@@ -24,6 +24,8 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
       TestGSP.class
+    , TestDSP.class
+    , TestModelStore.class
     , TestQueryExecHTTP.class
     , TestQueryExecCleanServer.class
     , TestUpdateExecHTTP.class
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestDSP.java b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestDSP.java
new file mode 100644
index 0000000..4927605
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestDSP.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.exec.http;
+
+import static org.apache.jena.fuseki.test.HttpTest.expect404;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.StringReader;
+
+import org.apache.jena.http.HttpOp;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphFactory;
+import org.apache.jena.sparql.util.IsoMatcher;
+import org.apache.jena.test.conn.EnvTest;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestDSP {
+
+    static String DIR = "testing/RDFLink/";
+
+    private static EnvTest env;
+    @BeforeClass public static void beforeClass() {
+        env = EnvTest.create("/ds");
+    }
+
+    @Before public void before() {
+        env.clear();
+    }
+
+    @AfterClass public static void afterClass() {
+        EnvTest.stop(env);
+    }
+
+    private String url(String path) { return env.datasetPath(path); }
+
+    // Test DSP against the dataset itself.
+    static String dspServiceURL()   { return env.datasetPath("/"); }
+
+    static DatasetGraph dataset = makeDatasetGraph();
+    static DatasetGraph makeDatasetGraph() {
+        DatasetGraph dataset = DatasetGraphFactory.createTxnMem();
+        RDFDataMgr.read(dataset, new StringReader("PREFIX : <http://example/> :s :p :o . :g { :sg :pg :og }"), null, Lang.TRIG);
+        return dataset;
+    }
+
+    // ----------------------------------------
+
+    @Test public void dsp_put_get_01() {
+        DSP.service(dspServiceURL()).PUT(dataset);
+        DatasetGraph dsg = DSP.service(dspServiceURL()).GET();
+        assertNotNull(dsg);
+        assertTrue(IsoMatcher.isomorphic(dataset, dsg));
+    }
+
+    @Test public void dsp_post_get_02() {
+        DSP.service(dspServiceURL()).POST(dataset);
+        DatasetGraph dsg = DSP.service(dspServiceURL()).GET();
+        assertNotNull(dsg);
+        assertTrue(IsoMatcher.isomorphic(dataset, dsg));
+    }
+
+    @Test public void dspHead_dataset_1() {
+        // Base dspServiceURL(), default content type => N-Quads (dump format)
+        String h = HttpOp.httpHead(dspServiceURL(), null);
+        assertNotNull(h);
+        assertEquals(Lang.NQUADS.getHeaderString(), h);
+    }
+
+    @Test public void dspHead_dataset_2() {
+        String ct = Lang.TRIG.getHeaderString();
+        String h = HttpOp.httpHead(dspServiceURL(), ct);
+        assertNotNull(h);
+        assertEquals(ct, h);
+    }
+
+    @Test public void dspHead_graph_1() {
+        String target = dspServiceURL()+"?default";
+        String h = HttpOp.httpHead(target, null);
+        assertNotNull(h);
+        // "Traditional default".
+        assertEquals(Lang.RDFXML.getHeaderString(), h);
+    }
+
+    @Test public void dspHead_graph_2() {
+        String target = dspServiceURL()+"?default";
+        String ct = Lang.TTL.getHeaderString();
+        String h = HttpOp.httpHead(target, ct);
+        assertNotNull(h);
+        assertEquals(ct, h);
+    }
+
+    @Test
+    public void dsp_clear_01() {
+        // DELETE on the DSP endpoint would be the default graph.
+        // DELETE on the dataset endpoint is not supported by Fuseki - this does "CLER ALL"
+        DSP.service(env.datasetURL()).clear();
+    }
+
+    @Test
+    public void dsp_clear_02() {
+        DSP.service(dspServiceURL()).POST(dataset);
+        DSP.service(env.datasetURL()).clear();
+        DatasetGraph dsg = DSP.service(dspServiceURL()).GET();
+        assertFalse(dsg.find().hasNext());
+    }
+
+    @Test public void dsp_put_delete_01() {
+        DSP.service(dspServiceURL()).PUT(dataset);
+        DSP.service(dspServiceURL()).clear();
+        DatasetGraph dsg = DSP.service(dspServiceURL()).GET();
+        assertTrue(dsg.isEmpty());
+    }
+
+    // 404
+
+    @Test public void dsp_404_dataset() {
+        expect404(
+            ()->DSP.service(dspServiceURL()+"junk").GET()
+        );
+    }
+}
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestGSP.java b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestGSP.java
index fbefb85..fa1f872 100644
--- a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestGSP.java
+++ b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestGSP.java
@@ -68,6 +68,7 @@ public class TestGSP {
 
     private String url(String path) { return env.datasetPath(path); }
 
+    // Test GSP against the /data endpoint (including dataset operations).
     static String gspServiceURL()   { return env.datasetPath("/data"); }
 
     static String defaultGraphURL() { return gspServiceURL()+"?default"; }
@@ -179,6 +180,7 @@ public class TestGSP {
 
     // ----------------------------------------
 
+    @SuppressWarnings("deprecation")
     @Test public void gsp_ds_put_get_01() {
         GSP.service(gspServiceURL()).putDataset(dataset);
         DatasetGraph dsg = GSP.service(gspServiceURL()).getDataset();
@@ -186,6 +188,7 @@ public class TestGSP {
         assertTrue(IsoMatcher.isomorphic(dataset, dsg));
     }
 
+    @SuppressWarnings("deprecation")
     @Test public void gsp_ds_post_get_02() {
         GSP.service(gspServiceURL()).postDataset(dataset);
         DatasetGraph dsg = GSP.service(gspServiceURL()).getDataset();
@@ -193,61 +196,38 @@ public class TestGSP {
         assertTrue(IsoMatcher.isomorphic(dataset, dsg));
     }
 
-    @Test(expected=HttpException.class)
-    public void gsp_ds_err_01() {
-        GSP.service(gspServiceURL()).defaultGraph().putDataset(dataset);
-    }
-
-    @Test
-    public void gsp_head_01() {
-        // HEAD on the GSP endpoint would be the default graph.
-        // DELETE on the dataset endpoint is not supported by Fuseki - this does "CLER ALL"
-        GSP.service(env.datasetURL()).clearDataset();
-    }
-
-    /*
     @Test public void gspHead_dataset_1() {
         // Base URL, default content type => N-Quads (dump format)
-        HttpOp.execHttpHead(URL, null, (base, response)->{
-            String h = response.getFirstHeader(HttpNames.hContentType).getValue();
-            assertNotNull(h);
-            assertEquals(Lang.NQUADS.getHeaderString(), h);
-        });
+        String h = HttpOp.httpHead(gspServiceURL(), null);
+        assertNotNull(h);
+        assertEquals(Lang.NQUADS.getHeaderString(), h);
     }
 
 
     @Test public void gspHead_dataset_2() {
         String ct = Lang.TRIG.getHeaderString();
-        HttpOp.execHttpHead(URL, ct, (base, response)->{
-            String h = response.getFirstHeader(HttpNames.hContentType).getValue();
-            assertNotNull(h);
-            assertEquals(ct, h);
-        });
+        String h = HttpOp.httpHead(gspServiceURL(), ct);
+        assertNotNull(h);
+        assertEquals(ct, h);
     }
 
     @Test public void gspHead_graph_1() {
-        String target = URL+"?default";
-        HttpOp.execHttpHead(target, null, (base, response)->{
-            String h = response.getFirstHeader(HttpNames.hContentType).getValue();
-            assertNotNull(h);
-            // "Traditional default".
-            assertEquals(Lang.RDFXML.getHeaderString(), h);
-        });
+        String target = defaultGraphURL();
+        String h = HttpOp.httpHead(target, null);
+        assertNotNull(h);
+        // "Traditional default".
+        assertEquals(Lang.RDFXML.getHeaderString(), h);
     }
 
     @Test public void gspHead_graph_2() {
-        String target = URL+"?default";
+        String target = defaultGraphURL();
         String ct = Lang.TTL.getHeaderString();
-        HttpOp.execHttpHead(target, ct, (base, response)->{
-            String h = response.getFirstHeader(HttpNames.hContentType).getValue();
-            assertNotNull(h);
-            assertEquals(ct, h);
-        });
+        String h = HttpOp.httpHead(target, ct);
+        assertNotNull(h);
+        assertEquals(ct, h);
     }
 
-     */
-
-
+    @SuppressWarnings("deprecation")
     @Test
     public void gsp_ds_clear_01() {
         // DELETE on the GSP endpoint would be the default graph.
@@ -255,6 +235,7 @@ public class TestGSP {
         GSP.service(env.datasetURL()).clearDataset();
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void gsp_ds_clear_02() {
         GSP.service(gspServiceURL()).postDataset(dataset);
@@ -264,12 +245,13 @@ public class TestGSP {
     }
 
 
-//    @Test public void gsp_ds_put_delete_01() {
-//        GSP.request(gspServiceURL()).putDataset(dataset);
-//        GSP.request(gspServiceURL()).clearDataset();
-//        DatasetGraph dsg = GSP.request(gspServiceURL()).getDataset();
-//        assertTrue(dsg.isEmpty());
-//    }
+    @SuppressWarnings("deprecation")
+    @Test public void gsp_ds_put_delete_01() {
+        GSP.service(gspServiceURL()).putDataset(dataset);
+        GSP.service(gspServiceURL()).clearDataset();
+        DatasetGraph dsg = GSP.service(gspServiceURL()).getDataset();
+        assertTrue(dsg.isEmpty());
+    }
 
     @Test public void gsp_union_get() {
         Node gn1 = NodeFactory.createURI("http://example/graph1");
@@ -294,7 +276,7 @@ public class TestGSP {
 
     // 404
 
-    @Test public void gsp_404_01() {
+    @Test public void gsp_404_put_delete_get() {
         String graphName = "http://example/graph2";
         Node gn = NodeFactory.createURI("http://example/graph2");
         GSP.service(gspServiceURL())
@@ -307,7 +289,6 @@ public class TestGSP {
         GSP.service(gspServiceURL())
             .graphName(gn)
             .DELETE();
-
         expect404(()->
             GSP.service(gspServiceURL())
                 .graphName(graphName)
@@ -315,14 +296,15 @@ public class TestGSP {
         );
     }
 
-    @Test public void gsp_404_1() {
+    @Test public void gsp_404_graph() {
         String graphName = "http://example/graph404";
         expect404(
             ()->GSP.service(gspServiceURL()).graphName(graphName).GET()
         );
     }
 
-    @Test public void gsp_404_2() {
+    @SuppressWarnings("deprecation")
+    @Test public void gsp_404_dataset() {
         expect404(
             ()->GSP.service(gspServiceURL()+"junk").getDataset()
         );
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestModelStore.java b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestModelStore.java
new file mode 100644
index 0000000..5909fd4
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/sparql/exec/http/TestModelStore.java
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.exec.http;
+
+import static org.apache.jena.fuseki.test.HttpTest.expect400;
+import static org.apache.jena.fuseki.test.HttpTest.expect404;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.StringReader;
+
+import org.apache.jena.atlas.web.HttpException;
+import org.apache.jena.http.HttpOp;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.query.DatasetFactory;
+import org.apache.jena.query.ModelStore;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.RDFFormat;
+import org.apache.jena.riot.WebContent;
+import org.apache.jena.sparql.sse.SSE;
+import org.apache.jena.sparql.util.IsoMatcher;
+import org.apache.jena.test.conn.EnvTest;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestModelStore {
+
+    static String DIR = "testing/RDFLink/";
+
+    private static EnvTest env;
+    @BeforeClass public static void beforeClass() {
+        env = EnvTest.create("/ds");
+    }
+
+    @Before public void before() {
+        env.clear();
+    }
+
+    @AfterClass public static void afterClass() {
+        EnvTest.stop(env);
+    }
+
+    private static Model model1 = ModelFactory.createModelForGraph(SSE.parseGraph("(graph (:s :p :x) (:s :p 1))"));
+    private static Model model2 = ModelFactory.createModelForGraph(SSE.parseGraph("(graph (:s :p :x) (:s :p 2))"));
+
+    private String url(String path) { return env.datasetPath(path); }
+
+    static String gspServiceURL()   { return env.datasetPath("/data"); }
+
+    static String defaultGraphURL() { return gspServiceURL()+"?default"; }
+    static String namedGraphURL()   { return gspServiceURL()+"?graph=http://example/g"; }
+
+    // Graph, with one triple in it.
+    static Model graph = makeModel();
+    static Model makeModel() {
+        Model graph = ModelFactory.createDefaultModel();
+        RDFDataMgr.read(graph, new StringReader("PREFIX : <http://example/> :s :p :o ."), null, Lang.TTL);
+        return graph;
+    }
+
+    static Dataset dataset = makeDataset();
+    static Dataset makeDataset() {
+        Dataset dataset = DatasetFactory.createTxnMem();
+        RDFDataMgr.read(dataset, new StringReader("PREFIX : <http://example/> :s :p :o . :g { :sg :pg :og }"), null, Lang.TRIG);
+        return dataset;
+    }
+
+    @Test public void gsp_put_get_01() {
+        ModelStore.service(gspServiceURL())
+            .defaultModel()
+            .PUT(graph);
+        Model m = ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .GET();
+        assertNotNull(m);
+        assertTrue(graph.isIsomorphicWith(m));
+    }
+
+    @Test(expected=HttpException.class)
+    public void gsp_bad_put_01() {
+        // No .defaultGraph
+        ModelStore.service(gspServiceURL()).PUT(graph);
+    }
+
+    @Test(expected=HttpException.class)
+    public void gsp_bad_get_err_02() {
+        // No .defaultGraph
+        ModelStore.service(gspServiceURL()).GET();
+    }
+
+    @Test public void gsp_post_get_ct_01() {
+        String graphName = "http://example/graph";
+        ModelStore.service(gspServiceURL())
+            .namedGraph(graphName)
+            .POST(graph);
+        Model m1 = ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .acceptHeader("application/rdf+xml")
+            .GET();
+        assertNotNull(m1);
+        assertTrue(m1.isEmpty());
+
+        Model m2 = ModelStore.service(gspServiceURL())
+            .namedGraph(graphName)
+            .acceptHeader("application/rdf+xml")
+            .GET();
+        assertNotNull(m2);
+        assertFalse(m2.isEmpty());
+        assertTrue(graph.isIsomorphicWith(m2));
+    }
+
+    @Test public void gsp_put_get_ct_02() {
+        ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .contentType(RDFFormat.NTRIPLES)
+            .PUT(graph);
+        Model m1 = ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .accept(Lang.RDFXML)
+            .GET();
+        assertNotNull(m1);
+        assertFalse(m1.isEmpty());
+        assertTrue(graph.isIsomorphicWith(m1));
+    }
+
+    @Test public void gsp_put_delete_01() {
+        ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .PUT(graph);
+        Model m1 = ModelStore.service(gspServiceURL())
+             .defaultGraph()
+             .GET();
+        assertFalse(m1.isEmpty());
+
+        ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .DELETE();
+        Model m2 = ModelStore.service(gspServiceURL())
+            .defaultGraph()
+            .GET();
+        assertTrue(m2.isEmpty());
+
+        // And just to make sure ...
+        String s2 = HttpOp.httpGetString(defaultGraphURL(), WebContent.contentTypeNTriples);
+        // Default always exists so this is the empty graph in N-triples.
+        assertTrue(s2.isEmpty());
+    }
+
+    @Test public void gsp_dft_ct_1() {
+        ModelStore.service(url("/ds")).defaultGraph().contentType(RDFFormat.RDFXML).PUT(DIR+"data-rdfxml");
+    }
+
+    @Test public void gsp_dft_ct_2() {
+        ModelStore.service(url("/ds")).defaultGraph().contentTypeHeader(WebContent.contentTypeRDFXML).PUT(DIR+"data-rdfxml");
+    }
+
+    // ----------------------------------------
+
+    @Test public void dsp_put_get_01() {
+        ModelStore.service(gspServiceURL()).putDataset(dataset);
+        Dataset ds = ModelStore.service(gspServiceURL()).getDataset();
+        assertNotNull(ds);
+        assertTrue(IsoMatcher.isomorphic(dataset.asDatasetGraph(), ds.asDatasetGraph()));
+    }
+
+    @Test public void dsp_post_get_02() {
+        ModelStore.service(gspServiceURL()).postDataset(dataset);
+        Dataset ds = ModelStore.service(gspServiceURL()).getDataset();
+        assertNotNull(ds);
+        assertTrue(IsoMatcher.isomorphic(dataset.asDatasetGraph(), ds.asDatasetGraph()));
+    }
+
+    // Not an error for ModelStore.
+//    @Test(expected=HttpException.class)
+//    public void dsp_err_01() {
+//        ModelStore.service(gspServiceURL()).defaultGraph().putDataset(dataset);
+//    }
+
+    @Test
+    public void gsp_head_01() {
+        // HEAD on the GSP endpoint would be the default graph.
+        // DELETE on the dataset endpoint is not supported by Fuseki - this does "CLER ALL"
+        ModelStore.service(env.datasetURL()).clearDataset();
+    }
+
+    @Test
+    public void dsp_clear_01() {
+        // DELETE on the GSP endpoint would be the default graph.
+        // DELETE on the dataset endpoint is not supported by Fuseki - this does "CLER ALL"
+        ModelStore.service(env.datasetURL()).clearDataset();
+    }
+
+    @Test
+    public void dsp_clear_02() {
+        ModelStore.service(gspServiceURL()).postDataset(dataset);
+        ModelStore.service(env.datasetURL()).clearDataset();
+        Dataset ds = ModelStore.service(gspServiceURL()).getDataset();
+        assertTrue(ds.isEmpty());
+    }
+
+
+    @Test public void dsp_put_delete_01() {
+        ModelStore.service(gspServiceURL()).putDataset(dataset);
+        ModelStore.service(gspServiceURL()).clearDataset();
+        Dataset ds = ModelStore.service(gspServiceURL()).getDataset();
+        assertTrue(ds.isEmpty());
+    }
+
+    @Test public void gsp_union_get() {
+        String gn1 = "http://example/graph1";
+        String gn2 = "http://example/graph2";
+        ModelStore.service(gspServiceURL())
+           .namedGraph(gn1)
+           .PUT(model1);
+        ModelStore.service(gspServiceURL())
+           .namedGraph(gn2)
+            .PUT(model2);
+        // get union
+
+        Model m = ModelStore.service(gspServiceURL()).namedGraph("union").GET();
+        assertEquals(3, m.size());
+    }
+
+    @Test public void gsp_union_post() {
+        expect400(()->{
+            ModelStore.service(gspServiceURL()).namedGraph("union").POST(model1);
+        });
+    }
+
+    // 404
+
+    @Test public void gsp_404_put_delete_get() {
+        String graphName = "http://example/graph2";
+        ModelStore.service(gspServiceURL())
+            .namedGraph(graphName)
+            .PUT(graph);
+        Model g = ModelStore.service(gspServiceURL())
+            .namedGraph(graphName)
+            .GET();
+        assertFalse(g.isEmpty());
+        ModelStore.service(gspServiceURL())
+            .namedGraph(graphName)
+            .DELETE();
+        expect404(()->
+            ModelStore.service(gspServiceURL())
+                .namedGraph(graphName)
+                .GET()
+        );
+    }
+
+    @Test public void gsp_404_graph() {
+        String graphName = "http://example/graph404";
+        expect404(
+            ()->ModelStore.service(gspServiceURL()).namedGraph(graphName).GET()
+        );
+    }
+
+    @Test public void dsp_404_dataset() {
+        expect404(
+            ()->ModelStore.service(gspServiceURL()+"junk").getDataset()
+        );
+    }
+}
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdflink/RDFLinkHTTP.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdflink/RDFLinkHTTP.java
index e75fea9..dc4285d 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdflink/RDFLinkHTTP.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdflink/RDFLinkHTTP.java
@@ -35,6 +35,7 @@ import org.apache.jena.sparql.core.Transactional;
 import org.apache.jena.sparql.core.TransactionalLock;
 import org.apache.jena.sparql.engine.binding.Binding;
 import org.apache.jena.sparql.exec.*;
+import org.apache.jena.sparql.exec.http.DSP;
 import org.apache.jena.sparql.exec.http.GSP;
 import org.apache.jena.sparql.exec.http.QueryExecHTTPBuilder;
 import org.apache.jena.sparql.exec.http.UpdateExecHTTPBuilder;
@@ -470,6 +471,10 @@ public class RDFLinkHTTP implements RDFLink {
         return GSP.service(svcGraphStore).httpClient(httpClient);
     }
 
+    private DSP dspRequest() {
+        return DSP.service(svcGraphStore).httpClient(httpClient);
+    }
+
     @Override
     public void delete(Node graphName) {
         checkGSP();
@@ -485,31 +490,31 @@ public class RDFLinkHTTP implements RDFLink {
     @Override
     public DatasetGraph getDataset() {
         checkDataset();
-        return gspRequest().dataset().acceptHeader(acceptDataset).getDataset();
+        return dspRequest().acceptHeader(acceptDataset).GET();
     }
 
     @Override
     public void loadDataset(String file) {
         checkDataset();
-        gspRequest().dataset().postDataset(file);
+        dspRequest().POST(file);
     }
 
     @Override
     public void loadDataset(DatasetGraph dataset) {
         checkDataset();
-        gspRequest().dataset().postDataset(dataset);
+        dspRequest().POST(dataset);
     }
 
     @Override
     public void putDataset(String file) {
         checkDataset();
-        gspRequest().dataset().putDataset(file);
+        dspRequest().PUT(file);
     }
 
     @Override
     public void putDataset(DatasetGraph dataset) {
         checkDataset();
-        gspRequest().dataset().putDataset(dataset);
+        dspRequest().PUT(dataset);
     }
 
     // -- Internal.