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 2018/02/27 23:10:21 UTC

[01/24] jena git commit: JENA-1490: Handle remote blank nodes if remote is Fuseki

Repository: jena
Updated Branches:
  refs/heads/master f5d3e3868 -> b75fccefa


JENA-1490: Handle remote blank nodes if remote is Fuseki

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/e4b4f99f
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/e4b4f99f
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/e4b4f99f

Branch: refs/heads/master
Commit: e4b4f99fe28c68315e08d481e9099c992dd7e0ed
Parents: 89e6a8d
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 13:53:28 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 13:53:28 2018 +0000

----------------------------------------------------------------------
 .../apache/jena/rdfconnection/LibRDFConn.java   |  72 +++
 .../org/apache/jena/rdfconnection/RDFConn.java  |  71 ---
 .../jena/rdfconnection/RDFConnection.java       |  12 +-
 .../jena/rdfconnection/RDFConnectionFuseki.java | 204 +++++++
 .../jena/rdfconnection/RDFConnectionLocal.java  |   6 +-
 .../jena/rdfconnection/RDFConnectionRemote.java |  14 +-
 .../rdfconnection/RDFConnectionRemote2.java     | 542 +++++++++++++++++++
 .../RDFConnectionRemoteBuilder.java             | 310 +++++++++++
 .../apache/jena/rdfconnection/TestRDFConn.java  |   4 +-
 9 files changed, 1150 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/LibRDFConn.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/LibRDFConn.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/LibRDFConn.java
new file mode 100644
index 0000000..69d5b79
--- /dev/null
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/LibRDFConn.java
@@ -0,0 +1,72 @@
+/*
+ * 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.rdfconnection;
+
+/** package-wide utilities etc */
+/*package*/ class LibRDFConn {
+    private static String dftName =  "default" ;
+    
+    /*package*/ static boolean isDefault(String name) {
+        return name == null || name.equals(dftName) ;
+    }
+    
+    private static String queryStringForGraph(String ch, String graphName) {
+        return 
+            ch + 
+                (LibRDFConn.isDefault(graphName)
+                ? "default"
+                : "graph="+graphName) ;
+    }
+    
+    /*package*/ static String urlForGraph(String graphStoreProtocolService, String graphName) {
+        // If query string
+        String ch = "?";
+        if ( graphStoreProtocolService.contains("?") )
+            // Already has a query string, append with "&"  
+            ch = "&";
+        return graphStoreProtocolService + queryStringForGraph(ch, graphName) ;
+    }
+
+    /*package*/ static String formServiceURL(String destination, String srvEndpoint) {
+        if ( srvEndpoint == null )
+            return null;
+        if ( srvEndpoint == RDFConnectionRemoteBuilder.SameAsDestination )
+            return destination;
+        if ( destination == null )
+            return srvEndpoint;
+
+        // If the srvEndpoint looks like an absolute URL, use as given. 
+        if ( srvEndpoint.startsWith("http:/") || srvEndpoint.startsWith("https:/") )
+            return srvEndpoint;
+        String queryString = null;
+        String dest = destination;
+        if ( destination.contains("?") ) {
+            // query string : remove and append later.
+            int i = destination.indexOf('?');
+            queryString = destination.substring(i);
+            dest = destination.substring(0, i);
+        }
+        if ( dest.endsWith("/") )
+            dest = dest.substring(0, dest.length()-1);
+        dest = dest+"/"+srvEndpoint;
+        if ( queryString != null )
+           dest = dest+queryString; 
+        return dest;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConn.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConn.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConn.java
deleted file mode 100644
index 63edd75..0000000
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConn.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.rdfconnection;
-
-import java.util.Objects;
-
-/** package-wide utilities etc */
-/*package*/ class RDFConn {
-    private static String dftName =  "default" ;
-    
-    /*package*/ static boolean isDefault(String name) {
-        return name == null || name.equals(dftName) ;
-    }
-    
-    private static String queryStringForGraph(String ch, String graphName) {
-        return 
-            ch + 
-                (RDFConn.isDefault(graphName)
-                ? "default"
-                : "graph="+graphName) ;
-    }
-    
-    /*package*/ static String urlForGraph(String graphStoreProtocolService, String graphName) {
-        // If query string
-        String ch = "?";
-        if ( graphStoreProtocolService.contains("?") )
-            // Already has a query string, append with "&"  
-            ch = "&";
-        return graphStoreProtocolService + queryStringForGraph(ch, graphName) ;
-    }
-
-    /*package*/ static String formServiceURL(String destination, String srvEndpoint) {
-        Objects.requireNonNull(srvEndpoint, "Service Endpoint");
-        if ( destination == null )
-            return srvEndpoint;
-        // If the srvEndpoint looks like an absolute URL, use as given. 
-        if ( srvEndpoint.startsWith("http:/") || srvEndpoint.startsWith("https:/") )
-            return srvEndpoint;
-        String queryString = null;
-        String dest = destination;
-        if ( destination.contains("?") ) {
-            // query string : remove and append later.
-            int i = destination.indexOf('?');
-            queryString = destination.substring(i);
-            dest = destination.substring(0, i);
-        }
-        if ( dest.endsWith("/") )
-            dest = dest.substring(0, dest.length()-1);
-        dest = dest+"/"+srvEndpoint;
-        if ( queryString != null )
-           dest = dest+queryString; 
-        return dest;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnection.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnection.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnection.java
index a1109d3..3c5e374 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnection.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnection.java
@@ -82,6 +82,7 @@ public interface RDFConnection extends
      */
     @Override
     public default void queryResultSet(String query, Consumer<ResultSet> resultSetAction) {
+        // XXX Parse point.
         queryResultSet(QueryFactory.create(query), resultSetAction);
     }
     
@@ -94,7 +95,6 @@ public interface RDFConnection extends
     public default void queryResultSet(Query query, Consumer<ResultSet> resultSetAction) {
         if ( ! query.isSelectType() )
             throw new JenaConnectionException("Query is not a SELECT query");
-
         Txn.executeRead(this, ()->{ 
             try ( QueryExecution qExec = query(query) ) {
                 ResultSet rs = qExec.execSelect();
@@ -110,7 +110,13 @@ public interface RDFConnection extends
      */
     @Override
     public default void querySelect(String query, Consumer<QuerySolution> rowAction) {
-        querySelect(QueryFactory.create(query), rowAction);
+        //Parse local: querySelect(QueryFactory.create(query), rowAction);
+        // XXX Parse point.
+        Txn.executeRead(this, ()->{ 
+            try ( QueryExecution qExec = query(query) ) {
+                qExec.execSelect().forEachRemaining(rowAction);
+            }
+        } ); 
     }
     
     /**
@@ -132,6 +138,7 @@ public interface RDFConnection extends
     /** Execute a CONSTRUCT query and return as a Model */
     @Override
     public default Model queryConstruct(String query) {
+        // XXX Parse point.
         return queryConstruct(QueryFactory.create(query));
     }
     
@@ -149,6 +156,7 @@ public interface RDFConnection extends
     /** Execute a DESCRIBE query and return as a Model */
     @Override
     public default Model queryDescribe(String query) {
+        // XXX Parse point.
         return queryDescribe(QueryFactory.create(query));
     }
     

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
new file mode 100644
index 0000000..5468dc7
--- /dev/null
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
@@ -0,0 +1,204 @@
+/*
+ * 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.rdfconnection;
+
+import java.util.stream.Stream;
+
+import org.apache.http.client.HttpClient;
+import org.apache.http.protocol.HttpContext;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.query.Query;
+import org.apache.jena.query.QueryExecution;
+import org.apache.jena.query.QueryFactory;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdfconnection.RDFConnection;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFFormat;
+import org.apache.jena.riot.WebContent;
+import org.apache.jena.riot.resultset.ResultSetLang;
+import org.apache.jena.riot.web.HttpOp;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Quad;
+import org.apache.jena.sparql.core.Transactional;
+import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
+import org.apache.jena.update.UpdateFactory;
+import org.apache.jena.update.UpdateRequest;
+
+/** 
+ * Implementation of the {@link RDFConnection} interface for connecting to an Apache Jena Fuseki.
+ * <p>
+ * This adds the ability to work with blank nodes across the network.
+ */
+public class RDFConnectionFuseki extends RDFConnectionRemote2 {
+
+    /**
+     * Create a connection builder which is initialized for the default Fuseki
+     * configuration. The application must call
+     * {@link RDFConnectionRemoteBuilder#destination(String)} to set the URL of the remote
+     * dataset.
+     * @return RDFConnectionRemoteBuilder
+     */
+    public static RDFConnectionRemoteBuilder create() {
+        return setupForFuseki(RDFConnectionRemote2.create());
+    }
+
+    /** 
+     * Create a connection builder which is initialized from an existing {@code RDFConnectionFuseki}.
+     * @param other The RDFConnectionFuseki to clone.
+     * @return RDFConnectionRemoteBuilder
+     */
+    public static RDFConnectionRemoteBuilder create(RDFConnectionFuseki other) {
+        return setupCreator(RDFConnectionRemote2.create(other));
+    }
+    
+    /** Fuseki settings */
+    private static RDFConnectionRemoteBuilder setupForFuseki(RDFConnectionRemoteBuilder builder) {
+        String ctRDFThrift = Lang.RDFTHRIFT.getContentType().getContentType();
+        return 
+            builder
+                .quadsFormat(RDFFormat.RDF_THRIFT)
+                .triplesFormat(RDFFormat.RDF_THRIFT)
+                .acceptHeaderGraph(ctRDFThrift)
+                .acceptHeaderDataset(ctRDFThrift)
+                .acceptHeaderSelectQuery(ResultSetLang.SPARQLResultSetThrift.getHeaderString())
+                .acceptHeaderAskQuery(ResultSetLang.SPARQLResultSetJSON.getHeaderString())
+                .acceptHeaderGraphQuery(ResultSetLang.SPARQLResultSetThrift.getHeaderString())
+                // Create object of this class.
+                .creator((b)->fusekiMaker(b));
+    }
+    
+    private static RDFConnectionRemoteBuilder setupCreator(RDFConnectionRemoteBuilder builder) {
+        return builder.creator((b)->fusekiMaker(b));
+    }
+    
+    static RDFConnectionFuseki fusekiMaker(RDFConnectionRemoteBuilder builder) {
+        return new RDFConnectionFuseki(builder);
+    }
+
+    protected RDFConnectionFuseki(RDFConnectionRemoteBuilder base) {
+        this(base.txnLifecycle, base.httpClient, base.httpContext, 
+            base.destination, base.queryURL, base.updateURL, base.gspURL,
+            base.outputQuads, base.outputTriples,
+            base.acceptDataset, base.acceptGraph,
+            base.acceptSelectResult, base.acceptAskResult, base.acceptGraphResult);
+    }
+    
+    protected RDFConnectionFuseki(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
+                                  String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
+                                  String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
+                                  String acceptGraphResult) {
+        super(txnLifecycle, httpClient, httpContext, 
+              destination, queryURL, updateURL, gspURL,
+              outputQuads, outputTriples, 
+              acceptDataset, acceptGraph,
+              acceptSelectResult, acceptAskResult, acceptGraphResult);
+    }
+    
+    // Fuseki specific operations.
+    
+    @Override
+    public void update(String updateString) {
+        checkUpdate();
+        if ( true ) {
+            // XXX Parse local, use original string.
+            UpdateRequest req = UpdateFactory.create(updateString);
+        }
+        exec(()->HttpOp.execHttpPost(svcUpdate, WebContent.contentTypeSPARQLUpdate, updateString, this.httpClient, this.httpContext));
+//        update(UpdateFactory.create(updateString));
+    }
+    
+//    @Override
+//    public void querySelect(String query, Consumer<QuerySolution> rowAction) {
+//        try ( QueryExecution qExec = query(query) ) {
+//            qExec.execSelect().forEachRemaining(rowAction);
+//        }
+//    }
+    
+    // Make sure all querygoes through query(String) or query(Query) 
+    
+    @Override
+    public QueryExecution query(String queryString) {
+        checkQuery();
+        
+        Query queryLocal = QueryFactory.create(queryString);
+        // XXX Kludge until QueryEngineHTTP.setAccept.
+        // XXX Accept header builder.
+        String acceptHeader = acceptSelectResult+","+acceptAskResult+";q=0.9,"+acceptGraphResult;
+        return exec(()-> {
+            QueryExecution qExec = new QueryEngineHTTP(svcQuery, queryString, httpClient, httpContext);
+            QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
+            // XXX qEngine.setAccept(acceptHeader);
+            // Only one choice, not "Accept:"
+            switch ( queryLocal.getQueryType() ) {
+                case Query.QueryTypeSelect:
+                    qEngine.setSelectContentType(acceptSelectResult);
+                    break;
+                case Query.QueryTypeAsk:
+                    qEngine.setAskContentType(acceptAskResult);
+                    break;
+                case Query.QueryTypeDescribe:
+                case Query.QueryTypeConstruct:
+                    qEngine.setModelContentType(acceptGraphResult);
+                    break;
+            }
+            return qEngine ;
+        });
+//        // XXX Better!
+//        String url = svcQuery+"?query="+queryString;
+//        // XXX Better accept.
+//        TypedInputStream in =  exec(()->HttpOp.execHttpGet(url, acceptSelectResult, this.httpClient,this.httpContext));
+//        QueryExecution qExec = 
+//        return qExec;
+    }
+
+    /**
+     * Return a {@link Model} that is proxy for a remote model in a Fuseki server. This
+     * support the model operations of accessing statements and changing the model.
+     * <p>
+     * This provide low level access to the remote data. The application will be working
+     * with and manipulating the remote model directly which may involve a significant
+     * overhead for every {@code Model} API operation.
+     * <p>
+     * <b><em>Warning</em>:</b> This is <b>not</b> performant for bulk changes. 
+     * <p>
+     * Getting the model, using {@link #fetch()}, which copies the whole model into a local
+     * {@code Model} object, maniupulating it and putting it back with {@link #put(Model)}
+     * provides another way to work with remote data.
+     * 
+     * @return Model
+     */
+    public Model getModelProxy() { return null; }
+    public Model getModelProxy(String graphName) { return null; }
+    
+    public Graph getGraphProxy() { return null; }
+    public Graph getGraphProxy(String graphName) { return null; }
+
+    public Dataset getDatasetProxy() { return null; }
+    public DatasetGraph getDatasetGraphProxy() { return null; }
+
+    // Or remote RDFStorage?
+    public Stream<Triple> findStream(Node s, Node p , Node o) { return null; }
+    public Stream<Quad> findStream(Node g, Node s, Node p , Node o) { return null; }
+
+    // Send Patch 
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionLocal.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionLocal.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionLocal.java
index 362a4be..2db693e 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionLocal.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionLocal.java
@@ -156,7 +156,7 @@ public class RDFConnectionLocal implements RDFConnection {
     public void delete(String graph) {
         checkOpen();
         Txn.executeWrite(dataset,() ->{
-            if ( RDFConn.isDefault(graph) ) 
+            if ( LibRDFConn.isDefault(graph) ) 
                 dataset.getDefaultModel().removeAll();
             else 
                 dataset.removeNamedModel(graph);
@@ -175,7 +175,7 @@ public class RDFConnectionLocal implements RDFConnection {
         
         Txn.executeWrite(dataset,() ->{
             if ( RDFLanguages.isTriples(lang) ) {
-                Model model = RDFConn.isDefault(graph) ? dataset.getDefaultModel() : dataset.getNamedModel(graph);
+                Model model = LibRDFConn.isDefault(graph) ? dataset.getDefaultModel() : dataset.getNamedModel(graph);
                 if ( replace )
                     model.removeAll();
                 RDFDataMgr.read(model, file); 
@@ -235,7 +235,7 @@ public class RDFConnectionLocal implements RDFConnection {
     }
 
     private Model modelFor(String graph) {
-        if ( RDFConn.isDefault(graph)) 
+        if ( LibRDFConn.isDefault(graph)) 
             return dataset.getDefaultModel();
         return dataset.getNamedModel(graph);
     }

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
index 371945e..34f903e 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
@@ -93,9 +93,9 @@ public class RDFConnectionRemote implements RDFConnection {
     /** Create connection, using URL of the dataset and short names for the services */
     public RDFConnectionRemote(HttpClient httpClient, String destination, String sQuery, String sUpdate, String sGSP) {
         this.destination = destination;
-        this.svcQuery = RDFConn.formServiceURL(destination, sQuery);
-        this.svcUpdate = RDFConn.formServiceURL(destination, sUpdate);
-        this.svcGraphStore = RDFConn.formServiceURL(destination, sGSP);
+        this.svcQuery = LibRDFConn.formServiceURL(destination, sQuery);
+        this.svcUpdate = LibRDFConn.formServiceURL(destination, sUpdate);
+        this.svcGraphStore = LibRDFConn.formServiceURL(destination, sGSP);
         this.httpClient = httpClient;
     }
     
@@ -131,7 +131,7 @@ public class RDFConnectionRemote implements RDFConnection {
     @Override
     public Model fetch(String graphName) {
         checkGSP();
-        String url = RDFConn.urlForGraph(svcGraphStore, graphName);
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graphName);
         Graph graph = fetch$(url);
         return ModelFactory.createModelForGraph(graph);
     }
@@ -207,7 +207,7 @@ public class RDFConnectionRemote implements RDFConnection {
             throw new ARQException("Can't load quads into a graph");
         if ( ! RDFLanguages.isTriples(lang) )
             throw new ARQException("Not an RDF format: "+file+" (lang="+lang+")");
-        String url = RDFConn.urlForGraph(svcGraphStore, graph);
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
         doPutPost(url, file, lang, replace);
     }
 
@@ -237,7 +237,7 @@ public class RDFConnectionRemote implements RDFConnection {
      * "Replace" means overwrite existing data, othewise the date is added to the target.
      */
     protected void doPutPost(Model model, String name, boolean replace) {
-        String url = RDFConn.urlForGraph(svcGraphStore, name);
+        String url = LibRDFConn.urlForGraph(svcGraphStore, name);
         exec(()->{
             Graph graph = model.getGraph();
             if ( replace )
@@ -250,7 +250,7 @@ public class RDFConnectionRemote implements RDFConnection {
     @Override
     public void delete(String graph) {
         checkGSP();
-        String url = RDFConn.urlForGraph(svcGraphStore, graph);
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
         exec(()->HttpOp.execHttpDelete(url));
     }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
new file mode 100644
index 0000000..3440f18
--- /dev/null
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
@@ -0,0 +1,542 @@
+/*
+ * 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.rdfconnection;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.File;
+import java.io.InputStream;
+import java.util.function.Supplier;
+
+import org.apache.http.HttpEntity;
+import org.apache.http.client.HttpClient;
+import org.apache.http.entity.EntityTemplate;
+import org.apache.http.protocol.HttpContext;
+import org.apache.jena.atlas.io.IO;
+import org.apache.jena.atlas.web.HttpException;
+import org.apache.jena.atlas.web.TypedInputStream;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.query.*;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.rdfconnection.RDFConnection;
+import org.apache.jena.rdfconnection.RDFConnectionFactory;
+import org.apache.jena.riot.*;
+import org.apache.jena.riot.web.HttpCaptureResponse;
+import org.apache.jena.riot.web.HttpOp;
+import org.apache.jena.riot.web.HttpResponseLib;
+import org.apache.jena.sparql.ARQException;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.Transactional;
+import org.apache.jena.sparql.core.TransactionalLock;
+import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
+import org.apache.jena.system.Txn;
+import org.apache.jena.update.UpdateExecutionFactory;
+import org.apache.jena.update.UpdateProcessor;
+import org.apache.jena.update.UpdateRequest;
+import org.apache.jena.web.HttpSC;
+
+/** 
+ * Implementation of the {@link RDFConnection} interface using remote SPARQL operations.  
+ */
+public class RDFConnectionRemote2 implements RDFConnection {
+    // Adds a Builder to help with HTTP details.
+    
+    private static final String fusekiDftSrvQuery   = "sparql";
+    private static final String fusekiDftSrvUpdate  = "update";
+    private static final String fusekiDftSrvGSP     = "data";
+    
+    private boolean isOpen = true; 
+    protected final String destination;
+    protected final String svcQuery;
+    protected final String svcUpdate;
+    protected final String svcGraphStore;
+    
+    protected final Transactional txnLifecycle;
+    protected final HttpClient httpClient;
+    protected final HttpContext httpContext;
+    
+    // On-the-wire settings.
+    protected final RDFFormat outputQuads; 
+    protected final RDFFormat outputTriples;
+    protected final String acceptGraph;
+    protected final String acceptDataset;
+    protected final String acceptSelectResult;
+    protected final String acceptAskResult;
+    protected final String acceptGraphResult;
+    
+    /** Create a {@link RDFConnectionRemoteBuilder}. */
+    public static RDFConnectionRemoteBuilder create() {
+        return new RDFConnectionRemoteBuilder();
+    }
+
+    /** 
+     * Create a {@link RDFConnectionRemoteBuilder} initialized with the
+     * settings of another {@code RDFConnectionRemote}.  
+     */
+    public static RDFConnectionRemoteBuilder create(RDFConnectionRemote2 base) {
+        return new RDFConnectionRemoteBuilder(base);
+    }
+    
+    /**
+     * Create connection that will use the {@link HttpClient} using URL of the dataset and
+     * default service names
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder}.
+     */
+    @Deprecated 
+    public RDFConnectionRemote2(HttpClient httpClient, String destination) {
+        this(httpClient,
+             requireNonNull(destination),
+             fusekiDftSrvQuery, 
+             fusekiDftSrvUpdate,
+             fusekiDftSrvGSP);
+    }
+
+    /**
+     * Create connection, using URL of the dataset and default service names
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated 
+    public RDFConnectionRemote2(String destination) {
+        this(requireNonNull(destination),
+             fusekiDftSrvQuery, 
+             fusekiDftSrvUpdate,
+             fusekiDftSrvGSP);
+    }
+
+    /**
+     * Create connection, using full URLs for services. Pass a null for "no service
+     * endpoint".
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated 
+    public RDFConnectionRemote2(String sQuery, String sUpdate, String sGSP) {
+        this(null, sQuery, sUpdate, sGSP);
+    }
+    
+    /**
+     * Create connection, using URL of the dataset and names for the services. Short names
+     * are expanded against the destination. Absolute URIs are left unchanged.
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated
+    public RDFConnectionRemote2(String destination, String sQuery, String sUpdate, String sGSP) {
+        this(null, destination, sQuery, sUpdate, sGSP);
+    }
+    
+    /**
+     * Create connection, using URL of the dataset and names for the services. Short names
+     * are expanded against the destination. Absolute URIs are left unchanged.
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated
+    public RDFConnectionRemote2(HttpClient httpClient, String destination, String sQuery, String sUpdate, String sGSP) {
+        this(null, httpClient, null, destination, sQuery, sUpdate, sGSP,
+            RDFFormat.NQUADS, RDFFormat.NTRIPLES,
+            WebContent.defaultGraphAcceptHeader, WebContent.defaultDatasetAcceptHeader,
+            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader(), QueryEngineHTTP.defaultConstructHeader()
+            );
+    }
+
+    // Used by the builder.
+    protected RDFConnectionRemote2(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
+                                   String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
+                                   String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
+                                   String acceptGraphResult) {
+        this.httpClient = httpClient;
+        this.httpContext = httpContext;
+        this.destination = destination;
+        this.svcQuery = queryURL;
+        this.svcUpdate = updateURL;
+        this.svcGraphStore = gspURL;
+        if ( txnLifecycle == null )
+            txnLifecycle  = TransactionalLock.createMRPlusSW();
+        this.txnLifecycle = txnLifecycle;
+        this.outputQuads = outputQuads;
+        this.outputTriples = outputTriples;
+        this.acceptDataset = acceptDataset;
+        this.acceptGraph = acceptGraph;
+        this.acceptSelectResult = acceptSelectResult;
+        this.acceptAskResult = acceptAskResult;
+        this.acceptGraphResult = acceptGraphResult;
+    }
+
+    /** Return the {@link HttpClient} in-use. */ 
+    public HttpClient getHttpClient() {
+        return httpClient;
+    }
+
+    /** Return the {@link HttpContext} in-use. */ 
+    public HttpContext getHttpContext() {
+        return httpContext;
+    }
+    
+    /** Return the destination URL for the connection. */
+    public String getDestination() {
+        return destination;
+    }
+
+    @Override
+    public QueryExecution query(Query query) {
+        checkQuery();
+        // XXX ResultSetFormat
+        
+        
+        
+        return exec(()-> {
+            QueryExecution qExec = QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext);
+            QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
+            
+            // Set general HTTP header.
+            
+//            // Only one choice, not "Accept:"
+//            switch ( query.getQueryType() ) {
+//                case Query.QueryTypeSelect:
+//                    qEngine.setSelectContentType("");
+//                    break;
+//                case Query.QueryTypeAsk:
+//                    qEngine.setAskContentType("");
+//                    break;
+//                case Query.QueryTypeDescribe:
+//                case Query.QueryTypeConstruct:
+//                    qEngine.setModelContentType("");
+//                    break;
+//            }
+            return qExec ;
+        });
+        
+    }
+
+    @Override
+    public void update(UpdateRequest update) {
+        checkUpdate();
+        UpdateProcessor proc = UpdateExecutionFactory.createRemote(update, svcUpdate, this.httpClient, this.httpContext);
+        exec(()->proc.execute());
+    }
+    
+    @Override
+    public Model fetch(String graphName) {
+        checkGSP();
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graphName);
+        Graph graph = fetch$(url);
+        return ModelFactory.createModelForGraph(graph);
+    }
+    
+    @Override
+    public Model fetch() {
+        checkGSP();
+        return fetch(null);
+    }
+    
+    private Graph fetch$(String url) {
+        HttpCaptureResponse<Graph> graph = HttpResponseLib.graphHandler();
+        exec(()->HttpOp.execHttpGet(url, acceptGraph, graph, this.httpClient, this.httpContext));
+        return graph.get();
+    }
+
+    @Override
+    public void load(String graph, String file) {
+        checkGSP();
+        upload(graph, file, false);
+    }
+    
+    @Override
+    public void load(String file) {
+        checkGSP();
+        upload(null, file, false);
+    }
+    
+    @Override
+    public void load(Model model) {
+        doPutPost(model, null, false);
+    }
+    
+    @Override
+    public void load(String graphName, Model model) {
+        doPutPost(model, graphName, false);
+    }
+    
+    @Override
+    public void put(String graph, String file) {
+        checkGSP();
+        upload(graph, file, true);
+    }
+    
+    @Override
+    public void put(String file) { 
+        checkGSP();
+        upload(null, file, true); 
+    }
+    
+    @Override
+    public void put(String graphName, Model model) {
+        checkGSP();
+        doPutPost(model, graphName, true);
+    }
+
+    @Override
+    public void put(Model model) {
+        checkGSP();
+        doPutPost(model, null, true);
+    }
+    
+    /** Send a file to named graph (or "default" or null for the default graph).
+     * <p>
+     * The Content-Type is inferred from the file extension.
+     * <p>
+     * "Replace" means overwrite existing data, othewise the date is added to the target.
+     */
+    protected void upload(String graph, String file, boolean replace) {
+        // if triples
+        Lang lang = RDFLanguages.filenameToLang(file);
+        if ( RDFLanguages.isQuads(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+")");
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
+        doPutPost(url, file, lang, replace);
+    }
+
+    /** Send a file to named graph (or "default" or null for the defaultl graph).
+     * <p>
+     * The Content-Type is taken from the given {@code Lang}.
+     * <p>
+     * "Replace" means overwrite existing data, othewise the date is added to the target.
+     */
+    protected void doPutPost(String url, String file, Lang lang, boolean replace) {
+        File f = new File(file);
+        long length = f.length(); 
+        InputStream source = IO.openFile(file);
+        // Charset.
+        exec(()->{
+            if ( replace )
+                HttpOp.execHttpPut(url, lang.getContentType().getContentType(), source, length, httpClient, this.httpContext);
+            else    
+                HttpOp.execHttpPost(url, lang.getContentType().getContentType(), source, length, null, null, httpClient, this.httpContext);
+        });
+    }
+
+    /** Send a model to named graph (or "default" or null for the defaultl graph).
+     * <p>
+     * The Content-Type is taken from the given {@code Lang}.
+     * <p>
+     * "Replace" means overwrite existing data, othewise the date is added to the target.
+     */
+    protected void doPutPost(Model model, String name, boolean replace) {
+        String url = LibRDFConn.urlForGraph(svcGraphStore, name);
+        exec(()->{
+            Graph graph = model.getGraph();
+            if ( replace )
+                HttpOp.execHttpPut(url, graphToHttpEntity(graph), httpClient, this.httpContext);
+            else    
+                HttpOp.execHttpPost(url, graphToHttpEntity(graph), null, null, httpClient, this.httpContext);
+        });
+    }
+
+    @Override
+    public void delete(String graph) {
+        checkGSP();
+        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
+        exec(()->HttpOp.execHttpDelete(url));
+    }
+
+    @Override
+    public void delete() {
+        checkGSP();
+        delete(null);
+    }
+
+    @Override
+    public Dataset fetchDataset() {
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URL provided"); 
+        Dataset ds = DatasetFactory.createTxnMem();
+        Txn.executeWrite(ds, ()->{
+            TypedInputStream s = exec(()->HttpOp.execHttpGet(destination, acceptDataset));
+            Lang lang = RDFLanguages.contentTypeToLang(s.getContentType());
+            RDFDataMgr.read(ds, s, lang);
+        });
+        return ds;
+    }
+
+    @Override
+    public void loadDataset(String file) { 
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
+        doPutPostDataset(file, false); 
+    }
+    
+    @Override
+    public void loadDataset(Dataset dataset) {
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
+        doPutPostDataset(dataset, false); 
+    }
+
+    @Override
+    public void putDataset(String file) {
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
+        doPutPostDataset(file, true);
+    }
+    
+    @Override
+    public void putDataset(Dataset dataset) {
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
+        doPutPostDataset(dataset, true); 
+    }
+
+    /** Do a PUT or POST to a dataset, sending the contents of the file.
+     * <p>
+     * The Content-Type is inferred from the file extension.
+     * <p>
+     * "Replace" implies PUT, otherwise a POST is used.
+     */
+    protected void doPutPostDataset(String file, boolean replace) {
+        Lang lang = RDFLanguages.filenameToLang(file);
+        File f = new File(file);
+        long length = f.length();
+        exec(()->{
+            InputStream source = IO.openFile(file);
+            if ( replace )
+                HttpOp.execHttpPut(destination, lang.getContentType().getContentType(), source, length, httpClient, httpContext);
+            else    
+                HttpOp.execHttpPost(destination, lang.getContentType().getContentType(), source, length, null, null, httpClient, httpContext);
+        });
+    }
+
+    /** Do a PUT or POST to a dataset, sending the contents of a daatsets.
+     * The Content-Type is {@code application/n-quads}.
+     * <p>
+     * "Replace" implies PUT, otherwise a POST is used.
+     */
+    protected void doPutPostDataset(Dataset dataset, boolean replace) {
+        exec(()->{
+            DatasetGraph dsg = dataset.asDatasetGraph();
+            if ( replace )
+                HttpOp.execHttpPut(destination, datasetToHttpEntity(dsg), httpClient, null);
+            else    
+                HttpOp.execHttpPost(destination, datasetToHttpEntity(dsg), httpClient, null);
+        });
+    }
+
+    protected void checkQuery() {
+        checkOpen();
+        if ( svcQuery == null )
+            throw new ARQException("No query service defined for this RDFConnection");
+    }
+    
+    protected void checkUpdate() {
+        checkOpen();
+        if ( svcUpdate == null )
+            throw new ARQException("No update service defined for this RDFConnection");
+    }
+    
+    protected void checkGSP() {
+        checkOpen();
+        if ( svcGraphStore == null )
+            throw new ARQException("No SPARQL Graph Store service defined for this RDFConnection");
+    }
+    
+    protected void checkDataset() {
+        checkOpen();
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URL provided"); 
+    }
+
+    protected void checkOpen() {
+        if ( ! isOpen )
+            throw new ARQException("closed");
+    }
+
+    @Override
+    public void close() {
+        isOpen = false;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return ! isOpen;
+    }
+
+    /** Create an HttpEntity for the graph */  
+    protected HttpEntity graphToHttpEntity(Graph graph) {
+        return graphToHttpEntity(graph, outputTriples);
+    }
+    
+    /** Create an HttpEntity for the graph */
+    protected HttpEntity graphToHttpEntity(Graph graph, RDFFormat syntax) {
+        EntityTemplate entity = new EntityTemplate((out)->RDFDataMgr.write(out, graph, syntax));
+        String ct = syntax.getLang().getContentType().getContentType();
+        entity.setContentType(ct);
+        return entity;
+    }
+
+    /** Create an HttpEntity for the dataset */  
+    protected HttpEntity datasetToHttpEntity(DatasetGraph dataset) {
+        return datasetToHttpEntity(dataset, outputQuads);
+    }
+    
+    /** Create an HttpEntity for the dataset */  
+    protected HttpEntity datasetToHttpEntity(DatasetGraph dataset, RDFFormat syntax) {
+        EntityTemplate entity = new EntityTemplate((out)->RDFDataMgr.write(out, dataset, syntax));
+        String ct = syntax.getLang().getContentType().getContentType();
+        entity.setContentType(ct);
+        return entity;
+    }
+
+    /** Convert HTTP status codes to exceptions */ 
+    static protected void exec(Runnable action)  {
+        try { action.run(); }
+        catch (HttpException ex) { handleHttpException(ex, false); }
+    }
+
+    /** Convert HTTP status codes to exceptions */ 
+    static protected <X> X exec(Supplier<X> action)  {
+        try { return action.get(); }
+        catch (HttpException ex) { handleHttpException(ex, true); return null;}
+    }
+
+    private static void handleHttpException(HttpException ex, boolean ignore404) {
+        if ( ex.getResponseCode() == HttpSC.NOT_FOUND_404 && ignore404 )
+            return ;
+        throw ex;
+    }
+
+    @Override public void begin()                       { txnLifecycle.begin(); }
+    @Override public void begin(TxnType txnType)        { txnLifecycle.begin(txnType); }
+    @Override public void begin(ReadWrite mode)         { txnLifecycle.begin(mode); }
+    @Override public boolean promote(Promote promote)   { return txnLifecycle.promote(promote); }
+    @Override public void commit()                      { txnLifecycle.commit(); }
+    @Override public void abort()                       { txnLifecycle.abort(); }
+    @Override public boolean isInTransaction()          { return txnLifecycle.isInTransaction(); }
+    @Override public void end()                         { txnLifecycle.end(); }
+    @Override public ReadWrite transactionMode()        { return txnLifecycle.transactionMode(); }
+    @Override public TxnType transactionType()          { return txnLifecycle.transactionType(); }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
new file mode 100644
index 0000000..d47f36a
--- /dev/null
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
@@ -0,0 +1,310 @@
+/*
+ * 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.rdfconnection;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.Objects;
+import java.util.function.Function;
+
+import org.apache.http.client.HttpClient;
+import org.apache.http.protocol.HttpContext;
+import org.apache.jena.rdfconnection.RDFConnection;
+import org.apache.jena.riot.*;
+import org.apache.jena.sparql.core.Transactional;
+import org.apache.jena.sparql.core.TransactionalLock;
+import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
+
+public class RDFConnectionRemoteBuilder {
+    /*package*/ static String SameAsDestination  = "";
+
+    protected Transactional txnLifecycle  = TransactionalLock.createMRPlusSW();
+    protected HttpClient    httpClient    = null;
+    protected HttpContext   httpContext   = null;
+    protected String        destination   = null;
+    
+    protected String        sQuery        = SameAsDestination;
+    protected String        sUpdate       = SameAsDestination;
+    protected String        sGSP          = SameAsDestination;
+
+    protected String        queryURL      = null;
+    protected String        updateURL     = null;
+    protected String        gspURL        = null;
+
+    // On-the-wire settings.
+    protected RDFFormat     outputQuads        = RDFFormat.NQUADS;
+    protected RDFFormat     outputTriples      = RDFFormat.NTRIPLES;
+    
+    protected String        acceptGraph        = WebContent.defaultGraphAcceptHeader;
+    protected String        acceptDataset      = WebContent.defaultDatasetAcceptHeader;
+    
+    protected String        acceptSelectResult = QueryEngineHTTP.defaultSelectHeader();
+    protected String        acceptAskResult    = QueryEngineHTTP.defaultAskHeader();
+    protected String        acceptGraphResult  = QueryEngineHTTP.defaultConstructHeader();
+
+    RDFConnectionRemoteBuilder() { 
+        // Default settings are the meber declarations.
+    }
+    
+    RDFConnectionRemoteBuilder(RDFConnectionRemote2 base) {
+        Objects.requireNonNull(base);
+        txnLifecycle = base.txnLifecycle;
+        if ( txnLifecycle == null )
+            txnLifecycle = TransactionalLock.createMRPlusSW();
+        httpClient          = base.httpClient;
+        httpContext         = base.httpContext;
+        destination         = base.destination;
+        sQuery              = base.svcQuery;
+        sUpdate             = base.svcUpdate;
+        sGSP                = base.svcGraphStore;
+        outputQuads         = base.outputQuads;
+        outputTriples       = base.outputTriples;
+        
+        acceptGraph         = base.acceptGraph;
+        acceptDataset       = base.acceptDataset;
+        
+        acceptSelectResult  = base.acceptSelectResult;
+        acceptAskResult     = base.acceptAskResult;
+        acceptGraphResult   = base.acceptGraphResult;
+    }
+    
+    /** URL of the remote SPARQL endpoint.
+     * For Fuseki, this is the URL of the dataset  e.g. http:/localhost:3030/dataset
+     */
+    public RDFConnectionRemoteBuilder destination(String destination) {
+        Objects.requireNonNull(destination);
+        this.destination = destination;
+        return this;
+    }
+    
+    /** Name of the SPARQL query service.
+     * <p>
+     * This can be a short name, relative to the destination URL,
+     * or a full URL (with "http:" or "https:")
+     * <p>
+     * Use {@code ""} for "same as destination".
+     * <br/>
+     * Use null for "none". 
+     */
+    public RDFConnectionRemoteBuilder queryEndpoint(String sQuery) {
+        this.sQuery = sQuery;
+        return this;
+    }
+    
+    /** Name of the SPARQL update service.
+     * <p>
+     * This can be a short name, relative to the destination URL,
+     * or a full URL (with "http:" or "https:")
+     * <p>
+     * Use {@code ""} for "same as destination".
+     * <br/>
+     * Use null for "none". 
+     */
+    public RDFConnectionRemoteBuilder updateEndpoint(String sUpdate) {
+        this.sUpdate = sUpdate;
+        return this;
+    }
+
+    /** Name of the SPARQL GraphStore Protocol endpoint.
+     * <p>
+     * This can be a short name, relative to the destination URL,
+     * or a full URL (with "http:" or "https:")
+     * <p>
+     * Use {@code ""} for "same as destination".
+     * <br/>
+     * Use null for "none". 
+     */
+    public RDFConnectionRemoteBuilder gspEndpoint(String sGSP) {
+        this.sGSP = sGSP;
+        return this;
+    }
+    
+    /** Set the transaction lifecycle. */
+    /*Future possibility*/
+    private RDFConnectionRemoteBuilder txnLifecycle(Transactional txnLifecycle) {
+        this.txnLifecycle = txnLifecycle;
+        return this;
+    
+    }
+
+    /** Set the {@link HttpClient} fir the conenction to tbe built */
+    public RDFConnectionRemoteBuilder httpClient(HttpClient httpClient) {
+        this.httpClient = httpClient;
+        return this;
+    }
+
+    /** Set the {@link HttpContext} for the conenction to tbe built */
+    public RDFConnectionRemoteBuilder httpContext(HttpContext httpContext) {
+        this.httpContext = httpContext;
+        return this;
+    }
+
+    /** Set the output format for sending RDF Datasets to the remote server.
+     * This is used for HTTP PUT and POST to a dataset. 
+     * This must be a quads format.
+     */
+    public RDFConnectionRemoteBuilder quadsFormat(RDFFormat fmtQuads) {
+        if ( ! RDFLanguages.isQuads(fmtQuads.getLang()) )
+            throw new RiotException("Not a language for RDF Datasets: "+fmtQuads);
+        this.outputQuads = fmtQuads;
+        return this;
+    }
+
+    /** Set the output format for sending RDF Datasets to the remote server.
+     * This is used for HTTP PUT and POST to a dataset. 
+     * This must be a quads format.
+     */
+    public RDFConnectionRemoteBuilder quadsFormat(Lang langQuads) {
+        Objects.requireNonNull(langQuads);
+        if ( ! RDFLanguages.isQuads(langQuads) )
+            throw new RiotException("Not a language for RDF Datasets: "+langQuads);
+        RDFFormat fmt = RDFWriterRegistry.defaultSerialization(langQuads);
+        if ( fmt == null )
+            throw new RiotException("Language name not recognized: "+langQuads);
+        quadsFormat(fmt);
+        return this;
+    }
+
+    /** Set the output format for sending RDF Datasets to the remote server.
+     * This is used for HTTP PUT and POST to a dataset. 
+     * This must be a quads format.
+     */
+    public RDFConnectionRemoteBuilder quadsFormat(String langQuads) {
+        Objects.requireNonNull(langQuads);
+        Lang lang = RDFLanguages.nameToLang(langQuads);
+        if ( lang == null )
+            throw new RiotException("Language name not recognized: "+langQuads);
+        quadsFormat(lang);
+        return this;
+    }
+
+    /** Set the output format for sending RDF graphs to the remote server. 
+     * This is used for the SPARQ Graph Store Protocol.
+     */
+    public RDFConnectionRemoteBuilder triplesFormat(RDFFormat fmtTriples) {
+        if ( ! RDFLanguages.isTriples(fmtTriples.getLang()) )
+            throw new RiotException("Not a language for RDF Graphs: "+fmtTriples);
+        this.outputTriples = fmtTriples;
+        return this;
+    }
+    
+    /** Set the output format for sending RDF graphs to the remote server. 
+     * This is used for the SPARQ Graph Store Protocol.
+     */
+    public RDFConnectionRemoteBuilder triplesFormat(Lang langTriples) {
+        Objects.requireNonNull(langTriples);
+        if ( ! RDFLanguages.isTriples(langTriples) )
+            throw new RiotException("Not a language for RDF triples: "+langTriples);
+        RDFFormat fmt = RDFWriterRegistry.defaultSerialization(langTriples);
+        if ( fmt == null )
+            throw new RiotException("Language name not recognized: "+langTriples);
+        triplesFormat(fmt);
+        return this;
+    }
+
+    /** Set the output format for sending RDF graphs to the remote server. 
+     * This is used for the SPARQ Graph Store Protocol.
+     */
+    public RDFConnectionRemoteBuilder triplesFormat(String langTriples) {
+        Objects.requireNonNull(langTriples);
+        Lang lang = RDFLanguages.nameToLang(langTriples);
+        if ( lang == null )
+            throw new RiotException("Language name not recognized: "+langTriples);
+        quadsFormat(lang);
+        return this;
+    }
+    
+    /** Set the HTTP {@code Accept:} header used to fetch RDF graph using the SPARQL Graph Store Protocol. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderGraph(String acceptGraph) {
+        this.acceptGraph = acceptGraph;
+        return this;
+    }
+
+    /** Set the HTTP {@code Accept:} header used to fetch RDF datasets using HTTP GET operations. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderDataset(String acceptDataset) {
+        this.acceptDataset = acceptDataset;
+        return this;
+    }
+
+    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol SELECT query. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderSelectQuery(String acceptSelectHeader) {
+        this.acceptSelectResult = acceptSelectHeader;
+        return this;
+    }
+
+    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol ASK query. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderAskQuery(String acceptAskHeader) {
+        this.acceptAskResult = acceptAskHeader;
+        return this;
+    }
+
+    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol CONSTRUCT or DESCRIBE query. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderGraphQuery(String acceptGraphResultHeader) {
+        this.acceptGraphResult = acceptGraphResultHeader;
+        return this;
+    }
+
+    // XXX Alternative:
+    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol query. */ 
+    public RDFConnectionRemoteBuilder acceptHeaderQuery(String acceptHeaderQuery) {
+        System.err.println("NOT YET IMPLEMENTED");
+        return this;
+    }
+
+    private Function<RDFConnectionRemoteBuilder, RDFConnection> creator = null;
+    /** Provide an alternative function to make the {@link RDFConnection} object.
+     * <p>
+     * Specialized use: This method allows for custom {@code RDFConnection}s.
+     */
+    public RDFConnectionRemoteBuilder creator(Function<RDFConnectionRemoteBuilder, RDFConnection> function) {
+        this.creator = function;
+        return this;
+    }
+
+    /** Build an {RDFConnection}. */ 
+    public RDFConnection build() {
+        requireNonNull(txnLifecycle);
+        requireNonNull(destination);
+        
+        Function<RDFConnectionRemoteBuilder, RDFConnection> maker = creator ;
+        
+        if ( maker == null )
+            maker = (b)->b.buildConnection();
+        
+        // Sort out service URLs.
+        // Delay until here because the order of destination and service settign sisnot
+        // defined and if being modifed, may be the wrong order to calculate as queryEndpoint() is called.
+        
+        queryURL = LibRDFConn.formServiceURL(destination, sQuery);
+        updateURL = LibRDFConn.formServiceURL(destination, sUpdate);
+        gspURL = LibRDFConn.formServiceURL(destination, sGSP);
+        
+        // XXX Sort out SPARQL Accept headers.
+        
+        return maker.apply(this);
+    }
+    
+    protected RDFConnectionRemote2 buildConnection() {
+        return new RDFConnectionRemote2(txnLifecycle, httpClient, httpContext, 
+                                        destination, queryURL, updateURL, gspURL,
+                                        outputQuads, outputTriples,
+                                        acceptDataset, acceptGraph,
+                                        acceptSelectResult, acceptAskResult, acceptGraphResult);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/e4b4f99f/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
index 0f6bfa4..46e2159 100644
--- a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
+++ b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
@@ -53,7 +53,7 @@ public class TestRDFConn {
     }
     
     private static void testServiceName(String destination, String service, String expected) {
-        String x = RDFConn.formServiceURL(destination, service);
+        String x = LibRDFConn.formServiceURL(destination, service);
         assertEquals(expected, x);
     }
     
@@ -120,7 +120,7 @@ public class TestRDFConn {
     }
 
     private void testGSP(String gsp, String graphName, String expected) {
-        String x = RDFConn.urlForGraph(gsp, graphName);
+        String x = LibRDFConn.urlForGraph(gsp, graphName);
         assertEquals(expected, x);
     }
     


[24/24] jena git commit: Merge commit 'refs/pull/365/head' of https://github.com/apache/jena

Posted by an...@apache.org.
Merge commit 'refs/pull/365/head' of https://github.com/apache/jena

This closes #365.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/b75fccef
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/b75fccef
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/b75fccef

Branch: refs/heads/master
Commit: b75fccefa94b4f9cc832f77ac2514a98009d288a
Parents: 42d3775 96d5ca0
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 27 22:53:52 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 27 22:53:52 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/riot/WebContent.java   |   5 +-
 .../apache/jena/riot/web/HttpResponseLib.java   |  36 +--
 .../sparql/engine/http/QueryEngineHTTP.java     |  58 +++-
 .../jena/sparql/modify/UpdateProcessRemote.java |   2 +-
 .../sparql/modify/UpdateProcessRemoteBase.java  |   8 +-
 .../jena/sparql/resultset/ResultsFormat.java    |   5 -
 .../jena/sparql/sse/lang/ParseHandlerPlain.java |   3 +-
 .../apache/jena/atlas/lib/cache/CacheOps.java   |   1 -
 .../jena/atlas/lib/cache/CacheSimple.java       |   2 +-
 .../main/java/org/apache/jena/fuseki/DEF.java   |   2 +-
 .../java/org/apache/jena/fuseki/FusekiLib.java  |  45 +++
 .../jena/fuseki/servlets/SPARQL_Query.java      |   8 +-
 .../test/rdfconnection/TS_RDFConnection2.java   |  36 ---
 .../TS_RDFConnectionIntegration.java            |  39 +++
 .../test/rdfconnection/TestBlankNodeBinary.java | 103 ++++++
 .../rdfconnection/TestRDFConnectionFuseki.java  |  30 ++
 .../TestRDFConnectionFusekiBinary.java          | 123 +++++++
 .../rdfconnection/TestRDFConnectionRemote.java  |   6 +-
 .../apache/jena/rdfconnection/LibRDFConn.java   |  72 +++++
 .../org/apache/jena/rdfconnection/RDFConn.java  |  71 -----
 .../jena/rdfconnection/RDFConnection.java       |  12 +-
 .../rdfconnection/RDFConnectionFactory.java     |  65 +++-
 .../jena/rdfconnection/RDFConnectionFuseki.java | 138 ++++++++
 .../jena/rdfconnection/RDFConnectionLocal.java  |   6 +-
 .../jena/rdfconnection/RDFConnectionRemote.java | 296 ++++++++++++-----
 .../RDFConnectionRemoteBuilder.java             | 318 +++++++++++++++++++
 .../rdfconnection/RDFConnectionWrapper.java     | 180 +++++++++++
 .../jena/rdfconnection/TS_RDFConnection.java    |   4 +-
 .../jena/rdfconnection/TestLibRDFConn.java      | 127 ++++++++
 .../apache/jena/rdfconnection/TestRDFConn.java  | 127 --------
 30 files changed, 1567 insertions(+), 361 deletions(-)
----------------------------------------------------------------------



[09/24] jena git commit: Direct setting of Accept header.

Posted by an...@apache.org.
Direct setting of Accept header.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/63e3dbe7
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/63e3dbe7
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/63e3dbe7

Branch: refs/heads/master
Commit: 63e3dbe79f9f61b4ffb7e17e320570d562ce0a9d
Parents: bd955ee
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 19:12:19 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 19:12:19 2018 +0000

----------------------------------------------------------------------
 .../sparql/engine/http/QueryEngineHTTP.java     | 26 +++++++++++++++++---
 1 file changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/63e3dbe7/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
index d793143..543e3a6 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
@@ -89,6 +89,9 @@ public class QueryEngineHTTP implements QueryExecution {
     private String constructContentType = defaultConstructHeader() ;
     private String datasetContentType   = defaultConstructDatasetHeader() ;
     
+    // If this is non-null, it overrides the ???ContentType choice. 
+    private String acceptHeader         = null;
+    
     // Received content type 
     private String httpResponseContentType = null ;
     /**
@@ -341,7 +344,7 @@ public class QueryEngineHTTP implements QueryExecution {
    private ResultSet execResultSetInner() {
         
         HttpQuery httpQuery = makeHttpQuery();
-        httpQuery.setAccept(selectContentType);
+        httpQuery.setAccept(chooseAcceptHeader(acceptHeader, selectContentType));
         InputStream in = httpQuery.exec();
 
         if (false) {
@@ -376,6 +379,13 @@ public class QueryEngineHTTP implements QueryExecution {
         return result;
     }
 
+   // XXX Move
+    private static String chooseAcceptHeader(String acceptHeader, String contentType) {
+        if ( acceptHeader != null )
+            return acceptHeader;
+        return contentType;
+    }
+
     @Override
     public Model execConstruct() {
         return execConstruct(GraphFactory.makeJenaDefaultModel());
@@ -458,7 +468,7 @@ public class QueryEngineHTTP implements QueryExecution {
     private Pair<InputStream, Lang> execConstructWorker(String contentType) {
         checkNotClosed() ;
         HttpQuery httpQuery = makeHttpQuery();
-        httpQuery.setAccept(contentType);
+        httpQuery.setAccept(chooseAcceptHeader(acceptHeader, contentType));
         InputStream in = httpQuery.exec();
         
         // Don't assume the endpoint actually gives back the content type we
@@ -483,7 +493,7 @@ public class QueryEngineHTTP implements QueryExecution {
     public boolean execAsk() {
         checkNotClosed() ;
         HttpQuery httpQuery = makeHttpQuery();
-        httpQuery.setAccept(askContentType);
+        httpQuery.setAccept(chooseAcceptHeader(acceptHeader, askContentType));
         try(InputStream in = httpQuery.exec()) {
             // Don't assume the endpoint actually gives back the content type we
             // asked for
@@ -883,5 +893,13 @@ public class QueryEngineHTTP implements QueryExecution {
         sBuff.append(str) ;
         if ( v < 1 )
             sBuff.append(";q=").append(v) ;
-    } 
+    }
+
+    /** Set the HTTP Accept header for the request.
+     * Unlike the {@code set??ContentType} operations, this is not checked 
+     * for validity.
+     */ 
+    public void setAcceptHeader(String acceptHeader) {
+        this.acceptHeader = acceptHeader;
+    }
 }
\ No newline at end of file


[10/24] jena git commit: Remove unused BIO format.

Posted by an...@apache.org.
Remove unused BIO format.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/f482aeaf
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/f482aeaf
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/f482aeaf

Branch: refs/heads/master
Commit: f482aeaf05a431e7978bf56a79ed73644cc4cce6
Parents: 63e3dbe
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 19:13:02 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 19:13:02 2018 +0000

----------------------------------------------------------------------
 jena-arq/src/main/java/org/apache/jena/riot/WebContent.java     | 4 ----
 .../java/org/apache/jena/sparql/resultset/ResultsFormat.java    | 5 -----
 2 files changed, 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/f482aeaf/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java b/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
index 344ce74..1928e36 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
@@ -127,10 +127,6 @@ public class WebContent
     public static final String      contentTypeResultsThrift     = "application/sparql-results+thrift" ;
     public static final ContentType ctResultsThrift              = ContentType.create(contentTypeResultsThrift) ;
     
-    // Unofficial
-    public static final String      contentTypeResultsBIO        = "application/sparql-results+bio" ;
-    public static final ContentType ctResultsBIO                 = ContentType.create(contentTypeResultsBIO) ;
-
     public static final String      contentTypeSPARQLQuery       = "application/sparql-query" ;
     public static final ContentType ctSPARQLQuery                = ContentType.create(contentTypeSPARQLQuery) ;
 

http://git-wip-us.apache.org/repos/asf/jena/blob/f482aeaf/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultsFormat.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultsFormat.java b/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultsFormat.java
index b959e60..a0ebb69 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultsFormat.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultsFormat.java
@@ -53,7 +53,6 @@ public class ResultsFormat extends Symbol
     static public ResultsFormat FMT_RS_CSV       = new ResultsFormat(contentTypeTextCSV) ;
     static public ResultsFormat FMT_RS_TSV       = new ResultsFormat(contentTypeTextTSV) ;
     static public ResultsFormat FMT_RS_SSE       = new ResultsFormat(contentTypeSSE) ;
-    static public ResultsFormat FMT_RS_BIO       = new ResultsFormat(contentTypeResultsBIO) ;
     static public ResultsFormat FMT_NONE         = new ResultsFormat("none") ;
     static public ResultsFormat FMT_TEXT         = new ResultsFormat("text") ;
     static public ResultsFormat FMT_TUPLES       = new ResultsFormat("tuples") ;
@@ -83,7 +82,6 @@ public class ResultsFormat extends Symbol
         names.put("sse",         FMT_RS_SSE) ;
         names.put("csv",         FMT_RS_CSV) ;
         names.put("tsv",         FMT_RS_TSV) ;
-        names.put("srb",         FMT_RS_BIO) ;
         names.put("text",        FMT_TEXT) ;
         names.put("count",       FMT_COUNT) ;
         names.put("tuples",      FMT_TUPLES) ;
@@ -162,9 +160,6 @@ public class ResultsFormat extends Symbol
         if ( url.endsWith(".sse") )
             return FMT_RS_SSE;
 
-        if ( url.endsWith(".srb") ) // BindingsIO format.
-            return FMT_RS_BIO;
-
         // Likely to be something completely different!
         if ( url.endsWith(".csv") )
             return FMT_RS_CSV;


[21/24] jena git commit: Revise safe accessing of properties.

Posted by an...@apache.org.
Revise safe accessing of properties.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/c88316f9
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/c88316f9
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/c88316f9

Branch: refs/heads/master
Commit: c88316f9410e2656efb97a5d72e362a8b504b9ba
Parents: 27771c3
Author: Andy Seaborne <an...@apache.org>
Authored: Mon Feb 26 16:14:20 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Mon Feb 26 16:14:20 2018 +0000

----------------------------------------------------------------------
 .../apache/jena/riot/adapters/AdapterRDFWriter.java | 10 ++++++----
 .../org/apache/jena/riot/lang/ReaderRIOTRDFXML.java | 16 ++++++++++------
 2 files changed, 16 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/c88316f9/jena-arq/src/main/java/org/apache/jena/riot/adapters/AdapterRDFWriter.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/adapters/AdapterRDFWriter.java b/jena-arq/src/main/java/org/apache/jena/riot/adapters/AdapterRDFWriter.java
index 76b87a2..89a5031 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/adapters/AdapterRDFWriter.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/adapters/AdapterRDFWriter.java
@@ -64,17 +64,19 @@ public abstract class AdapterRDFWriter extends WriterGraphRIOTBase
         w.write(ModelFactory.createModelForGraph(graph), out, baseURI) ;
     }
     
-    private static void setProperties(RDFWriter w, Context context) {
+    private void setProperties(RDFWriter w, Context context) {
         if ( context == null )
             return;
+        Map<String, Object> properties = null;
         try { 
             @SuppressWarnings("unchecked")
             Map<String, Object> p = (Map<String, Object>)(context.get(SysRIOT.sysRdfWriterProperties)) ;
-            if ( p != null )
-                p.forEach((k,v) -> w.setProperty(k, v)) ;
+            properties = p;
         } catch (Throwable ex) {
-            Log.warn(AdapterRDFWriter.class, "Problem setting properties", ex);
+            Log.warn(this, "Problem accessing the RDF/XML writer properties: properties ignored", ex);
         }
+        if ( properties != null )
+            properties.forEach((k,v) -> w.setProperty(k, v)) ;
     }
 }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/c88316f9/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java b/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
index 3c18978..2bc7593 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
@@ -36,9 +36,11 @@ import org.apache.jena.rdf.model.RDFErrorHandler ;
 import org.apache.jena.rdfxml.xmlinput.* ;
 import org.apache.jena.rdfxml.xmlinput.impl.ARPSaxErrorHandler ;
 import org.apache.jena.riot.*;
-import org.apache.jena.riot.adapters.AdapterRDFWriter;
 import org.apache.jena.riot.checker.CheckerLiterals ;
-import org.apache.jena.riot.system.*;
+import org.apache.jena.riot.system.ErrorHandler;
+import org.apache.jena.riot.system.IRIResolver;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.system.StreamRDF;
 import org.apache.jena.sparql.util.Context;
 import org.xml.sax.SAXException ;
 import org.xml.sax.SAXParseException ;
@@ -166,14 +168,16 @@ public class ReaderRIOTRDFXML implements ReaderRIOT
             arp.getOptions().setIRIFactory(IRIResolver.iriFactory());
 
         if ( context != null ) {
+            Map<String, Object> properties = null;
             try { 
                 @SuppressWarnings("unchecked")
                 Map<String, Object> p = (Map<String, Object>)(context.get(SysRIOT.sysRdfReaderProperties)) ;
-                if ( p != null )
-                    p.forEach((k,v) -> oneProperty(arpOptions, k, v)) ;
-            } catch (Throwable ex) {
-                Log.warn(AdapterRDFWriter.class, "Problem setting properties", ex);
+                properties = p;
+            } catch(Throwable ex) {
+                Log.warn(this, "Problem accessing the RDF/XML reader properties: properties ignored", ex);
             }
+            if ( properties != null )
+                properties.forEach((k,v) -> oneProperty(arpOptions, k, v)) ;
         }
         arp.setOptionsWith(arpOptions) ;
         


[17/24] jena git commit: Inline the example data. Consistent format.

Posted by an...@apache.org.
Inline the example data. Consistent format.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/27771c3f
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/27771c3f
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/27771c3f

Branch: refs/heads/master
Commit: 27771c3faf67634dc53287fb0d99ee37ef8befb2
Parents: e9eb97f
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 08:27:20 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 08:27:34 2018 +0000

----------------------------------------------------------------------
 .../arq/examples/riot/ExRIOT_5.java             | 103 +++++++++----------
 1 file changed, 51 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/27771c3f/jena-arq/src-examples/arq/examples/riot/ExRIOT_5.java
----------------------------------------------------------------------
diff --git a/jena-arq/src-examples/arq/examples/riot/ExRIOT_5.java b/jena-arq/src-examples/arq/examples/riot/ExRIOT_5.java
index 7d22ddc..e6bd556 100644
--- a/jena-arq/src-examples/arq/examples/riot/ExRIOT_5.java
+++ b/jena-arq/src-examples/arq/examples/riot/ExRIOT_5.java
@@ -16,25 +16,35 @@
  * limitations under the License.
  */
 
-package arq.examples.riot ;
-
-import java.io.InputStream ;
-import java.io.Reader ;
-import java.util.Iterator ;
-
-import org.apache.jena.atlas.logging.LogCtl ;
-import org.apache.jena.atlas.web.ContentType ;
-import org.apache.jena.graph.Graph ;
-import org.apache.jena.graph.Triple ;
-import org.apache.jena.rdf.model.Model ;
-import org.apache.jena.rdf.model.ModelFactory ;
-import org.apache.jena.riot.* ;
-import org.apache.jena.riot.adapters.RDFReaderRIOT ;
-import org.apache.jena.riot.system.*;
-import org.apache.jena.sparql.sse.Item ;
-import org.apache.jena.sparql.sse.SSE ;
-import org.apache.jena.sparql.sse.builders.BuilderGraph ;
-import org.apache.jena.sparql.util.Context ;
+package arq.examples.riot;
+
+import java.io.InputStream;
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.Iterator;
+
+import org.apache.jena.atlas.logging.LogCtl;
+import org.apache.jena.atlas.web.ContentType;
+import org.apache.jena.graph.Graph;
+import org.apache.jena.graph.Triple;
+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.LangBuilder;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.riot.RDFLanguages;
+import org.apache.jena.riot.RDFParserRegistry;
+import org.apache.jena.riot.ReaderRIOT;
+import org.apache.jena.riot.ReaderRIOTFactory;
+import org.apache.jena.riot.adapters.RDFReaderRIOT;
+import org.apache.jena.riot.system.ErrorHandler;
+import org.apache.jena.riot.system.ErrorHandlerFactory;
+import org.apache.jena.riot.system.ParserProfile;
+import org.apache.jena.riot.system.StreamRDF;
+import org.apache.jena.sparql.sse.Item;
+import org.apache.jena.sparql.sse.SSE;
+import org.apache.jena.sparql.sse.builders.BuilderGraph;
+import org.apache.jena.sparql.util.Context;
 
 /** Example of using RIOT : register a new input language */
 public class ExRIOT_5
@@ -42,40 +52,34 @@ public class ExRIOT_5
     static { LogCtl.setCmdLogging(); }
     
     public static void main(String... argv) {
-        Lang lang = LangBuilder.create("SSE", "text/x-sse").addFileExtensions("rsse").build() ;
+        Lang lang = LangBuilder.create("SSE", "text/x-sse").addFileExtensions("rsse").build();
         // This just registers the name, not the parser.
-        RDFLanguages.register(lang) ;
+        RDFLanguages.register(lang);
 
         // Register the parser factory.
-        ReaderRIOTFactory factory = new SSEReaderFactory() ;
-        RDFParserRegistry.registerLangTriples(lang, factory) ;
-
-        // use it ...
-        String filename = "/home/afs/tmp/data.rsse" ;
-        // model.read(filename)
-        System.out.println("## -- RDFDataMgr.loadModel") ;
-        Model model = RDFDataMgr.loadModel(filename) ;
-
-        // print results.
-        RDFDataMgr.write(System.out, model, Lang.TTL) ;
+        ReaderRIOTFactory factory = new SSEReaderFactory();
+        RDFParserRegistry.registerLangTriples(lang, factory);
 
-        System.out.println("## -- Model.read") ;
+        // use it ... inline data
+        String x = "(graph (<s> <p1> 123) (<s> <p2> 456) )"; 
         // Model.read( , "SSE")
-        Model model2 = ModelFactory.createDefaultModel().read(filename, "SSE") ;
-        RDFDataMgr.write(System.out, model2, Lang.TTL) ;
+        Model model = ModelFactory.createDefaultModel();
+        RDFDataMgr.read(model, new StringReader(x), "http://example/", lang);
+        // print results.
+        RDFDataMgr.write(System.out, model, Lang.TTL);      
     }
 
     static class SSEReaderFactory implements ReaderRIOTFactory
     {
         @Override
         public ReaderRIOT create(Lang language, ParserProfile profile) {
-            return new SSEReader() ;
+            return new SSEReader();
         }
     }
 
     static class SSEReader implements ReaderRIOT
     {
-        private ErrorHandler errorHandler = ErrorHandlerFactory.getDefaultErrorHandler() ;
+        private ErrorHandler errorHandler = ErrorHandlerFactory.getDefaultErrorHandler();
         
         // This is just an example - it reads a graph in
         // http://jena.apache.org/documentation/notes/sse.html
@@ -85,22 +89,21 @@ public class ExRIOT_5
 
         @Override
         public void read(InputStream in, String baseURI, ContentType ct, StreamRDF output, Context context) {
-            Item item = SSE.parse(in) ;
-            read(item, baseURI, ct, output, context) ;
-
+            Item item = SSE.parse(in);
+            read(item, baseURI, ct, output, context);
         }
 
         @Override
         public void read(Reader in, String baseURI, ContentType ct, StreamRDF output, Context context) {
-            Item item = SSE.parse(in) ;
-            read(item, baseURI, ct, output, context) ;
+            Item item = SSE.parse(in);
+            read(item, baseURI, ct, output, context);
         }
 
         private void read(Item item, String baseURI, ContentType ct, StreamRDF output, Context context) {
-            Graph graph = BuilderGraph.buildGraph(item) ;
-            Iterator<Triple> iter = graph.find(null, null, null) ;
-            for ( ; iter.hasNext() ; )
-                output.triple(iter.next()) ;
+            Graph graph = BuilderGraph.buildGraph(item);
+            Iterator<Triple> iter = graph.find(null, null, null);
+            for (; iter.hasNext(); )
+                output.triple(iter.next());
         }
     }
 
@@ -108,11 +111,7 @@ public class ExRIOT_5
     public static class RDFReaderSSE extends RDFReaderRIOT
     {
         public RDFReaderSSE() {
-            super("SSE") ;
+            super("SSE");
         }
     }
-
-    /*
-     * data.rsse : (graph (<s> <p1> 123) (<s> <p2> 456) )
-     */
-}
+}
\ No newline at end of file


[02/24] jena git commit: Sync for CacheSimple.getOrFill.

Posted by an...@apache.org.
Sync for CacheSimple.getOrFill.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/da71d0f6
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/da71d0f6
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/da71d0f6

Branch: refs/heads/master
Commit: da71d0f64f2d073429d2732c59d50e5eeb4e30c7
Parents: e4b4f99
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 13:54:06 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 13:54:06 2018 +0000

----------------------------------------------------------------------
 .../src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java    | 1 -
 .../src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/da71d0f6/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java
index c8560bb..2002f5c 100644
--- a/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheOps.java
@@ -47,4 +47,3 @@ class CacheOps {
         }
     }
 }
-

http://git-wip-us.apache.org/repos/asf/jena/blob/da71d0f6/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java
index 9e09d23..0ab01a7 100644
--- a/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/cache/CacheSimple.java
@@ -98,7 +98,7 @@ public class CacheSimple<K,V> implements Cache<K,V>
 
     @Override
     public V getOrFill(K key, Callable<V> callable) {
-        return CacheOps.getOrFill(this, key, callable) ;
+        return CacheOps.getOrFillSync(this, key, callable) ;
     }
 
     @Override


[06/24] jena git commit: JENA-1490: Wire in new code

Posted by an...@apache.org.
JENA-1490: Wire in new code


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/8aad1f62
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/8aad1f62
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/8aad1f62

Branch: refs/heads/master
Commit: 8aad1f622e3c4a96f33b88f8998ad29531bfdade
Parents: 036491f
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 14:00:51 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 14:00:51 2018 +0000

----------------------------------------------------------------------
 .../rdfconnection/RDFConnectionFactory.java     |  15 +-
 .../jena/rdfconnection/RDFConnectionFuseki.java |   6 +-
 .../jena/rdfconnection/RDFConnectionRemote.java | 236 +++++---
 .../rdfconnection/RDFConnectionRemote2.java     | 542 -------------------
 .../RDFConnectionRemoteBuilder.java             |   6 +-
 5 files changed, 191 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/8aad1f62/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
index 1e69b19..ef6371c 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
@@ -41,7 +41,7 @@ public class RDFConnectionFactory {
      * @see #connect(String, String, String, String)
      */
     public static RDFConnection connect(String destination) {
-        return new RDFConnectionRemote(destination);
+        return RDFConnectionRemote.create().destination(destination).build();
     }
 
     /** Create a connection specifying the URLs of the service.
@@ -54,7 +54,11 @@ public class RDFConnectionFactory {
     public static RDFConnection connect(String queryServiceEndpoint,
                                         String updateServiceEndpoint,
                                         String graphStoreProtocolEndpoint) {
-        return new RDFConnectionRemote(queryServiceEndpoint, updateServiceEndpoint, graphStoreProtocolEndpoint);
+        return RDFConnectionRemote.create()
+            .queryEndpoint(queryServiceEndpoint)
+            .updateEndpoint(updateServiceEndpoint)
+            .gspEndpoint(graphStoreProtocolEndpoint)
+            .build();
     }
     
     /** Create a connection to a remote location by URL.
@@ -71,7 +75,12 @@ public class RDFConnectionFactory {
                                         String queryServiceEndpoint,
                                         String updateServiceEndpoint,
                                         String graphStoreProtocolEndpoint) {
-        return new RDFConnectionRemote(datasetURL, queryServiceEndpoint, updateServiceEndpoint, graphStoreProtocolEndpoint);
+        return RDFConnectionRemote.create()
+            .destination(datasetURL)
+            .queryEndpoint(queryServiceEndpoint)
+            .updateEndpoint(updateServiceEndpoint)
+            .gspEndpoint(graphStoreProtocolEndpoint)
+            .build();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/jena/blob/8aad1f62/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
index 5468dc7..425450c 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
@@ -48,7 +48,7 @@ import org.apache.jena.update.UpdateRequest;
  * <p>
  * This adds the ability to work with blank nodes across the network.
  */
-public class RDFConnectionFuseki extends RDFConnectionRemote2 {
+public class RDFConnectionFuseki extends RDFConnectionRemote {
 
     /**
      * Create a connection builder which is initialized for the default Fuseki
@@ -58,7 +58,7 @@ public class RDFConnectionFuseki extends RDFConnectionRemote2 {
      * @return RDFConnectionRemoteBuilder
      */
     public static RDFConnectionRemoteBuilder create() {
-        return setupForFuseki(RDFConnectionRemote2.create());
+        return setupForFuseki(RDFConnectionRemote.create());
     }
 
     /** 
@@ -67,7 +67,7 @@ public class RDFConnectionFuseki extends RDFConnectionRemote2 {
      * @return RDFConnectionRemoteBuilder
      */
     public static RDFConnectionRemoteBuilder create(RDFConnectionFuseki other) {
-        return setupCreator(RDFConnectionRemote2.create(other));
+        return setupCreator(RDFConnectionRemote.create(other));
     }
     
     /** Fuseki settings */

http://git-wip-us.apache.org/repos/asf/jena/blob/8aad1f62/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
index 34f903e..cf17c49 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
@@ -30,10 +30,13 @@ import org.apache.http.entity.EntityTemplate;
 import org.apache.http.protocol.HttpContext;
 import org.apache.jena.atlas.io.IO;
 import org.apache.jena.atlas.web.HttpException;
+import org.apache.jena.atlas.web.TypedInputStream;
 import org.apache.jena.graph.Graph;
 import org.apache.jena.query.*;
 import org.apache.jena.rdf.model.Model;
 import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.rdfconnection.RDFConnection;
+import org.apache.jena.rdfconnection.RDFConnectionFactory;
 import org.apache.jena.riot.*;
 import org.apache.jena.riot.web.HttpCaptureResponse;
 import org.apache.jena.riot.web.HttpOp;
@@ -42,6 +45,8 @@ import org.apache.jena.sparql.ARQException;
 import org.apache.jena.sparql.core.DatasetGraph;
 import org.apache.jena.sparql.core.Transactional;
 import org.apache.jena.sparql.core.TransactionalLock;
+import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
+import org.apache.jena.system.Txn;
 import org.apache.jena.update.UpdateExecutionFactory;
 import org.apache.jena.update.UpdateProcessor;
 import org.apache.jena.update.UpdateRequest;
@@ -51,19 +56,51 @@ import org.apache.jena.web.HttpSC;
  * Implementation of the {@link RDFConnection} interface using remote SPARQL operations.  
  */
 public class RDFConnectionRemote implements RDFConnection {
+    // Adds a Builder to help with HTTP details.
+    
     private static final String fusekiDftSrvQuery   = "sparql";
     private static final String fusekiDftSrvUpdate  = "update";
     private static final String fusekiDftSrvGSP     = "data";
     
     private boolean isOpen = true; 
-    private final String destination;
-    private final String svcQuery;
-    private final String svcUpdate;
-    private final String svcGraphStore;
-    private HttpClient httpClient;
-    private HttpContext httpContext = null;
+    protected final String destination;
+    protected final String svcQuery;
+    protected final String svcUpdate;
+    protected final String svcGraphStore;
+    
+    protected final Transactional txnLifecycle;
+    protected final HttpClient httpClient;
+    protected final HttpContext httpContext;
+    
+    // On-the-wire settings.
+    protected final RDFFormat outputQuads; 
+    protected final RDFFormat outputTriples;
+    protected final String acceptGraph;
+    protected final String acceptDataset;
+    protected final String acceptSelectResult;
+    protected final String acceptAskResult;
+    protected final String acceptGraphResult;
+    
+    /** Create a {@link RDFConnectionRemoteBuilder}. */
+    public static RDFConnectionRemoteBuilder create() {
+        return new RDFConnectionRemoteBuilder();
+    }
+
+    /** 
+     * Create a {@link RDFConnectionRemoteBuilder} initialized with the
+     * settings of another {@code RDFConnectionRemote}.  
+     */
+    public static RDFConnectionRemoteBuilder create(RDFConnectionRemote base) {
+        return new RDFConnectionRemoteBuilder(base);
+    }
     
-    /** Create connection that will use the {@link HttpClient} using URL of the dataset and default service names */
+    /**
+     * Create connection that will use the {@link HttpClient} using URL of the dataset and
+     * default service names
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder}.
+     */
+    @Deprecated 
     public RDFConnectionRemote(HttpClient httpClient, String destination) {
         this(httpClient,
              requireNonNull(destination),
@@ -72,7 +109,13 @@ public class RDFConnectionRemote implements RDFConnection {
              fusekiDftSrvGSP);
     }
 
-    /** Create connection, using URL of the dataset and default service names */
+    /**
+     * Create connection, using URL of the dataset and default service names
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated 
     public RDFConnectionRemote(String destination) {
         this(requireNonNull(destination),
              fusekiDftSrvQuery, 
@@ -80,45 +123,113 @@ public class RDFConnectionRemote implements RDFConnection {
              fusekiDftSrvGSP);
     }
 
-    /** Create connection, using full URLs for services. Pass a null for "no service endpoint". */
+    /**
+     * Create connection, using full URLs for services. Pass a null for "no service
+     * endpoint".
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated 
     public RDFConnectionRemote(String sQuery, String sUpdate, String sGSP) {
         this(null, sQuery, sUpdate, sGSP);
     }
     
-    /** Create connection, using URL of the dataset and short names for the services */
+    /**
+     * Create connection, using URL of the dataset and names for the services. Short names
+     * are expanded against the destination. Absolute URIs are left unchanged.
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated
     public RDFConnectionRemote(String destination, String sQuery, String sUpdate, String sGSP) {
         this(null, destination, sQuery, sUpdate, sGSP);
     }
     
-    /** Create connection, using URL of the dataset and short names for the services */
+    /**
+     * Create connection, using URL of the dataset and names for the services. Short names
+     * are expanded against the destination. Absolute URIs are left unchanged.
+     * 
+     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
+     *             {@link RDFConnectionFactory} operation.
+     */
+    @Deprecated
     public RDFConnectionRemote(HttpClient httpClient, String destination, String sQuery, String sUpdate, String sGSP) {
-        this.destination = destination;
-        this.svcQuery = LibRDFConn.formServiceURL(destination, sQuery);
-        this.svcUpdate = LibRDFConn.formServiceURL(destination, sUpdate);
-        this.svcGraphStore = LibRDFConn.formServiceURL(destination, sGSP);
+        this(null, httpClient, null, destination, sQuery, sUpdate, sGSP,
+            RDFFormat.NQUADS, RDFFormat.NTRIPLES,
+            WebContent.defaultGraphAcceptHeader, WebContent.defaultDatasetAcceptHeader,
+            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader(), QueryEngineHTTP.defaultConstructHeader()
+            );
+    }
+
+    // Used by the builder.
+    protected RDFConnectionRemote(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
+                                   String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
+                                   String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
+                                   String acceptGraphResult) {
         this.httpClient = httpClient;
-    }
-    
+        this.httpContext = httpContext;
+        this.destination = destination;
+        this.svcQuery = queryURL;
+        this.svcUpdate = updateURL;
+        this.svcGraphStore = gspURL;
+        if ( txnLifecycle == null )
+            txnLifecycle  = TransactionalLock.createMRPlusSW();
+        this.txnLifecycle = txnLifecycle;
+        this.outputQuads = outputQuads;
+        this.outputTriples = outputTriples;
+        this.acceptDataset = acceptDataset;
+        this.acceptGraph = acceptGraph;
+        this.acceptSelectResult = acceptSelectResult;
+        this.acceptAskResult = acceptAskResult;
+        this.acceptGraphResult = acceptGraphResult;
+    }
+
+    /** Return the {@link HttpClient} in-use. */ 
     public HttpClient getHttpClient() {
         return httpClient;
     }
 
-    public void setHttpClient(HttpClient httpClient) {
-        this.httpClient = httpClient;
-    }
-
+    /** Return the {@link HttpContext} in-use. */ 
     public HttpContext getHttpContext() {
         return httpContext;
     }
-
-    public void setHttpContext(HttpContext httpContext) {
-        this.httpContext = httpContext;
+    
+    /** Return the destination URL for the connection. */
+    public String getDestination() {
+        return destination;
     }
 
     @Override
     public QueryExecution query(Query query) {
         checkQuery();
-        return exec(()->QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext));
+        // XXX ResultSetFormat
+        
+        
+        
+        return exec(()-> {
+            QueryExecution qExec = QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext);
+            QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
+            
+            // Set general HTTP header.
+            
+//            // Only one choice, not "Accept:"
+//            switch ( query.getQueryType() ) {
+//                case Query.QueryTypeSelect:
+//                    qEngine.setSelectContentType("");
+//                    break;
+//                case Query.QueryTypeAsk:
+//                    qEngine.setAskContentType("");
+//                    break;
+//                case Query.QueryTypeDescribe:
+//                case Query.QueryTypeConstruct:
+//                    qEngine.setModelContentType("");
+//                    break;
+//            }
+            return qExec ;
+        });
+        
     }
 
     @Override
@@ -144,7 +255,7 @@ public class RDFConnectionRemote implements RDFConnection {
     
     private Graph fetch$(String url) {
         HttpCaptureResponse<Graph> graph = HttpResponseLib.graphHandler();
-        exec(()->HttpOp.execHttpGet(url, WebContent.defaultGraphAcceptHeader, graph, this.httpClient, this.httpContext));
+        exec(()->HttpOp.execHttpGet(url, acceptGraph, graph, this.httpClient, this.httpContext));
         return graph.get();
     }
 
@@ -211,7 +322,7 @@ public class RDFConnectionRemote implements RDFConnection {
         doPutPost(url, file, lang, replace);
     }
 
-    /** Send a file to named graph (or "default" or null for the default graph).
+    /** Send a file to named graph (or "default" or null for the defaultl graph).
      * <p>
      * The Content-Type is taken from the given {@code Lang}.
      * <p>
@@ -230,7 +341,7 @@ public class RDFConnectionRemote implements RDFConnection {
         });
     }
 
-    /** Send a model to named graph (or "default" or null for the default graph).
+    /** Send a model to named graph (or "default" or null for the defaultl graph).
      * <p>
      * The Content-Type is taken from the given {@code Lang}.
      * <p>
@@ -259,29 +370,31 @@ public class RDFConnectionRemote implements RDFConnection {
         checkGSP();
         delete(null);
     }
-    
+
     @Override
     public Dataset fetchDataset() {
-        checkDataset();
-        DatasetGraph dsg = fetchDataset$();
-        return DatasetFactory.wrap(dsg);
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URL provided"); 
+        Dataset ds = DatasetFactory.createTxnMem();
+        Txn.executeWrite(ds, ()->{
+            TypedInputStream s = exec(()->HttpOp.execHttpGet(destination, acceptDataset));
+            Lang lang = RDFLanguages.contentTypeToLang(s.getContentType());
+            RDFDataMgr.read(ds, s, lang);
+        });
+        return ds;
     }
 
-    private DatasetGraph fetchDataset$() {
-        HttpCaptureResponse<DatasetGraph> dsg = HttpResponseLib.datasetHandler();
-        exec(()->HttpOp.execHttpGet(destination, WebContent.defaultDatasetAcceptHeader, dsg, this.httpClient, this.httpContext));
-        return dsg.get();
-    }
-    
     @Override
     public void loadDataset(String file) { 
-        checkDataset();
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
         doPutPostDataset(file, false); 
     }
     
     @Override
     public void loadDataset(Dataset dataset) {
-        checkDataset();
+        if ( destination == null )
+            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
         doPutPostDataset(dataset, false); 
     }
 
@@ -318,7 +431,7 @@ public class RDFConnectionRemote implements RDFConnection {
         });
     }
 
-    /** Do a PUT or POST to a dataset, sending the contents of a datasets.
+    /** Do a PUT or POST to a dataset, sending the contents of a daatsets.
      * The Content-Type is {@code application/n-quads}.
      * <p>
      * "Replace" implies PUT, otherwise a POST is used.
@@ -333,31 +446,31 @@ public class RDFConnectionRemote implements RDFConnection {
         });
     }
 
-    private void checkQuery() {
+    protected void checkQuery() {
         checkOpen();
         if ( svcQuery == null )
             throw new ARQException("No query service defined for this RDFConnection");
     }
     
-    private void checkUpdate() {
+    protected void checkUpdate() {
         checkOpen();
         if ( svcUpdate == null )
             throw new ARQException("No update service defined for this RDFConnection");
     }
     
-    private void checkGSP() {
+    protected void checkGSP() {
         checkOpen();
         if ( svcGraphStore == null )
             throw new ARQException("No SPARQL Graph Store service defined for this RDFConnection");
     }
     
-    private void checkDataset() {
+    protected void checkDataset() {
         checkOpen();
         if ( destination == null )
             throw new ARQException("Dataset operations not available - no dataset URL provided"); 
     }
 
-    private void checkOpen() {
+    protected void checkOpen() {
         if ( ! isOpen )
             throw new ARQException("closed");
     }
@@ -374,7 +487,7 @@ public class RDFConnectionRemote implements RDFConnection {
 
     /** Create an HttpEntity for the graph */  
     protected HttpEntity graphToHttpEntity(Graph graph) {
-        return graphToHttpEntity(graph, RDFFormat.NTRIPLES);
+        return graphToHttpEntity(graph, outputTriples);
     }
     
     /** Create an HttpEntity for the graph */
@@ -387,7 +500,7 @@ public class RDFConnectionRemote implements RDFConnection {
 
     /** Create an HttpEntity for the dataset */  
     protected HttpEntity datasetToHttpEntity(DatasetGraph dataset) {
-        return datasetToHttpEntity(dataset, RDFFormat.NQUADS);
+        return datasetToHttpEntity(dataset, outputQuads);
     }
     
     /** Create an HttpEntity for the dataset */  
@@ -399,13 +512,13 @@ public class RDFConnectionRemote implements RDFConnection {
     }
 
     /** Convert HTTP status codes to exceptions */ 
-    static void exec(Runnable action)  {
+    static protected void exec(Runnable action)  {
         try { action.run(); }
         catch (HttpException ex) { handleHttpException(ex, false); }
     }
 
     /** Convert HTTP status codes to exceptions */ 
-    static <X> X exec(Supplier<X> action)  {
+    static protected <X> X exec(Supplier<X> action)  {
         try { return action.get(); }
         catch (HttpException ex) { handleHttpException(ex, true); return null;}
     }
@@ -416,17 +529,14 @@ public class RDFConnectionRemote implements RDFConnection {
         throw ex;
     }
 
-    private final Transactional txn = TransactionalLock.createMRPlusSW();
-    
-    @Override public void begin()                       { txn.begin(); }
-    @Override public void begin(TxnType txnType)        { txn.begin(txnType); }
-    @Override public void begin(ReadWrite mode)         { txn.begin(mode); }
-    @Override public boolean promote(Promote promote)   { return txn.promote(promote); }
-    @Override public void commit()                      { txn.commit(); }
-    @Override public void abort()                       { txn.abort(); }
-    @Override public boolean isInTransaction()          { return txn.isInTransaction(); }
-    @Override public void end()                         { txn.end(); }
-    @Override public ReadWrite transactionMode()        { return txn.transactionMode(); }
-    @Override public TxnType transactionType()          { return txn.transactionType(); }
+    @Override public void begin()                       { txnLifecycle.begin(); }
+    @Override public void begin(TxnType txnType)        { txnLifecycle.begin(txnType); }
+    @Override public void begin(ReadWrite mode)         { txnLifecycle.begin(mode); }
+    @Override public boolean promote(Promote promote)   { return txnLifecycle.promote(promote); }
+    @Override public void commit()                      { txnLifecycle.commit(); }
+    @Override public void abort()                       { txnLifecycle.abort(); }
+    @Override public boolean isInTransaction()          { return txnLifecycle.isInTransaction(); }
+    @Override public void end()                         { txnLifecycle.end(); }
+    @Override public ReadWrite transactionMode()        { return txnLifecycle.transactionMode(); }
+    @Override public TxnType transactionType()          { return txnLifecycle.transactionType(); }
 }
-

http://git-wip-us.apache.org/repos/asf/jena/blob/8aad1f62/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
deleted file mode 100644
index 3440f18..0000000
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote2.java
+++ /dev/null
@@ -1,542 +0,0 @@
-/*
- * 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.rdfconnection;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.File;
-import java.io.InputStream;
-import java.util.function.Supplier;
-
-import org.apache.http.HttpEntity;
-import org.apache.http.client.HttpClient;
-import org.apache.http.entity.EntityTemplate;
-import org.apache.http.protocol.HttpContext;
-import org.apache.jena.atlas.io.IO;
-import org.apache.jena.atlas.web.HttpException;
-import org.apache.jena.atlas.web.TypedInputStream;
-import org.apache.jena.graph.Graph;
-import org.apache.jena.query.*;
-import org.apache.jena.rdf.model.Model;
-import org.apache.jena.rdf.model.ModelFactory;
-import org.apache.jena.rdfconnection.RDFConnection;
-import org.apache.jena.rdfconnection.RDFConnectionFactory;
-import org.apache.jena.riot.*;
-import org.apache.jena.riot.web.HttpCaptureResponse;
-import org.apache.jena.riot.web.HttpOp;
-import org.apache.jena.riot.web.HttpResponseLib;
-import org.apache.jena.sparql.ARQException;
-import org.apache.jena.sparql.core.DatasetGraph;
-import org.apache.jena.sparql.core.Transactional;
-import org.apache.jena.sparql.core.TransactionalLock;
-import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
-import org.apache.jena.system.Txn;
-import org.apache.jena.update.UpdateExecutionFactory;
-import org.apache.jena.update.UpdateProcessor;
-import org.apache.jena.update.UpdateRequest;
-import org.apache.jena.web.HttpSC;
-
-/** 
- * Implementation of the {@link RDFConnection} interface using remote SPARQL operations.  
- */
-public class RDFConnectionRemote2 implements RDFConnection {
-    // Adds a Builder to help with HTTP details.
-    
-    private static final String fusekiDftSrvQuery   = "sparql";
-    private static final String fusekiDftSrvUpdate  = "update";
-    private static final String fusekiDftSrvGSP     = "data";
-    
-    private boolean isOpen = true; 
-    protected final String destination;
-    protected final String svcQuery;
-    protected final String svcUpdate;
-    protected final String svcGraphStore;
-    
-    protected final Transactional txnLifecycle;
-    protected final HttpClient httpClient;
-    protected final HttpContext httpContext;
-    
-    // On-the-wire settings.
-    protected final RDFFormat outputQuads; 
-    protected final RDFFormat outputTriples;
-    protected final String acceptGraph;
-    protected final String acceptDataset;
-    protected final String acceptSelectResult;
-    protected final String acceptAskResult;
-    protected final String acceptGraphResult;
-    
-    /** Create a {@link RDFConnectionRemoteBuilder}. */
-    public static RDFConnectionRemoteBuilder create() {
-        return new RDFConnectionRemoteBuilder();
-    }
-
-    /** 
-     * Create a {@link RDFConnectionRemoteBuilder} initialized with the
-     * settings of another {@code RDFConnectionRemote}.  
-     */
-    public static RDFConnectionRemoteBuilder create(RDFConnectionRemote2 base) {
-        return new RDFConnectionRemoteBuilder(base);
-    }
-    
-    /**
-     * Create connection that will use the {@link HttpClient} using URL of the dataset and
-     * default service names
-     * 
-     * @deprecated Use {@link RDFConnectionRemoteBuilder}.
-     */
-    @Deprecated 
-    public RDFConnectionRemote2(HttpClient httpClient, String destination) {
-        this(httpClient,
-             requireNonNull(destination),
-             fusekiDftSrvQuery, 
-             fusekiDftSrvUpdate,
-             fusekiDftSrvGSP);
-    }
-
-    /**
-     * Create connection, using URL of the dataset and default service names
-     * 
-     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
-     *             {@link RDFConnectionFactory} operation.
-     */
-    @Deprecated 
-    public RDFConnectionRemote2(String destination) {
-        this(requireNonNull(destination),
-             fusekiDftSrvQuery, 
-             fusekiDftSrvUpdate,
-             fusekiDftSrvGSP);
-    }
-
-    /**
-     * Create connection, using full URLs for services. Pass a null for "no service
-     * endpoint".
-     * 
-     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
-     *             {@link RDFConnectionFactory} operation.
-     */
-    @Deprecated 
-    public RDFConnectionRemote2(String sQuery, String sUpdate, String sGSP) {
-        this(null, sQuery, sUpdate, sGSP);
-    }
-    
-    /**
-     * Create connection, using URL of the dataset and names for the services. Short names
-     * are expanded against the destination. Absolute URIs are left unchanged.
-     * 
-     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
-     *             {@link RDFConnectionFactory} operation.
-     */
-    @Deprecated
-    public RDFConnectionRemote2(String destination, String sQuery, String sUpdate, String sGSP) {
-        this(null, destination, sQuery, sUpdate, sGSP);
-    }
-    
-    /**
-     * Create connection, using URL of the dataset and names for the services. Short names
-     * are expanded against the destination. Absolute URIs are left unchanged.
-     * 
-     * @deprecated Use {@link RDFConnectionRemoteBuilder} or an
-     *             {@link RDFConnectionFactory} operation.
-     */
-    @Deprecated
-    public RDFConnectionRemote2(HttpClient httpClient, String destination, String sQuery, String sUpdate, String sGSP) {
-        this(null, httpClient, null, destination, sQuery, sUpdate, sGSP,
-            RDFFormat.NQUADS, RDFFormat.NTRIPLES,
-            WebContent.defaultGraphAcceptHeader, WebContent.defaultDatasetAcceptHeader,
-            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader(), QueryEngineHTTP.defaultConstructHeader()
-            );
-    }
-
-    // Used by the builder.
-    protected RDFConnectionRemote2(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
-                                   String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
-                                   String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
-                                   String acceptGraphResult) {
-        this.httpClient = httpClient;
-        this.httpContext = httpContext;
-        this.destination = destination;
-        this.svcQuery = queryURL;
-        this.svcUpdate = updateURL;
-        this.svcGraphStore = gspURL;
-        if ( txnLifecycle == null )
-            txnLifecycle  = TransactionalLock.createMRPlusSW();
-        this.txnLifecycle = txnLifecycle;
-        this.outputQuads = outputQuads;
-        this.outputTriples = outputTriples;
-        this.acceptDataset = acceptDataset;
-        this.acceptGraph = acceptGraph;
-        this.acceptSelectResult = acceptSelectResult;
-        this.acceptAskResult = acceptAskResult;
-        this.acceptGraphResult = acceptGraphResult;
-    }
-
-    /** Return the {@link HttpClient} in-use. */ 
-    public HttpClient getHttpClient() {
-        return httpClient;
-    }
-
-    /** Return the {@link HttpContext} in-use. */ 
-    public HttpContext getHttpContext() {
-        return httpContext;
-    }
-    
-    /** Return the destination URL for the connection. */
-    public String getDestination() {
-        return destination;
-    }
-
-    @Override
-    public QueryExecution query(Query query) {
-        checkQuery();
-        // XXX ResultSetFormat
-        
-        
-        
-        return exec(()-> {
-            QueryExecution qExec = QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext);
-            QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
-            
-            // Set general HTTP header.
-            
-//            // Only one choice, not "Accept:"
-//            switch ( query.getQueryType() ) {
-//                case Query.QueryTypeSelect:
-//                    qEngine.setSelectContentType("");
-//                    break;
-//                case Query.QueryTypeAsk:
-//                    qEngine.setAskContentType("");
-//                    break;
-//                case Query.QueryTypeDescribe:
-//                case Query.QueryTypeConstruct:
-//                    qEngine.setModelContentType("");
-//                    break;
-//            }
-            return qExec ;
-        });
-        
-    }
-
-    @Override
-    public void update(UpdateRequest update) {
-        checkUpdate();
-        UpdateProcessor proc = UpdateExecutionFactory.createRemote(update, svcUpdate, this.httpClient, this.httpContext);
-        exec(()->proc.execute());
-    }
-    
-    @Override
-    public Model fetch(String graphName) {
-        checkGSP();
-        String url = LibRDFConn.urlForGraph(svcGraphStore, graphName);
-        Graph graph = fetch$(url);
-        return ModelFactory.createModelForGraph(graph);
-    }
-    
-    @Override
-    public Model fetch() {
-        checkGSP();
-        return fetch(null);
-    }
-    
-    private Graph fetch$(String url) {
-        HttpCaptureResponse<Graph> graph = HttpResponseLib.graphHandler();
-        exec(()->HttpOp.execHttpGet(url, acceptGraph, graph, this.httpClient, this.httpContext));
-        return graph.get();
-    }
-
-    @Override
-    public void load(String graph, String file) {
-        checkGSP();
-        upload(graph, file, false);
-    }
-    
-    @Override
-    public void load(String file) {
-        checkGSP();
-        upload(null, file, false);
-    }
-    
-    @Override
-    public void load(Model model) {
-        doPutPost(model, null, false);
-    }
-    
-    @Override
-    public void load(String graphName, Model model) {
-        doPutPost(model, graphName, false);
-    }
-    
-    @Override
-    public void put(String graph, String file) {
-        checkGSP();
-        upload(graph, file, true);
-    }
-    
-    @Override
-    public void put(String file) { 
-        checkGSP();
-        upload(null, file, true); 
-    }
-    
-    @Override
-    public void put(String graphName, Model model) {
-        checkGSP();
-        doPutPost(model, graphName, true);
-    }
-
-    @Override
-    public void put(Model model) {
-        checkGSP();
-        doPutPost(model, null, true);
-    }
-    
-    /** Send a file to named graph (or "default" or null for the default graph).
-     * <p>
-     * The Content-Type is inferred from the file extension.
-     * <p>
-     * "Replace" means overwrite existing data, othewise the date is added to the target.
-     */
-    protected void upload(String graph, String file, boolean replace) {
-        // if triples
-        Lang lang = RDFLanguages.filenameToLang(file);
-        if ( RDFLanguages.isQuads(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+")");
-        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
-        doPutPost(url, file, lang, replace);
-    }
-
-    /** Send a file to named graph (or "default" or null for the defaultl graph).
-     * <p>
-     * The Content-Type is taken from the given {@code Lang}.
-     * <p>
-     * "Replace" means overwrite existing data, othewise the date is added to the target.
-     */
-    protected void doPutPost(String url, String file, Lang lang, boolean replace) {
-        File f = new File(file);
-        long length = f.length(); 
-        InputStream source = IO.openFile(file);
-        // Charset.
-        exec(()->{
-            if ( replace )
-                HttpOp.execHttpPut(url, lang.getContentType().getContentType(), source, length, httpClient, this.httpContext);
-            else    
-                HttpOp.execHttpPost(url, lang.getContentType().getContentType(), source, length, null, null, httpClient, this.httpContext);
-        });
-    }
-
-    /** Send a model to named graph (or "default" or null for the defaultl graph).
-     * <p>
-     * The Content-Type is taken from the given {@code Lang}.
-     * <p>
-     * "Replace" means overwrite existing data, othewise the date is added to the target.
-     */
-    protected void doPutPost(Model model, String name, boolean replace) {
-        String url = LibRDFConn.urlForGraph(svcGraphStore, name);
-        exec(()->{
-            Graph graph = model.getGraph();
-            if ( replace )
-                HttpOp.execHttpPut(url, graphToHttpEntity(graph), httpClient, this.httpContext);
-            else    
-                HttpOp.execHttpPost(url, graphToHttpEntity(graph), null, null, httpClient, this.httpContext);
-        });
-    }
-
-    @Override
-    public void delete(String graph) {
-        checkGSP();
-        String url = LibRDFConn.urlForGraph(svcGraphStore, graph);
-        exec(()->HttpOp.execHttpDelete(url));
-    }
-
-    @Override
-    public void delete() {
-        checkGSP();
-        delete(null);
-    }
-
-    @Override
-    public Dataset fetchDataset() {
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URL provided"); 
-        Dataset ds = DatasetFactory.createTxnMem();
-        Txn.executeWrite(ds, ()->{
-            TypedInputStream s = exec(()->HttpOp.execHttpGet(destination, acceptDataset));
-            Lang lang = RDFLanguages.contentTypeToLang(s.getContentType());
-            RDFDataMgr.read(ds, s, lang);
-        });
-        return ds;
-    }
-
-    @Override
-    public void loadDataset(String file) { 
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
-        doPutPostDataset(file, false); 
-    }
-    
-    @Override
-    public void loadDataset(Dataset dataset) {
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
-        doPutPostDataset(dataset, false); 
-    }
-
-    @Override
-    public void putDataset(String file) {
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
-        doPutPostDataset(file, true);
-    }
-    
-    @Override
-    public void putDataset(Dataset dataset) {
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URl provided"); 
-        doPutPostDataset(dataset, true); 
-    }
-
-    /** Do a PUT or POST to a dataset, sending the contents of the file.
-     * <p>
-     * The Content-Type is inferred from the file extension.
-     * <p>
-     * "Replace" implies PUT, otherwise a POST is used.
-     */
-    protected void doPutPostDataset(String file, boolean replace) {
-        Lang lang = RDFLanguages.filenameToLang(file);
-        File f = new File(file);
-        long length = f.length();
-        exec(()->{
-            InputStream source = IO.openFile(file);
-            if ( replace )
-                HttpOp.execHttpPut(destination, lang.getContentType().getContentType(), source, length, httpClient, httpContext);
-            else    
-                HttpOp.execHttpPost(destination, lang.getContentType().getContentType(), source, length, null, null, httpClient, httpContext);
-        });
-    }
-
-    /** Do a PUT or POST to a dataset, sending the contents of a daatsets.
-     * The Content-Type is {@code application/n-quads}.
-     * <p>
-     * "Replace" implies PUT, otherwise a POST is used.
-     */
-    protected void doPutPostDataset(Dataset dataset, boolean replace) {
-        exec(()->{
-            DatasetGraph dsg = dataset.asDatasetGraph();
-            if ( replace )
-                HttpOp.execHttpPut(destination, datasetToHttpEntity(dsg), httpClient, null);
-            else    
-                HttpOp.execHttpPost(destination, datasetToHttpEntity(dsg), httpClient, null);
-        });
-    }
-
-    protected void checkQuery() {
-        checkOpen();
-        if ( svcQuery == null )
-            throw new ARQException("No query service defined for this RDFConnection");
-    }
-    
-    protected void checkUpdate() {
-        checkOpen();
-        if ( svcUpdate == null )
-            throw new ARQException("No update service defined for this RDFConnection");
-    }
-    
-    protected void checkGSP() {
-        checkOpen();
-        if ( svcGraphStore == null )
-            throw new ARQException("No SPARQL Graph Store service defined for this RDFConnection");
-    }
-    
-    protected void checkDataset() {
-        checkOpen();
-        if ( destination == null )
-            throw new ARQException("Dataset operations not available - no dataset URL provided"); 
-    }
-
-    protected void checkOpen() {
-        if ( ! isOpen )
-            throw new ARQException("closed");
-    }
-
-    @Override
-    public void close() {
-        isOpen = false;
-    }
-
-    @Override
-    public boolean isClosed() {
-        return ! isOpen;
-    }
-
-    /** Create an HttpEntity for the graph */  
-    protected HttpEntity graphToHttpEntity(Graph graph) {
-        return graphToHttpEntity(graph, outputTriples);
-    }
-    
-    /** Create an HttpEntity for the graph */
-    protected HttpEntity graphToHttpEntity(Graph graph, RDFFormat syntax) {
-        EntityTemplate entity = new EntityTemplate((out)->RDFDataMgr.write(out, graph, syntax));
-        String ct = syntax.getLang().getContentType().getContentType();
-        entity.setContentType(ct);
-        return entity;
-    }
-
-    /** Create an HttpEntity for the dataset */  
-    protected HttpEntity datasetToHttpEntity(DatasetGraph dataset) {
-        return datasetToHttpEntity(dataset, outputQuads);
-    }
-    
-    /** Create an HttpEntity for the dataset */  
-    protected HttpEntity datasetToHttpEntity(DatasetGraph dataset, RDFFormat syntax) {
-        EntityTemplate entity = new EntityTemplate((out)->RDFDataMgr.write(out, dataset, syntax));
-        String ct = syntax.getLang().getContentType().getContentType();
-        entity.setContentType(ct);
-        return entity;
-    }
-
-    /** Convert HTTP status codes to exceptions */ 
-    static protected void exec(Runnable action)  {
-        try { action.run(); }
-        catch (HttpException ex) { handleHttpException(ex, false); }
-    }
-
-    /** Convert HTTP status codes to exceptions */ 
-    static protected <X> X exec(Supplier<X> action)  {
-        try { return action.get(); }
-        catch (HttpException ex) { handleHttpException(ex, true); return null;}
-    }
-
-    private static void handleHttpException(HttpException ex, boolean ignore404) {
-        if ( ex.getResponseCode() == HttpSC.NOT_FOUND_404 && ignore404 )
-            return ;
-        throw ex;
-    }
-
-    @Override public void begin()                       { txnLifecycle.begin(); }
-    @Override public void begin(TxnType txnType)        { txnLifecycle.begin(txnType); }
-    @Override public void begin(ReadWrite mode)         { txnLifecycle.begin(mode); }
-    @Override public boolean promote(Promote promote)   { return txnLifecycle.promote(promote); }
-    @Override public void commit()                      { txnLifecycle.commit(); }
-    @Override public void abort()                       { txnLifecycle.abort(); }
-    @Override public boolean isInTransaction()          { return txnLifecycle.isInTransaction(); }
-    @Override public void end()                         { txnLifecycle.end(); }
-    @Override public ReadWrite transactionMode()        { return txnLifecycle.transactionMode(); }
-    @Override public TxnType transactionType()          { return txnLifecycle.transactionType(); }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/8aad1f62/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
index d47f36a..e77e738 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
@@ -62,7 +62,7 @@ public class RDFConnectionRemoteBuilder {
         // Default settings are the meber declarations.
     }
     
-    RDFConnectionRemoteBuilder(RDFConnectionRemote2 base) {
+    RDFConnectionRemoteBuilder(RDFConnectionRemote base) {
         Objects.requireNonNull(base);
         txnLifecycle = base.txnLifecycle;
         if ( txnLifecycle == null )
@@ -300,8 +300,8 @@ public class RDFConnectionRemoteBuilder {
         return maker.apply(this);
     }
     
-    protected RDFConnectionRemote2 buildConnection() {
-        return new RDFConnectionRemote2(txnLifecycle, httpClient, httpContext, 
+    protected RDFConnectionRemote buildConnection() {
+        return new RDFConnectionRemote(txnLifecycle, httpClient, httpContext, 
                                         destination, queryURL, updateURL, gspURL,
                                         outputQuads, outputTriples,
                                         acceptDataset, acceptGraph,


[03/24] jena git commit: Correct accept content negotiations headers.

Posted by an...@apache.org.
Correct accept content negotiations headers.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/0a73699a
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/0a73699a
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/0a73699a

Branch: refs/heads/master
Commit: 0a73699a8c9146b6c163b09abe8b9ee1ece2a766
Parents: da71d0f
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 13:54:43 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 13:54:43 2018 +0000

----------------------------------------------------------------------
 jena-arq/src/main/java/org/apache/jena/riot/WebContent.java     | 1 +
 .../org/apache/jena/sparql/engine/http/QueryEngineHTTP.java     | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/0a73699a/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java b/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
index f265acc..344ce74 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/WebContent.java
@@ -123,6 +123,7 @@ public class WebContent
     public static final String      contentTypeJSON              = "application/json" ;
     public static final ContentType ctJSON                       = ContentType.create(contentTypeJSON) ;
     
+    // Unofficial
     public static final String      contentTypeResultsThrift     = "application/sparql-results+thrift" ;
     public static final ContentType ctResultsThrift              = ContentType.create(contentTypeResultsThrift) ;
     

http://git-wip-us.apache.org/repos/asf/jena/blob/0a73699a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
index d424b76..d793143 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
@@ -95,12 +95,13 @@ public class QueryEngineHTTP implements QueryExecution {
      * Supported content types for SELECT queries
      */
     public static String[] supportedSelectContentTypes = new String[] { WebContent.contentTypeResultsXML,
-            WebContent.contentTypeResultsJSON, WebContent.contentTypeTextTSV, WebContent.contentTypeTextCSV };
+            WebContent.contentTypeResultsJSON, WebContent.contentTypeTextTSV, WebContent.contentTypeTextCSV,
+            WebContent.contentTypeResultsThrift};
     /**
      * Supported content types for ASK queries
      */
     public static String[] supportedAskContentTypes = new String[] { WebContent.contentTypeResultsXML,
-            WebContent.contentTypeJSON, WebContent.contentTypeTextTSV, WebContent.contentTypeTextCSV };
+            WebContent.contentTypeResultsJSON, WebContent.contentTypeTextTSV, WebContent.contentTypeTextCSV };
 
     // Releasing HTTP input streams is important. We remember this for SELECT,
     // and will close when the engine is closed


[08/24] jena git commit: Hide content type to ResultsFormat conversion (old world).

Posted by an...@apache.org.
Hide content type to ResultsFormat conversion (old world).

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/bd955ee0
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/bd955ee0
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/bd955ee0

Branch: refs/heads/master
Commit: bd955ee0ec4e048ec232875899576c090e7cae9f
Parents: 09863b4
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 19:11:36 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 19:11:36 2018 +0000

----------------------------------------------------------------------
 .../apache/jena/riot/web/HttpResponseLib.java   | 36 +++++++++-----------
 1 file changed, 17 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/bd955ee0/jena-arq/src/main/java/org/apache/jena/riot/web/HttpResponseLib.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/web/HttpResponseLib.java b/jena-arq/src/main/java/org/apache/jena/riot/web/HttpResponseLib.java
index e545808..a249493 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/web/HttpResponseLib.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/web/HttpResponseLib.java
@@ -53,8 +53,7 @@ public class HttpResponseLib
     {
         private Graph graph = null ;
         @Override
-        final public void handle(String baseIRI, HttpResponse response)
-        {
+        final public void handle(String baseIRI, HttpResponse response) {
             try {
                 Graph g = GraphFactory.createDefaultGraph() ;
                 HttpEntity entity = response.getEntity() ;
@@ -79,8 +78,7 @@ public class HttpResponseLib
     {
         private DatasetGraph dsg = null ;
         @Override
-        final public void handle(String baseIRI, HttpResponse response)
-        {
+        final public void handle(String baseIRI, HttpResponse response) {
             try {
                 DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
                 HttpEntity entity = response.getEntity() ;
@@ -133,10 +131,11 @@ public class HttpResponseLib
         }
     } ;
     
-    public static ResultsFormat contentTypeToResultSet(String contentType) { return mapContentTypeToResultSet.get(contentType) ; }
+    // Old world.
+    // See also ResultSetFactory.load(in, fmt) 
+    private static ResultsFormat contentTypeToResultsFormat(String contentType) { return mapContentTypeToResultSet.get(contentType) ; }
     private static final Map<String, ResultsFormat> mapContentTypeToResultSet = new HashMap<>() ;
-    static
-    {
+    static {
         mapContentTypeToResultSet.put(WebContent.contentTypeResultsXML, ResultsFormat.FMT_RS_XML) ;
         mapContentTypeToResultSet.put(WebContent.contentTypeResultsJSON, ResultsFormat.FMT_RS_JSON) ;
         mapContentTypeToResultSet.put(WebContent.contentTypeTextTSV, ResultsFormat.FMT_RS_TSV) ;
@@ -144,23 +143,22 @@ public class HttpResponseLib
 
     /** Response handling for SPARQL result sets. */
     public static class HttpCaptureResponseResultSet implements HttpCaptureResponse<ResultSet>
-    {    
-        ResultSet rs = null ;
+    {
+        private ResultSet rs = null;
+
         @Override
-        public void handle(String baseIRI , HttpResponse response ) throws IOException
-        {
-            String ct = contentType(response) ;
-            ResultsFormat fmt = mapContentTypeToResultSet.get(ct) ;
-            InputStream in = response.getEntity().getContent() ;
-            rs = ResultSetFactory.load(in, fmt) ;
+        public void handle(String baseIRI, HttpResponse response) throws IOException {
+            String ct = contentType(response);
+            ResultsFormat fmt = contentTypeToResultsFormat(ct);
+            InputStream in = response.getEntity().getContent();
+            rs = ResultSetFactory.load(in, fmt);
             // Force reading
-            rs = ResultSetFactory.copyResults(rs) ;
+            rs = ResultSetFactory.copyResults(rs);
         }
 
         @Override
-        public ResultSet get()
-        {
-            return rs ;
+        public ResultSet get() {
+            return rs;
         }
     }
     


[07/24] jena git commit: Interceptor wrapper for RDFConnection

Posted by an...@apache.org.
Interceptor wrapper for RDFConnection


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/09863b41
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/09863b41
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/09863b41

Branch: refs/heads/master
Commit: 09863b41c2a7cb94d3e507e5f55f8b91b2a64133
Parents: 8aad1f6
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 14:02:38 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 14:02:38 2018 +0000

----------------------------------------------------------------------
 .../rdfconnection/RDFConnectionWrapper.java     | 180 +++++++++++++++++++
 1 file changed, 180 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/09863b41/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionWrapper.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionWrapper.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionWrapper.java
new file mode 100644
index 0000000..fd73740
--- /dev/null
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionWrapper.java
@@ -0,0 +1,180 @@
+/*
+ * 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.rdfconnection;
+
+import org.apache.jena.query.*;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.update.UpdateRequest;
+
+/** Wrapper for an {@link RDFConnection}. */ 
+public class RDFConnectionWrapper implements RDFConnection {
+    private final RDFConnection other ;
+    protected RDFConnection get() { return other; }
+    public RDFConnectionWrapper(RDFConnection other) { this.other = other; }
+    
+    @Override
+    public Model fetch() {
+        return get().fetch();
+    }
+
+    @Override
+    public Model fetch(String graphName) {
+        return get().fetch(graphName);
+    }
+
+    @Override
+    public Dataset fetchDataset() {
+        return get().fetchDataset();
+    }
+
+    @Override
+    public QueryExecution query(Query query) {
+        return get().query(query);
+    }
+
+    @Override
+    public void update(UpdateRequest update) {
+        get().update(update);
+    }
+
+    @Override
+    public void load(String graphName, String file) {
+        get().load(graphName, file);
+    }
+
+    @Override
+    public void load(String file) {
+        get().load(file);
+    }
+
+    @Override
+    public void load(String graphName, Model model) {
+        get().load(graphName, model);
+    }
+
+    @Override
+    public void load(Model model) {
+        get().load(model);
+    }
+
+    @Override
+    public void put(String graphName, String file) {
+        get().put(graphName, file);
+    }
+
+    @Override
+    public void put(String file) {
+        get().put(file);
+    }
+
+    @Override
+    public void put(String graphName, Model model) {
+        get().put(graphName, model);
+    }
+
+    @Override
+    public void put(Model model) {
+        get().put(model);
+    }
+
+    @Override
+    public void delete(String graphName) {
+        get().delete(graphName);
+    }
+
+    @Override
+    public void delete() {
+        get().delete();
+    }
+
+    @Override
+    public void loadDataset(String file) {
+        get().loadDataset(file);
+    }
+
+    @Override
+    public void loadDataset(Dataset dataset) {
+        get().loadDataset(dataset);
+    }
+
+    @Override
+    public void putDataset(String file) {
+        get().putDataset(file);
+    }
+
+    @Override
+    public void putDataset(Dataset dataset) {
+        get().putDataset(dataset);
+    }
+
+    @Override
+    public boolean isClosed() {
+        return get().isClosed();
+    }
+
+    @Override
+    public void close() {
+        get().close();
+    }
+    
+    @Override
+    public void begin(TxnType type) {
+        get().begin(type);
+    }
+
+    @Override
+    public void begin(ReadWrite readWrite) {
+        get().begin(readWrite);
+    }
+
+    @Override
+    public boolean promote(Promote mode) {
+        return get().promote(mode);
+    }
+
+    @Override
+    public void commit() {
+        get().commit();
+    }
+
+    @Override
+    public void abort() {
+        get().abort();
+    }
+
+    @Override
+    public void end() {
+        get().end();
+    }
+
+    @Override
+    public ReadWrite transactionMode() {
+        return get().transactionMode();
+    }
+
+    @Override
+    public TxnType transactionType() {
+        return get().transactionType();
+    }
+
+    @Override
+    public boolean isInTransaction() {
+        return get().isInTransaction();
+    }
+}


[15/24] jena git commit: JENA-1494: ARP property setting from RIOT.

Posted by an...@apache.org.
JENA-1494: ARP property setting from RIOT.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/9237d24a
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/9237d24a
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/9237d24a

Branch: refs/heads/master
Commit: 9237d24a99617c219ce9d4a4ab085621be3784d4
Parents: 6550960
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 08:25:02 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 08:25:02 2018 +0000

----------------------------------------------------------------------
 .../org/apache/jena/riot/RDFWriterBuilder.java  | 33 ++++++++++-
 .../main/java/org/apache/jena/riot/SysRIOT.java | 15 +++--
 .../apache/jena/riot/lang/ReaderRIOTRDFXML.java | 60 ++++++++++++++++++--
 .../apache/jena/rdfxml/xmlinput/JenaReader.java |  3 +-
 4 files changed, 98 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/9237d24a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterBuilder.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterBuilder.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterBuilder.java
index 336e629..ab7aea4 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterBuilder.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterBuilder.java
@@ -21,6 +21,8 @@ package org.apache.jena.riot;
 import java.io.OutputStream;
 
 import org.apache.jena.graph.Graph ;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.rdf.model.Model;
 import org.apache.jena.sparql.core.DatasetGraph ;
 import org.apache.jena.sparql.util.Context;
 
@@ -40,7 +42,7 @@ public class RDFWriterBuilder {
 
     /** Set the source of writing to the graph argument.
      * <p>
-     * Any prrvious source setting is cleared.
+     * Any previous source setting is cleared.
      * @param graph A {@link Graph}.
      * @return this
      */
@@ -50,9 +52,22 @@ public class RDFWriterBuilder {
         return this;
     }
 
+    /** Set the source of writing to the graph argument.
+     * <p>
+     * Any previous source setting is cleared.
+     * <p>
+     * Equivalent to {@code source(model.getGraph()(s)}
+     * 
+     * @param model A {@link Model}.
+     * @return this
+     */
+    public RDFWriterBuilder source(Model model) {
+        return source(model.getGraph());
+    }
+
     /** Set the source of writing to the {@code DatasetGraph} argument.
      * <p>
-     * Any prrvious source setting is cleared.
+     * Any previous source setting is cleared.
      * @param dataset A {@link DatasetGraph}.
      * @return this
      */
@@ -62,6 +77,20 @@ public class RDFWriterBuilder {
         return this;
     }
 
+    /** Set the source of writing to the {@code DatasetGraph} argument.
+     * <p>
+     * Any previous source setting is cleared.
+     * <p>
+     * Equivalent to {@code source(dataset.asDatasetGraph())}
+     * 
+     * @param dataset A {@link DatasetGraph}.
+     * @return this
+     */
+    public RDFWriterBuilder source(Dataset dataset) {
+        return source(dataset.asDatasetGraph());
+    }
+
+
 //    // Not implemented
 //    public RDFWriterBuilder labels(NodeToLabel nodeToLabel) { return this; }
 //    

http://git-wip-us.apache.org/repos/asf/jena/blob/9237d24a/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java b/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
index c46228e..49611d6 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
@@ -20,6 +20,7 @@ package org.apache.jena.riot;
 
 import org.apache.jena.atlas.lib.IRILib ;
 import org.apache.jena.riot.system.IRIResolver ;
+import org.apache.jena.sparql.util.Context;
 import org.apache.jena.sparql.util.Symbol ;
 import org.apache.jena.util.FileUtils ;
 import org.slf4j.Logger ;
@@ -50,14 +51,20 @@ public class SysRIOT
     /**
      * Context key for old style RDFWriter properties. The value of this in a
      * {@link Context} must be a {@code Map<String, Object>}. The entries of the
-     * map are used to set writer properties before the Jena legalacy
+     * map are used to set writer properties before the Jena legacy
      * {@link RDFWriter} is called. Only has any effect on RDF/XML and
      * RDF/XML-ABBREV.
      */
-
-    /** Context key for old style RDFWriter properties */ 
     public static final Symbol sysRdfWriterProperties      = Symbol.create(riotBase+"rdfWriter_properties") ;
-    
+
+    /**
+     * Context key for old style RDFReader properties. The value of this in a
+     * {@link Context} must be a {@code Map<String, Object>}. The entries of the
+     * map are used to set reader properties before the Jena legalacy
+     * {@link RDFWriter} is called. Only has any effect on RDF/XML,
+     */
+    public static final Symbol sysRdfReaderProperties      = Symbol.create(riotBase+"rdfReader_properties") ;
+
     /** @deprecated Use {@link #sysRdfWriterProperties} */
     @Deprecated
     public static final Symbol rdfWriterProperties      = sysRdfWriterProperties ;

http://git-wip-us.apache.org/repos/asf/jena/blob/9237d24a/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java b/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
index 4f151ed..3c18978 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/lang/ReaderRIOTRDFXML.java
@@ -21,9 +21,11 @@ package org.apache.jena.riot.lang;
 import java.io.IOException ;
 import java.io.InputStream ;
 import java.io.Reader ;
+import java.util.Map;
 
 import org.apache.jena.JenaRuntime;
 import org.apache.jena.atlas.lib.Pair ;
+import org.apache.jena.atlas.logging.Log;
 import org.apache.jena.atlas.web.ContentType;
 import org.apache.jena.datatypes.RDFDatatype ;
 import org.apache.jena.datatypes.TypeMapper ;
@@ -34,6 +36,7 @@ import org.apache.jena.rdf.model.RDFErrorHandler ;
 import org.apache.jena.rdfxml.xmlinput.* ;
 import org.apache.jena.rdfxml.xmlinput.impl.ARPSaxErrorHandler ;
 import org.apache.jena.riot.*;
+import org.apache.jena.riot.adapters.AdapterRDFWriter;
 import org.apache.jena.riot.checker.CheckerLiterals ;
 import org.apache.jena.riot.system.*;
 import org.apache.jena.sparql.util.Context;
@@ -44,7 +47,7 @@ import org.xml.sax.SAXParseException ;
  *
  * @see <a href="http://www.w3.org/TR/rdf-syntax-grammar/">http://www.w3.org/TR/rdf-syntax-grammar/</a>
  */
-public class ReaderRIOTRDFXML  implements ReaderRIOT
+public class ReaderRIOTRDFXML implements ReaderRIOT
 {
     public static class Factory implements ReaderRIOTFactory {
         @Override
@@ -63,7 +66,9 @@ public class ReaderRIOTRDFXML  implements ReaderRIOT
     private String xmlBase ;
     private String filename ;
     private StreamRDF sink ;
-    private ErrorHandler errorHandler; 
+    private ErrorHandler errorHandler;
+
+    private Context context; 
     
     public ReaderRIOTRDFXML(ErrorHandler errorHandler) {
         this.errorHandler = errorHandler; 
@@ -75,6 +80,7 @@ public class ReaderRIOTRDFXML  implements ReaderRIOT
         this.xmlBase = baseURI_RDFXML(baseURI) ;
         this.filename = baseURI ;
         this.sink = output ;
+        this.context = context;
         parse();
     }
 
@@ -84,6 +90,7 @@ public class ReaderRIOTRDFXML  implements ReaderRIOT
         this.xmlBase = baseURI_RDFXML(baseURI) ;
         this.filename = baseURI ;
         this.sink = output ;
+        this.context = context;
         parse();
     }
     
@@ -107,6 +114,37 @@ public class ReaderRIOTRDFXML  implements ReaderRIOT
     // It's a pragmatic compromise.
     private static boolean errorForSpaceInURI = true;
     
+    // Extracted from org.apache.jena.rdfxml.xmlinput.JenaReader
+    private void oneProperty(ARPOptions options, String pName, Object value) {
+        if (! pName.startsWith("ERR_") && ! pName.startsWith("IGN_") && ! pName.startsWith("WARN_"))
+            return ;
+        int cond = ParseException.errorCode(pName);
+        if (cond == -1)
+            throw new RiotException("No such ARP property: '"+pName+"'");
+        int val;
+        if (value instanceof String) {
+            if (!((String) value).startsWith("EM_"))
+                throw new RiotException("Value for ARP property does not start EM_: '"+pName+"' = '"+value+"'" );
+            val = ParseException.errorCode((String) value);
+            if (val == -1 )
+                throw new RiotException("Illegal value for ARP property: '"+pName+"' = '"+value+"'" );
+        } else if (value instanceof Integer) {
+            val = ((Integer) value).intValue();
+            switch (val) {
+                case ARPErrorNumbers.EM_IGNORE:
+                case ARPErrorNumbers.EM_WARNING:
+                case ARPErrorNumbers.EM_ERROR:
+                case ARPErrorNumbers.EM_FATAL:
+                    break;
+                default:
+                    throw new RiotException("Illegal value for ARP property: '"+pName+"' = '"+value+"'" );
+            }
+        } else {
+            throw new RiotException("Property \"" + pName + "\" cannot have value: " + value.toString());
+        }
+        options.setErrorMode(cond, val);
+    }
+    
     public void parse() {
         // Hacked out of ARP because of all the "private" methods
         // JenaReader has reset the options since new ARP() was called.
@@ -116,17 +154,29 @@ public class ReaderRIOTRDFXML  implements ReaderRIOT
         arp.getHandlers().setErrorHandler(rslt) ;
         arp.getHandlers().setNamespaceHandler(rslt) ;
 
+        // ARPOptions.
+        ARPOptions arpOptions = arp.getOptions() ;
         if ( RiotUniformCompatibility ) {
-            ARPOptions options = arp.getOptions() ;
             // Convert some warnings to errors for compatible behaviour for all parsers.
             for ( int code : additionalErrors )
-                options.setErrorMode(code, ARPErrorNumbers.EM_ERROR) ;
-            arp.setOptionsWith(options) ;
+                arpOptions.setErrorMode(code, ARPErrorNumbers.EM_ERROR) ;
         }
         
         if ( JenaRuntime.isRDF11 )
             arp.getOptions().setIRIFactory(IRIResolver.iriFactory());
 
+        if ( context != null ) {
+            try { 
+                @SuppressWarnings("unchecked")
+                Map<String, Object> p = (Map<String, Object>)(context.get(SysRIOT.sysRdfReaderProperties)) ;
+                if ( p != null )
+                    p.forEach((k,v) -> oneProperty(arpOptions, k, v)) ;
+            } catch (Throwable ex) {
+                Log.warn(AdapterRDFWriter.class, "Problem setting properties", ex);
+            }
+        }
+        arp.setOptionsWith(arpOptions) ;
+        
         try {
             if ( reader != null )
                 arp.load(reader, xmlBase) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/9237d24a/jena-core/src/main/java/org/apache/jena/rdfxml/xmlinput/JenaReader.java
----------------------------------------------------------------------
diff --git a/jena-core/src/main/java/org/apache/jena/rdfxml/xmlinput/JenaReader.java b/jena-core/src/main/java/org/apache/jena/rdfxml/xmlinput/JenaReader.java
index 3f1c6f0..1c5c1b8 100644
--- a/jena-core/src/main/java/org/apache/jena/rdfxml/xmlinput/JenaReader.java
+++ b/jena-core/src/main/java/org/apache/jena/rdfxml/xmlinput/JenaReader.java
@@ -498,8 +498,7 @@ public class JenaReader implements RDFReader, ARPErrorNumbers {
             return old;
 
         }
-        if (str.startsWith("ERR_") || str.startsWith("IGN_")
-                || str.startsWith("WARN_")) {
+        if (str.startsWith("ERR_") || str.startsWith("IGN_") || str.startsWith("WARN_")) {
             int cond = ParseException.errorCode(str);
             if (cond == -1) {
                 // error, see end of function.


[23/24] jena git commit: Merge commit 'refs/pull/366/head' of https://github.com/apache/jena

Posted by an...@apache.org.
Merge commit 'refs/pull/366/head' of https://github.com/apache/jena

This closes #366.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/42d3775e
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/42d3775e
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/42d3775e

Branch: refs/heads/master
Commit: 42d3775e65a29a3579d77e87efce6002ab846f2a
Parents: f5d3e38 88de3eb
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 27 22:53:07 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 27 22:53:07 2018 +0000

----------------------------------------------------------------------
 .../arq/examples/riot/ExRIOT_5.java             | 103 +++++++++----------
 .../riot/ExRIOT_RDFXML_ReaderProperties.java    |  71 +++++++++++++
 .../riot/ExRIOT_RDFXML_WriteProperties.java     |  81 +++++++++++++++
 .../java/org/apache/jena/riot/RDFParser.java    |  20 +++-
 .../org/apache/jena/riot/RDFParserBuilder.java  |  25 +++++
 .../java/org/apache/jena/riot/RDFWriter.java    |   8 +-
 .../org/apache/jena/riot/RDFWriterBuilder.java  |  33 +++++-
 .../main/java/org/apache/jena/riot/SysRIOT.java |  16 ++-
 .../jena/riot/adapters/AdapterRDFWriter.java    |  10 +-
 .../apache/jena/riot/lang/ReaderRIOTRDFXML.java |  66 ++++++++++--
 .../apache/jena/rdfxml/xmlinput/JenaReader.java |   3 +-
 11 files changed, 361 insertions(+), 75 deletions(-)
----------------------------------------------------------------------



[20/24] jena git commit: Fix comment; add class javadoc

Posted by an...@apache.org.
Fix comment; add class javadoc


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/96d5ca0d
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/96d5ca0d
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/96d5ca0d

Branch: refs/heads/master
Commit: 96d5ca0d9f2c7761b7968ed15c27e2bc21782929
Parents: 27c4e47
Author: Andy Seaborne <an...@apache.org>
Authored: Mon Feb 26 15:54:11 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Mon Feb 26 15:54:11 2018 +0000

----------------------------------------------------------------------
 .../jena/rdfconnection/RDFConnectionRemoteBuilder.java  | 12 +++---------
 1 file changed, 3 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/96d5ca0d/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
index 617290b..5785e31 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
@@ -30,6 +30,7 @@ import org.apache.jena.sparql.core.Transactional;
 import org.apache.jena.sparql.core.TransactionalLock;
 import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
 
+/** Builder class for {@link RDFConnectionRemote} */
 public class RDFConnectionRemoteBuilder {
     /*package*/ static String SameAsDestination  = "";
 
@@ -296,20 +297,13 @@ public class RDFConnectionRemoteBuilder {
             maker = (b)->b.buildConnection();
         
         // Sort out service URLs.
-        // Delay until here because the order of destination and service settign sisnot
-        // defined and if being modifed, may be the wrong order to calculate as queryEndpoint() is called.
+        // Delay until here. The builder may be setting destination and service endpoint
+        // names. We can't calculate the full URL until build() is called.
         
         queryURL = LibRDFConn.formServiceURL(destination, sQuery);
         updateURL = LibRDFConn.formServiceURL(destination, sUpdate);
         gspURL = LibRDFConn.formServiceURL(destination, sGSP);
         
-//        if ( acceptSparqlResults == null ) {
-//            if ( acceptSelectResult != acceptAskResult )
-//                acceptSparqlResults = String.join(",", acceptSelectResult, acceptAskResult, acceptGraphResult);
-//            else
-//                acceptSparqlResults = String.join(",", acceptSelectResult, acceptGraphResult);
-//        }
-//        
         return maker.apply(this);
     }
     


[05/24] jena git commit: Better version output.

Posted by an...@apache.org.
Better version output.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/036491f7
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/036491f7
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/036491f7

Branch: refs/heads/master
Commit: 036491f75dca0ae28b92f99552e354cb03d587c1
Parents: fc3c3c2
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 13:55:25 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 13:55:25 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/fuseki/servlets/SPARQL_Query.java   | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/036491f7/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_Query.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_Query.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_Query.java
index be465ef..278b904 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_Query.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/SPARQL_Query.java
@@ -237,8 +237,12 @@ public abstract class SPARQL_Query extends SPARQL_Protocol
 
     protected void execute(String queryString, HttpAction action) {
         String queryStringLog = ServletOps.formatForLog(queryString) ;
-        if ( action.verbose )
-            action.log.info(format("[%d] Query = \n%s", action.id, queryString)) ;
+        if ( action.verbose ) {
+            String str = queryString;
+            if ( str.endsWith("\n") )
+                str = str.substring(0, str.length()-1);
+            action.log.info(format("[%d] Query = \n%s", action.id, str)) ;
+        }
         else
             action.log.info(format("[%d] Query = %s", action.id, queryStringLog)) ;
 


[11/24] jena git commit: Direct setting of the "Accept" header.

Posted by an...@apache.org.
Direct setting of the "Accept" header.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/da1ccbc8
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/da1ccbc8
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/da1ccbc8

Branch: refs/heads/master
Commit: da1ccbc887fbcf75adad337b7727f6d99cc64640
Parents: f482aea
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 19:21:17 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 20:20:10 2018 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/jena/fuseki/DEF.java   |  2 +-
 .../jena/rdfconnection/RDFConnectionFuseki.java | 51 +++++++-------------
 .../jena/rdfconnection/RDFConnectionRemote.java | 46 ++++++++----------
 .../RDFConnectionRemoteBuilder.java             | 33 ++++++-------
 4 files changed, 54 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/da1ccbc8/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/DEF.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/DEF.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/DEF.java
index 71eaf08..05e9bf9 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/DEF.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/DEF.java
@@ -89,7 +89,7 @@ public class DEF
                                                                           ) ;
          
     // Offer for ASK
-    // This include application/xml and application/json.
+    // This includes application/xml and application/json and excludes application/sparql-results+thrift 
     public static final AcceptList rsOfferBoolean      = AcceptList.create(contentTypeResultsJSON,
                                                                            contentTypeJSON,
                                                                            contentTypeTextCSV,

http://git-wip-us.apache.org/repos/asf/jena/blob/da1ccbc8/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
index 425450c..ece1987 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
@@ -26,11 +26,8 @@ import org.apache.jena.graph.Graph;
 import org.apache.jena.graph.Node;
 import org.apache.jena.graph.Triple;
 import org.apache.jena.query.Dataset;
-import org.apache.jena.query.Query;
 import org.apache.jena.query.QueryExecution;
-import org.apache.jena.query.QueryFactory;
 import org.apache.jena.rdf.model.Model;
-import org.apache.jena.rdfconnection.RDFConnection;
 import org.apache.jena.riot.Lang;
 import org.apache.jena.riot.RDFFormat;
 import org.apache.jena.riot.WebContent;
@@ -73,6 +70,10 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
     /** Fuseki settings */
     private static RDFConnectionRemoteBuilder setupForFuseki(RDFConnectionRemoteBuilder builder) {
         String ctRDFThrift = Lang.RDFTHRIFT.getContentType().getContentType();
+        String acceptHeaderSPARQL = String.join("," 
+                            , ResultSetLang.SPARQLResultSetThrift.getHeaderString()
+                            , ResultSetLang.SPARQLResultSetJSON.getHeaderString()+";q=0.9"
+                            , Lang.RDFTHRIFT.getHeaderString());
         return 
             builder
                 .quadsFormat(RDFFormat.RDF_THRIFT)
@@ -81,7 +82,7 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
                 .acceptHeaderDataset(ctRDFThrift)
                 .acceptHeaderSelectQuery(ResultSetLang.SPARQLResultSetThrift.getHeaderString())
                 .acceptHeaderAskQuery(ResultSetLang.SPARQLResultSetJSON.getHeaderString())
-                .acceptHeaderGraphQuery(ResultSetLang.SPARQLResultSetThrift.getHeaderString())
+                .acceptHeaderQuery(acceptHeaderSPARQL)
                 // Create object of this class.
                 .creator((b)->fusekiMaker(b));
     }
@@ -99,18 +100,18 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
             base.destination, base.queryURL, base.updateURL, base.gspURL,
             base.outputQuads, base.outputTriples,
             base.acceptDataset, base.acceptGraph,
-            base.acceptSelectResult, base.acceptAskResult, base.acceptGraphResult);
+            base.acceptSparqlResults, base.acceptSelectResult, base.acceptAskResult);
     }
     
     protected RDFConnectionFuseki(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
                                   String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
-                                  String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
-                                  String acceptGraphResult) {
+                                  String acceptDataset, String acceptGraph, 
+                                  String acceptSparqlResults, String acceptSelectResult, String acceptAskResult) {
         super(txnLifecycle, httpClient, httpContext, 
               destination, queryURL, updateURL, gspURL,
               outputQuads, outputTriples, 
               acceptDataset, acceptGraph,
-              acceptSelectResult, acceptAskResult, acceptGraphResult);
+              acceptSparqlResults, acceptSelectResult, acceptAskResult);
     }
     
     // Fuseki specific operations.
@@ -133,41 +134,25 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
 //        }
 //    }
     
-    // Make sure all querygoes through query(String) or query(Query) 
+    // Make sure all query goes through query(String) or query(Query) 
     
     @Override
     public QueryExecution query(String queryString) {
         checkQuery();
-        
-        Query queryLocal = QueryFactory.create(queryString);
-        // XXX Kludge until QueryEngineHTTP.setAccept.
-        // XXX Accept header builder.
-        String acceptHeader = acceptSelectResult+","+acceptAskResult+";q=0.9,"+acceptGraphResult;
         return exec(()-> {
             QueryExecution qExec = new QueryEngineHTTP(svcQuery, queryString, httpClient, httpContext);
             QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
-            // XXX qEngine.setAccept(acceptHeader);
-            // Only one choice, not "Accept:"
-            switch ( queryLocal.getQueryType() ) {
-                case Query.QueryTypeSelect:
-                    qEngine.setSelectContentType(acceptSelectResult);
-                    break;
-                case Query.QueryTypeAsk:
-                    qEngine.setAskContentType(acceptAskResult);
-                    break;
-                case Query.QueryTypeDescribe:
-                case Query.QueryTypeConstruct:
-                    qEngine.setModelContentType(acceptGraphResult);
-                    break;
+            // We do not know the kind of query unless we parse it locally.
+            if ( acceptSparqlResults != null )
+                qEngine.setAcceptHeader(super.acceptSparqlResults);
+            else {
+                qEngine.setSelectContentType(acceptSelectResult);
+                qEngine.setAskContentType(acceptAskResult);
+                qEngine.setModelContentType(acceptGraph);
+                qEngine.setDatasetContentType(acceptDataset);
             }
             return qEngine ;
         });
-//        // XXX Better!
-//        String url = svcQuery+"?query="+queryString;
-//        // XXX Better accept.
-//        TypedInputStream in =  exec(()->HttpOp.execHttpGet(url, acceptSelectResult, this.httpClient,this.httpContext));
-//        QueryExecution qExec = 
-//        return qExec;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/jena/blob/da1ccbc8/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
index cf17c49..d0a797a 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
@@ -77,9 +77,9 @@ public class RDFConnectionRemote implements RDFConnection {
     protected final RDFFormat outputTriples;
     protected final String acceptGraph;
     protected final String acceptDataset;
+    protected final String acceptSparqlResults;
     protected final String acceptSelectResult;
     protected final String acceptAskResult;
-    protected final String acceptGraphResult;
     
     /** Create a {@link RDFConnectionRemoteBuilder}. */
     public static RDFConnectionRemoteBuilder create() {
@@ -159,15 +159,16 @@ public class RDFConnectionRemote implements RDFConnection {
         this(null, httpClient, null, destination, sQuery, sUpdate, sGSP,
             RDFFormat.NQUADS, RDFFormat.NTRIPLES,
             WebContent.defaultGraphAcceptHeader, WebContent.defaultDatasetAcceptHeader,
-            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader(), QueryEngineHTTP.defaultConstructHeader()
-            );
+            null,
+            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader());
     }
 
     // Used by the builder.
     protected RDFConnectionRemote(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
                                    String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
-                                   String acceptDataset, String acceptGraph, String acceptSelectResult, String acceptAskResult,
-                                   String acceptGraphResult) {
+                                   String acceptDataset, String acceptGraph,
+                                   String acceptSparqlResults,
+                                   String acceptSelectResult, String acceptAskResult) {
         this.httpClient = httpClient;
         this.httpContext = httpContext;
         this.destination = destination;
@@ -181,9 +182,9 @@ public class RDFConnectionRemote implements RDFConnection {
         this.outputTriples = outputTriples;
         this.acceptDataset = acceptDataset;
         this.acceptGraph = acceptGraph;
+        this.acceptSparqlResults = acceptSparqlResults;
         this.acceptSelectResult = acceptSelectResult;
         this.acceptAskResult = acceptAskResult;
-        this.acceptGraphResult = acceptGraphResult;
     }
 
     /** Return the {@link HttpClient} in-use. */ 
@@ -204,32 +205,23 @@ public class RDFConnectionRemote implements RDFConnection {
     @Override
     public QueryExecution query(Query query) {
         checkQuery();
-        // XXX ResultSetFormat
-        
-        
-        
         return exec(()-> {
             QueryExecution qExec = QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext);
             QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
-            
-            // Set general HTTP header.
-            
-//            // Only one choice, not "Accept:"
-//            switch ( query.getQueryType() ) {
-//                case Query.QueryTypeSelect:
-//                    qEngine.setSelectContentType("");
-//                    break;
-//                case Query.QueryTypeAsk:
-//                    qEngine.setAskContentType("");
-//                    break;
-//                case Query.QueryTypeDescribe:
-//                case Query.QueryTypeConstruct:
-//                    qEngine.setModelContentType("");
-//                    break;
-//            }
+            if ( acceptSparqlResults != null )
+                qEngine.setAcceptHeader(acceptSparqlResults);
+            else {
+                if ( query.isSelectType() && acceptSelectResult != null )
+                    qEngine.setAcceptHeader(acceptSelectResult);
+                if ( query.isAskType() && acceptAskResult != null )
+                    qEngine.setAcceptHeader(acceptAskResult);
+                if ( ( query.isConstructType() || query.isDescribeType() ) && acceptGraph != null )
+                    qEngine.setAcceptHeader(acceptGraph);
+                if ( query.isConstructQuad() )
+                    qEngine.setDatasetContentType(acceptDataset);
+            }
             return qExec ;
         });
-        
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/jena/blob/da1ccbc8/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
index e77e738..3b5de7f 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
@@ -25,7 +25,6 @@ import java.util.function.Function;
 
 import org.apache.http.client.HttpClient;
 import org.apache.http.protocol.HttpContext;
-import org.apache.jena.rdfconnection.RDFConnection;
 import org.apache.jena.riot.*;
 import org.apache.jena.sparql.core.Transactional;
 import org.apache.jena.sparql.core.TransactionalLock;
@@ -56,7 +55,8 @@ public class RDFConnectionRemoteBuilder {
     
     protected String        acceptSelectResult = QueryEngineHTTP.defaultSelectHeader();
     protected String        acceptAskResult    = QueryEngineHTTP.defaultAskHeader();
-    protected String        acceptGraphResult  = QueryEngineHTTP.defaultConstructHeader();
+    // All-purpose head that works for any query type (but is quite long!)
+    protected String        acceptSparqlResults = null;
 
     RDFConnectionRemoteBuilder() { 
         // Default settings are the meber declarations.
@@ -81,7 +81,6 @@ public class RDFConnectionRemoteBuilder {
         
         acceptSelectResult  = base.acceptSelectResult;
         acceptAskResult     = base.acceptAskResult;
-        acceptGraphResult   = base.acceptGraphResult;
     }
     
     /** URL of the remote SPARQL endpoint.
@@ -235,7 +234,7 @@ public class RDFConnectionRemoteBuilder {
         this.acceptGraph = acceptGraph;
         return this;
     }
-
+    
     /** Set the HTTP {@code Accept:} header used to fetch RDF datasets using HTTP GET operations. */ 
     public RDFConnectionRemoteBuilder acceptHeaderDataset(String acceptDataset) {
         this.acceptDataset = acceptDataset;
@@ -254,16 +253,11 @@ public class RDFConnectionRemoteBuilder {
         return this;
     }
 
-    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol CONSTRUCT or DESCRIBE query. */ 
-    public RDFConnectionRemoteBuilder acceptHeaderGraphQuery(String acceptGraphResultHeader) {
-        this.acceptGraphResult = acceptGraphResultHeader;
-        return this;
-    }
-
-    // XXX Alternative:
-    /** Set the HTTP {@code Accept:} header used to when making a SPARQL Protocol query. */ 
-    public RDFConnectionRemoteBuilder acceptHeaderQuery(String acceptHeaderQuery) {
-        System.err.println("NOT YET IMPLEMENTED");
+    /** Set the HTTP {@code Accept:} header used to when making a 
+     * SPARQL Protocol query if no query type specific setting available.
+     */ 
+    public RDFConnectionRemoteBuilder acceptHeaderQuery(String acceptHeader) {
+        this.acceptSparqlResults = acceptHeader;
         return this;
     }
 
@@ -295,8 +289,13 @@ public class RDFConnectionRemoteBuilder {
         updateURL = LibRDFConn.formServiceURL(destination, sUpdate);
         gspURL = LibRDFConn.formServiceURL(destination, sGSP);
         
-        // XXX Sort out SPARQL Accept headers.
-        
+//        if ( acceptSparqlResults == null ) {
+//            if ( acceptSelectResult != acceptAskResult )
+//                acceptSparqlResults = String.join(",", acceptSelectResult, acceptAskResult, acceptGraphResult);
+//            else
+//                acceptSparqlResults = String.join(",", acceptSelectResult, acceptGraphResult);
+//        }
+//        
         return maker.apply(this);
     }
     
@@ -305,6 +304,6 @@ public class RDFConnectionRemoteBuilder {
                                         destination, queryURL, updateURL, gspURL,
                                         outputQuads, outputTriples,
                                         acceptDataset, acceptGraph,
-                                        acceptSelectResult, acceptAskResult, acceptGraphResult);
+                                        acceptSparqlResults, acceptSelectResult, acceptAskResult);
     }
 }
\ No newline at end of file


[16/24] jena git commit: Examples RDF/XML handling.

Posted by an...@apache.org.
Examples RDF/XML handling.

JENA-1168, JENA-1494.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/e9eb97fd
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/e9eb97fd
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/e9eb97fd

Branch: refs/heads/master
Commit: e9eb97fdfe7f9e3e738ff5fbac2f53ddee5c15ad
Parents: 9237d24
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 08:25:51 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 08:25:51 2018 +0000

----------------------------------------------------------------------
 .../riot/ExRIOT_RDFXML_ReaderProperties.java    | 71 +++++++++++++++++
 .../riot/ExRIOT_RDFXML_WriteProperties.java     | 81 ++++++++++++++++++++
 2 files changed, 152 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/e9eb97fd/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_ReaderProperties.java
----------------------------------------------------------------------
diff --git a/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_ReaderProperties.java b/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_ReaderProperties.java
new file mode 100644
index 0000000..1ba3245
--- /dev/null
+++ b/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_ReaderProperties.java
@@ -0,0 +1,71 @@
+/*
+ * 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 arq.examples.riot;
+
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.jena.atlas.lib.StrUtils;
+import org.apache.jena.atlas.logging.LogCtl;
+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.RDFParser;
+import org.apache.jena.riot.SysRIOT;
+import org.apache.jena.sparql.util.Context;
+
+/** Set proeprties of the RDF/XML parser (ARP) */
+public class ExRIOT_RDFXML_ReaderProperties {
+    static { LogCtl.setCmdLogging(); }
+    
+    public static void main(String[] args) {
+        // Inline illustrative data.
+        String data = StrUtils.strjoinNL
+            ("<?xml version=\"1.0\" encoding=\"utf-8\"?>"
+            ,"<rdf:RDF xmlns:rdf=\"http://www.w3.org/1999/02/22-rdf-syntax-ns#\""
+            ,"         xmlns:ex=\"http://examples.org/\">"
+            // This rdf:ID startswith a digit which normal causes a warning.
+            ,"  <ex:Type rdf:ID='012345'></ex:Type>"
+            ,"</rdf:RDF>"
+            );
+        System.out.println(data);
+        System.out.println();
+        // Properties to be set.
+        // This is a map propertyName->value 
+        Map<String, Object> properties = new HashMap<>();
+        // See class ARPErrorNumbers for the possible ARP properies.
+        properties.put("WARN_BAD_NAME", "EM_IGNORE");
+        
+        // Put a properties object into the Context.
+        Context cxt = new Context();
+        cxt.set(SysRIOT.sysRdfReaderProperties, properties);
+        
+        Model model = ModelFactory.createDefaultModel();
+        // Build and run a parser
+        RDFParser.create()
+            .lang(Lang.RDFXML)
+            .source(new StringReader(data))
+            .context(cxt)
+            .parse(model);
+        System.out.println("== Parsed data output in Turtle");
+        RDFDataMgr.write(System.out,  model, Lang.TURTLE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/e9eb97fd/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_WriteProperties.java
----------------------------------------------------------------------
diff --git a/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_WriteProperties.java b/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_WriteProperties.java
new file mode 100644
index 0000000..5e6506a
--- /dev/null
+++ b/jena-arq/src-examples/arq/examples/riot/ExRIOT_RDFXML_WriteProperties.java
@@ -0,0 +1,81 @@
+/*
+ * or more contributor license agreements.  See the NOTICE fil
+ * 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 arq.examples.riot;
+
+import java.io.StringReader;
+import java.util.HashMap ;
+import java.util.Map ;
+
+import org.apache.jena.atlas.lib.StrUtils;
+import org.apache.jena.atlas.logging.LogCtl;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.riot.*;
+import org.apache.jena.sparql.util.Context;
+
+/** Example of setting properties for RDF/XML writer via RIOT */
+public class ExRIOT_RDFXML_WriteProperties {
+    static { LogCtl.setCmdLogging(); }
+
+    public static void main(String... args) {
+        // Data.
+        String x = StrUtils.strjoinNL
+            ("PREFIX : <http://example.org/>"
+            ,":s :p :o ."
+            );
+        Model model = ModelFactory.createDefaultModel();
+        RDFDataMgr.read(model, new StringReader(x), null, Lang.TURTLE);
+        
+        // Write, default settings.
+        writePlain(model);
+        System.out.println();
+
+        // Write, with properties 
+        writeProperties(model);
+    }
+
+    /** Write in plain, not pretty ("abbrev") format. */
+    private static void writePlain(Model model) {
+        System.out.println("**** RDFXML_PLAIN");
+        RDFDataMgr.write(System.out, model, RDFFormat.RDFXML_PLAIN);
+        System.out.println();
+    }
+
+    /** Write with properties */
+    public static void writeProperties(Model model) {
+        System.out.println("**** RDFXML_PLAIN+properties");
+        System.out.println("**** Adds XML declaration");
+
+        // Properties to be set.
+        // See https://jena.apache.org/documentation/io/rdfxml_howto.html#advanced-rdfxml-output
+        // for details of properties.
+        Map<String, Object> properties = new HashMap<>() ;
+        properties.put("showXmlDeclaration", "true");
+
+        // Put a properties object into the Context.
+        Context cxt = new Context();
+        cxt.set(SysRIOT.sysRdfWriterProperties, properties);
+
+        RDFWriter.create()
+            .base("http://example.org/")
+            .format(RDFFormat.RDFXML_ABBREV)
+            .context(cxt)
+            .source(model)
+            .output(System.out);
+    }
+}


[12/24] jena git commit: RDFConnectionFactory functions and tests for RDFConnectionFuseki

Posted by an...@apache.org.
RDFConnectionFactory functions and tests for RDFConnectionFuseki


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/6b2dcba8
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/6b2dcba8
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/6b2dcba8

Branch: refs/heads/master
Commit: 6b2dcba8a4b70295863a666af3eb56090abaafa0
Parents: da1ccbc
Author: Andy Seaborne <an...@apache.org>
Authored: Thu Feb 22 20:00:58 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Thu Feb 22 20:00:58 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/fuseki/FusekiLib.java  |  45 +++++
 .../test/rdfconnection/TS_RDFConnection2.java   |   3 +-
 .../test/rdfconnection/TestBlankNodeBinary.java | 192 +++++++++++++++++++
 .../rdfconnection/RDFConnectionFactory.java     |  50 ++++-
 4 files changed, 288 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/6b2dcba8/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/FusekiLib.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/FusekiLib.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/FusekiLib.java
index 823593a..dcfa590 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/FusekiLib.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/FusekiLib.java
@@ -24,7 +24,14 @@ import java.util.Iterator ;
 
 import javax.servlet.http.HttpServletRequest ;
 
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpOptions;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.protocol.HttpContext;
+import org.apache.jena.atlas.logging.Log;
 import org.apache.jena.atlas.web.ContentType ;
+import org.apache.jena.atlas.web.HttpException;
 import org.apache.jena.ext.com.google.common.collect.ArrayListMultimap;
 import org.apache.jena.ext.com.google.common.collect.Multimap;
 import org.apache.jena.fuseki.server.SystemState ;
@@ -37,8 +44,10 @@ import org.apache.jena.rdf.model.Literal ;
 import org.apache.jena.rdf.model.Model ;
 import org.apache.jena.rdf.model.RDFNode ;
 import org.apache.jena.rdf.model.Resource ;
+import org.apache.jena.rdfconnection.RDFConnectionRemote;
 import org.apache.jena.riot.Lang ;
 import org.apache.jena.riot.RDFLanguages ;
+import org.apache.jena.riot.web.HttpOp;
 import org.apache.jena.shared.PrefixMapping ;
 import org.apache.jena.sparql.core.DatasetGraph ;
 import org.apache.jena.sparql.core.Quad ;
@@ -265,4 +274,40 @@ public class FusekiLib {
             throw new FusekiException("Failed to find a port");
         }
     }
+
+    /** Test whether a URL identifies a Fuseki server */  
+    public static boolean isFuseki(String datasetURL) {
+        HttpOptions request = new HttpOptions(datasetURL);
+        HttpClient httpClient = HttpOp.getDefaultHttpClient();
+        if ( httpClient == null ) 
+            httpClient = HttpClients.createSystem();
+        return isFuseki(request, httpClient, null);
+    }
+
+    /** Test whether a {@link RDFConnectionRemote} connects to a Fuseki server */  
+    public static boolean isFuseki(RDFConnectionRemote connection) {
+        HttpOptions request = new HttpOptions(connection.getDestination());
+        HttpClient httpClient = connection.getHttpClient();
+        if ( httpClient == null ) 
+            httpClient = HttpClients.createSystem();
+        HttpContext httpContext = connection.getHttpContext();
+        return isFuseki(request, httpClient, httpContext);
+    }
+
+    private static boolean isFuseki(HttpOptions request, HttpClient httpClient, HttpContext httpContext) {
+        try {
+            HttpResponse response = httpClient.execute(request);
+            // Fuseki-Request-ID:
+            //String reqId = response.getFirstHeader("Fuseki-Request-ID").getValue();
+            // Server:
+            String serverIdent = response.getFirstHeader("Server").getValue();
+            Log.debug(ARQ.getHttpRequestLogger(), "Server: "+serverIdent);
+            boolean isFuseki = serverIdent.startsWith("Apache Jena Fuseki");
+            if ( !isFuseki )
+                isFuseki = serverIdent.toLowerCase().contains("fuseki");
+            return isFuseki; // Maybe
+        } catch (IOException ex) {
+            throw new HttpException("Failed to check for a Fuseki server", ex);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/6b2dcba8/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
index 55a0dff..ea37a51 100644
--- a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
@@ -29,7 +29,8 @@ import org.junit.runners.Suite ;
     
     // Addition tests added here.
     TestRDFConnectionLocalTDB.class ,
-    TestRDFConnectionRemote.class
+    TestRDFConnectionRemote.class,
+    TestBlankNodeBinary.class
 })
 
 public class TS_RDFConnection2 {}

http://git-wip-us.apache.org/repos/asf/jena/blob/6b2dcba8/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
new file mode 100644
index 0000000..a3eee4d
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
@@ -0,0 +1,192 @@
+/*
+ * 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.test.rdfconnection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+
+import org.apache.jena.fuseki.FusekiLib;
+import org.apache.jena.fuseki.embedded.FusekiServer;
+import org.apache.jena.graph.*;
+import org.apache.jena.query.Query;
+import org.apache.jena.query.QueryExecution;
+import org.apache.jena.query.QueryFactory;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.rdfconnection.RDFConnectionFuseki;
+import org.apache.jena.rdfconnection.RDFConnectionRemoteBuilder;
+import org.apache.jena.riot.Lang;
+import org.apache.jena.riot.RDFDataMgr;
+import org.apache.jena.sparql.algebra.Algebra;
+import org.apache.jena.sparql.algebra.Op;
+import org.apache.jena.sparql.algebra.op.OpBGP;
+import org.apache.jena.sparql.core.BasicPattern;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphFactory;
+import org.apache.jena.sparql.core.TriplePath;
+import org.apache.jena.sparql.modify.request.UpdateDataInsert;
+import org.apache.jena.sparql.sse.SSE;
+import org.apache.jena.sparql.syntax.Element;
+import org.apache.jena.sparql.syntax.ElementGroup;
+import org.apache.jena.sparql.syntax.ElementPathBlock;
+import org.apache.jena.update.Update;
+import org.apache.jena.update.UpdateFactory;
+import org.apache.jena.update.UpdateRequest;
+import org.junit.Test;
+
+/* Tests that blanknodes work over RDFConnectionFuseki
+ * This consists of testing each of the necessary components,
+ * and then a test of a connection itself.  
+ */
+
+public class TestBlankNodeBinary {
+    private static Node n(String str) { return SSE.parseNode(str) ; }
+    
+    // Check RDF Thrift round-trips blank nodes.
+    @Test public void binaryThrift() {
+        Triple t = Triple.create(n(":s"), n(":p"), NodeFactory.createBlankNode("ABCD"));
+        Node obj = t.getObject(); 
+        Graph graph = Factory.createDefaultGraph();
+        graph.add(t);
+        
+        ByteArrayOutputStream bout = new ByteArrayOutputStream();
+        RDFDataMgr.write(bout, graph, Lang.RDFTHRIFT);
+        
+        ByteArrayInputStream bin = new ByteArrayInputStream(bout.toByteArray());
+        Graph graph1 = Factory.createDefaultGraph();
+        RDFDataMgr.read(graph1, bin, Lang.RDFTHRIFT);
+        
+        Node obj1 = graph1.find().next().getObject();
+        assertEquals(obj, obj1);
+        assertTrue(obj1.isBlank());
+        assertEquals(obj.getBlankNodeLabel(), obj1.getBlankNodeLabel());  
+    }
+    
+    // Check SPARQL parsing.
+    @Test public void bNodeSPARQL_Query_1() {
+        String qs = "SELECT * { ?s ?p <_:ABC>}";
+        Query query = QueryFactory.create(qs);
+        Element el = ((ElementGroup)query.getQueryPattern()).get(0);
+        ElementPathBlock epb = (ElementPathBlock)el;
+        TriplePath tp = epb.getPattern().get(0);
+        Triple t = tp.asTriple();
+        assertEquals("ABC", t.getObject().getBlankNodeLabel());  
+    }
+    
+    @Test public void bNodeSPARQL_Query_2() {
+        String qs = "SELECT * { ?s ?p <_:BCD>}";
+        Query query = QueryFactory.create(qs);
+        Op op = Algebra.compile(query);
+        BasicPattern bp = ((OpBGP)op).getPattern();
+        Triple t = bp.get(0);
+        assertEquals("BCD", t.getObject().getBlankNodeLabel());  
+    }
+
+    @Test public void bNodeSPARQL_Update_1() {
+        String str = "INSERT DATA { <x:s> <x:p> <_:789> }";
+        UpdateRequest req = UpdateFactory.create(str);
+        Update update = req.getOperations().get(0);
+        UpdateDataInsert ins = (UpdateDataInsert)update;
+        Node obj = ins.getQuads().get(0).getObject();
+        assertEquals("789", obj.getBlankNodeLabel());
+    }
+    
+    // RDFConnection level testing. 
+
+    @Test public void rdfconnection_binary_1() {
+        // Tests on one connection.
+        Triple triple = SSE.parseTriple("(:s :p <_:b3456>)");
+        // Goes in as URI! (pre this PR)
+        Model model = ModelFactory.createDefaultModel();
+        model.getGraph().add(triple);
+        
+        int PORT = FusekiLib.choosePort();
+        FusekiServer server = createFusekiServer(PORT).build().start();
+        try {
+            String dsURL = "http://localhost:"+PORT+"/ds" ;
+            assertTrue(FusekiLib.isFuseki(dsURL)); 
+
+            RDFConnectionRemoteBuilder builder = RDFConnectionFuseki.create().destination(dsURL);
+
+            try (RDFConnectionFuseki conn = (RDFConnectionFuseki)builder.build()) {
+                assertTrue(FusekiLib.isFuseki(conn));
+                // GSP
+                conn.put(model);
+                checkModel(conn, "b3456");
+
+                // Query forms.
+                conn.querySelect("SELECT * {?s ?p ?o}", x-> {
+                    Node obj = x.get("o").asNode();
+                    assertTrue(obj.isBlank());
+                    assertEquals("b3456", obj.getBlankNodeLabel());
+                });
+
+                try(QueryExecution qExec = conn.query("ASK {?s ?p <_:b3456>}")){
+                    boolean bool = qExec.execAsk();
+                    assertTrue(bool);
+                }
+
+                try (QueryExecution qExec = conn.query("CONSTRUCT { ?s ?p ?o } WHERE { ?s ?p ?o . FILTER (sameTerm(?o, <_:b3456>)) }")){
+                    Model model2 = qExec.execConstruct() ;
+                    checkModel(model2, "b3456");
+                }
+
+                try(QueryExecution qExec = conn.query("DESCRIBE ?s WHERE { ?s ?p <_:b3456>}")){
+                    Model model2 = qExec.execConstruct() ;
+                    checkModel(model2, "b3456");
+                }
+
+                // Update
+                conn.update("CLEAR DEFAULT" );
+                conn.update("INSERT DATA { <x:s> <x:p> <_:b789> }" );
+                checkModel(conn, "b789");
+                conn.update("CLEAR DEFAULT" );
+                conn.update("INSERT DATA { <x:s> <x:p> <_:6789> }" );
+                checkModel(conn, "6789");
+            }
+        } finally { server.stop(); } 
+    }
+
+    private void checkModel(RDFConnectionFuseki conn, String label) {
+        Model model2 = conn.fetch();
+        checkModel(model2, label);
+    }
+
+    private void checkModel(Model model2, String label) {
+        assertEquals(1, model2.size());
+        Node n = model2.listStatements().next().getObject().asNode();
+        assertTrue(n.isBlank());
+        assertEquals(label, n.getBlankNodeLabel());
+    }
+
+    
+    private static FusekiServer.Builder createFusekiServer(int PORT) {
+        DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
+        return  
+            FusekiServer.create()
+                .setPort(PORT)
+                //.setStaticFileBase("/home/afs/ASF/jena-fuseki-cmds/sparqler")
+                .add("/ds", dsg)
+                //.setVerbose(true)
+                ;
+    }               
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/6b2dcba8/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
index ef6371c..3049812 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
@@ -34,7 +34,7 @@ public class RDFConnectionFactory {
      *  <li>SPARQL Graph Store Protocol : "data"
      *  </ul>
      *  These are the default names in <a href="http://jena.apache.org/documentation/fuseki2">Fuseki</a> 
-     *  Other names can be specificied using {@link #connect(String, String, String, String)}
+     *  Other names can be specified using {@link #connect(String, String, String, String)}
      *     
      * @param destination
      * @return RDFConnection
@@ -117,4 +117,52 @@ public class RDFConnectionFactory {
     public static RDFConnection connect(Dataset dataset, Isolation isolation) {
         return new RDFConnectionLocal(dataset, isolation);
     }
+
+    /** Create a connection to a remote Fuseki server by URL.
+     * This is the URL for the dataset.
+     * <p>
+     * A {@link RDFConnectionFuseki} is an {@link RDFConnection} that:
+     * <ul> 
+     * <li>provides round-trip of blank nodes between this application and the server
+     * <li>uses the more efficient <a href="http://jena.apache.org/documentation/io/rdf-binary.html">RDF Thrift binary</a> format. 
+     * </ul>
+     * 
+     *  This factory call assumes the names of services as:
+     *  <ul>
+     *  <li>SPARQL Query endpoint : "sparql"
+     *  <li>SPARQL Update endpoint : "update"
+     *  <li>SPARQL Graph Store Protocol : "data"
+     *  </ul>
+     *  These are the default names in <a href="http://jena.apache.org/documentation/fuseki2">Fuseki</a> 
+     *  Other names can be specified using {@link #connectFuseki(String, String, String, String)}.
+     *     
+     * @param destination
+     * @return RDFConnectionFuseki
+     */
+    public static RDFConnectionFuseki connectFuseki(String destination) {
+        return (RDFConnectionFuseki)RDFConnectionFuseki.create().destination(destination).build();
+    }
+
+    /** Create a connection to a remote Fuseki server by URL.
+         * This is the URL for the dataset.
+         * 
+         * Each service is then specified by a URL which is relative to the {@code datasetURL}.
+         * 
+         * @param datasetURL
+         * @param queryServiceEndpoint
+         * @param updateServiceEndpoint
+         * @param graphStoreProtocolEndpoint
+         * @return RDFConnectionFuseki
+    s     */
+        public static RDFConnectionFuseki connectFuseki(String datasetURL, 
+                                                        String queryServiceEndpoint, 
+                                                        String updateServiceEndpoint,
+                                                        String graphStoreProtocolEndpoint) {
+            return (RDFConnectionFuseki)RDFConnectionFuseki.create()
+                .destination(datasetURL)
+                .queryEndpoint(queryServiceEndpoint)
+                .updateEndpoint(updateServiceEndpoint)
+                .gspEndpoint(graphStoreProtocolEndpoint)
+                .build();
+        }
 }


[18/24] jena git commit: Allow raw SPARQL string to be used in HTTP requests. Add Fuseki-specific tests.

Posted by an...@apache.org.
Allow raw SPARQL string to be used in HTTP requests. Add Fuseki-specific tests.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/d7c1e970
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/d7c1e970
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/d7c1e970

Branch: refs/heads/master
Commit: d7c1e97087f71b98d88cba9dec388cedf20d8fdc
Parents: 6b2dcba
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 10:57:10 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 10:57:10 2018 +0000

----------------------------------------------------------------------
 .../sparql/engine/http/QueryEngineHTTP.java     |   6 +
 .../jena/sparql/modify/UpdateProcessRemote.java |   2 +-
 .../sparql/modify/UpdateProcessRemoteBase.java  |   8 +-
 .../test/rdfconnection/TS_RDFConnection2.java   |  37 ------
 .../TS_RDFConnectionIntegration.java            |  39 ++++++
 .../test/rdfconnection/TestBlankNodeBinary.java |  89 -------------
 .../rdfconnection/TestRDFConnectionFuseki.java  |  30 +++++
 .../TestRDFConnectionFusekiBinary.java          | 123 ++++++++++++++++++
 .../rdfconnection/TestRDFConnectionRemote.java  |   6 +-
 .../rdfconnection/RDFConnectionFactory.java     |  30 ++---
 .../jena/rdfconnection/RDFConnectionFuseki.java | 119 +++++------------
 .../jena/rdfconnection/RDFConnectionRemote.java |  76 +++++++++--
 .../RDFConnectionRemoteBuilder.java             |  21 ++-
 .../jena/rdfconnection/TS_RDFConnection.java    |   4 +-
 .../jena/rdfconnection/TestLibRDFConn.java      | 127 +++++++++++++++++++
 .../apache/jena/rdfconnection/TestRDFConn.java  | 127 -------------------
 16 files changed, 470 insertions(+), 374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
index 543e3a6..165108d 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
@@ -895,6 +895,11 @@ public class QueryEngineHTTP implements QueryExecution {
             sBuff.append(";q=").append(v) ;
     }
 
+    /** Get the HTTP Accept header for the request. */ 
+    public String getAcceptHeader() {
+        return this.acceptHeader;
+    }
+    
     /** Set the HTTP Accept header for the request.
      * Unlike the {@code set??ContentType} operations, this is not checked 
      * for validity.
@@ -902,4 +907,5 @@ public class QueryEngineHTTP implements QueryExecution {
     public void setAcceptHeader(String acceptHeader) {
         this.acceptHeader = acceptHeader;
     }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemote.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemote.java b/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemote.java
index 1d8ef65..f1dd978 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemote.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemote.java
@@ -71,7 +71,7 @@ public class UpdateProcessRemote extends UpdateProcessRemoteBase
         
         // Build endpoint URL
         String endpoint = this.getEndpoint();
-        String querystring = this.getQueryString();
+        String querystring = this.getUpdateString();
         if (querystring != null && !querystring.equals("")) {
             endpoint = endpoint.contains("?") ? endpoint + "&" + querystring : endpoint + "?" + querystring;
         }

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemoteBase.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemoteBase.java b/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemoteBase.java
index e03b087..fdf402d 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemoteBase.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/modify/UpdateProcessRemoteBase.java
@@ -130,6 +130,12 @@ public abstract class UpdateProcessRemoteBase implements UpdateProcessor {
         return this.endpoint;
     }
 
+    /** @deprecated Use {@link #getUpdateString()} */
+    @Deprecated
+    public String getQueryString() {
+        return getUpdateString();
+    }
+
     /**
      * Gets the generated HTTP query string portion of the endpoint URL if applicable
      * <p>
@@ -140,7 +146,7 @@ public abstract class UpdateProcessRemoteBase implements UpdateProcessor {
      * 
      * @return Generated query string
      */
-    public String getQueryString() {
+    public String getUpdateString() {
         return this.getParams().httpString();
     }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
deleted file mode 100644
index ea37a51..0000000
--- a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnection2.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.test.rdfconnection;
-
-import org.junit.runner.RunWith ;
-import org.junit.runners.Suite ;
-
-@RunWith(Suite.class)
-@Suite.SuiteClasses( {
-    // Done in the module
-//    TestRDFConnectionLocalTxnMem.class ,
-//    TestRDFConnectionLocalMRSW.class ,
-    
-    // Addition tests added here.
-    TestRDFConnectionLocalTDB.class ,
-    TestRDFConnectionRemote.class,
-    TestBlankNodeBinary.class
-})
-
-public class TS_RDFConnection2 {}
-

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnectionIntegration.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnectionIntegration.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnectionIntegration.java
new file mode 100644
index 0000000..ab092a3
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TS_RDFConnectionIntegration.java
@@ -0,0 +1,39 @@
+/*
+ * 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.test.rdfconnection;
+
+import org.junit.runner.RunWith ;
+import org.junit.runners.Suite ;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses( {
+    // Done in the module
+    //    TestRDFConnectionLocalTxnMem
+    //    TestRDFConnectionLocalMRSW
+
+    TestBlankNodeBinary.class,
+
+    // Addition tests added here.
+    TestRDFConnectionLocalTDB.class,
+    TestRDFConnectionRemote.class,
+    TestRDFConnectionFuseki.class,
+    TestRDFConnectionFusekiBinary.class
+})
+
+public class TS_RDFConnectionIntegration {}

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
index a3eee4d..6d7d3a5 100644
--- a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestBlankNodeBinary.java
@@ -24,24 +24,15 @@ import static org.junit.Assert.assertTrue;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 
-import org.apache.jena.fuseki.FusekiLib;
-import org.apache.jena.fuseki.embedded.FusekiServer;
 import org.apache.jena.graph.*;
 import org.apache.jena.query.Query;
-import org.apache.jena.query.QueryExecution;
 import org.apache.jena.query.QueryFactory;
-import org.apache.jena.rdf.model.Model;
-import org.apache.jena.rdf.model.ModelFactory;
-import org.apache.jena.rdfconnection.RDFConnectionFuseki;
-import org.apache.jena.rdfconnection.RDFConnectionRemoteBuilder;
 import org.apache.jena.riot.Lang;
 import org.apache.jena.riot.RDFDataMgr;
 import org.apache.jena.sparql.algebra.Algebra;
 import org.apache.jena.sparql.algebra.Op;
 import org.apache.jena.sparql.algebra.op.OpBGP;
 import org.apache.jena.sparql.core.BasicPattern;
-import org.apache.jena.sparql.core.DatasetGraph;
-import org.apache.jena.sparql.core.DatasetGraphFactory;
 import org.apache.jena.sparql.core.TriplePath;
 import org.apache.jena.sparql.modify.request.UpdateDataInsert;
 import org.apache.jena.sparql.sse.SSE;
@@ -109,84 +100,4 @@ public class TestBlankNodeBinary {
         Node obj = ins.getQuads().get(0).getObject();
         assertEquals("789", obj.getBlankNodeLabel());
     }
-    
-    // RDFConnection level testing. 
-
-    @Test public void rdfconnection_binary_1() {
-        // Tests on one connection.
-        Triple triple = SSE.parseTriple("(:s :p <_:b3456>)");
-        // Goes in as URI! (pre this PR)
-        Model model = ModelFactory.createDefaultModel();
-        model.getGraph().add(triple);
-        
-        int PORT = FusekiLib.choosePort();
-        FusekiServer server = createFusekiServer(PORT).build().start();
-        try {
-            String dsURL = "http://localhost:"+PORT+"/ds" ;
-            assertTrue(FusekiLib.isFuseki(dsURL)); 
-
-            RDFConnectionRemoteBuilder builder = RDFConnectionFuseki.create().destination(dsURL);
-
-            try (RDFConnectionFuseki conn = (RDFConnectionFuseki)builder.build()) {
-                assertTrue(FusekiLib.isFuseki(conn));
-                // GSP
-                conn.put(model);
-                checkModel(conn, "b3456");
-
-                // Query forms.
-                conn.querySelect("SELECT * {?s ?p ?o}", x-> {
-                    Node obj = x.get("o").asNode();
-                    assertTrue(obj.isBlank());
-                    assertEquals("b3456", obj.getBlankNodeLabel());
-                });
-
-                try(QueryExecution qExec = conn.query("ASK {?s ?p <_:b3456>}")){
-                    boolean bool = qExec.execAsk();
-                    assertTrue(bool);
-                }
-
-                try (QueryExecution qExec = conn.query("CONSTRUCT { ?s ?p ?o } WHERE { ?s ?p ?o . FILTER (sameTerm(?o, <_:b3456>)) }")){
-                    Model model2 = qExec.execConstruct() ;
-                    checkModel(model2, "b3456");
-                }
-
-                try(QueryExecution qExec = conn.query("DESCRIBE ?s WHERE { ?s ?p <_:b3456>}")){
-                    Model model2 = qExec.execConstruct() ;
-                    checkModel(model2, "b3456");
-                }
-
-                // Update
-                conn.update("CLEAR DEFAULT" );
-                conn.update("INSERT DATA { <x:s> <x:p> <_:b789> }" );
-                checkModel(conn, "b789");
-                conn.update("CLEAR DEFAULT" );
-                conn.update("INSERT DATA { <x:s> <x:p> <_:6789> }" );
-                checkModel(conn, "6789");
-            }
-        } finally { server.stop(); } 
-    }
-
-    private void checkModel(RDFConnectionFuseki conn, String label) {
-        Model model2 = conn.fetch();
-        checkModel(model2, label);
-    }
-
-    private void checkModel(Model model2, String label) {
-        assertEquals(1, model2.size());
-        Node n = model2.listStatements().next().getObject().asNode();
-        assertTrue(n.isBlank());
-        assertEquals(label, n.getBlankNodeLabel());
-    }
-
-    
-    private static FusekiServer.Builder createFusekiServer(int PORT) {
-        DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
-        return  
-            FusekiServer.create()
-                .setPort(PORT)
-                //.setStaticFileBase("/home/afs/ASF/jena-fuseki-cmds/sparqler")
-                .add("/ds", dsg)
-                //.setVerbose(true)
-                ;
-    }               
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFuseki.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFuseki.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFuseki.java
new file mode 100644
index 0000000..d54a72c
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFuseki.java
@@ -0,0 +1,30 @@
+/*
+ * 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.test.rdfconnection;
+
+import org.apache.jena.rdfconnection.RDFConnection;
+import org.apache.jena.rdfconnection.RDFConnectionFactory;
+
+public class TestRDFConnectionFuseki extends TestRDFConnectionRemote {
+    @Override
+    protected RDFConnection connection() {
+        return RDFConnectionFactory.connectFuseki("http://localhost:"+PORT+"/ds");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFusekiBinary.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFusekiBinary.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFusekiBinary.java
new file mode 100644
index 0000000..9619798
--- /dev/null
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionFusekiBinary.java
@@ -0,0 +1,123 @@
+/*
+ * 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.test.rdfconnection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.jena.fuseki.FusekiLib;
+import org.apache.jena.fuseki.embedded.FusekiServer;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.query.QueryExecution;
+import org.apache.jena.rdf.model.Model;
+import org.apache.jena.rdf.model.ModelFactory;
+import org.apache.jena.rdfconnection.RDFConnectionFuseki;
+import org.apache.jena.rdfconnection.RDFConnectionRemoteBuilder;
+import org.apache.jena.sparql.core.DatasetGraph;
+import org.apache.jena.sparql.core.DatasetGraphFactory;
+import org.apache.jena.sparql.sse.SSE;
+import org.junit.Test;
+
+/* Tests that blanknodes work over RDFConnectionFuseki
+ * This consists of testing each of the necessary components,
+ * and then a test of a connection itself.  
+ */
+
+public class TestRDFConnectionFusekiBinary {
+    private static Node n(String str) { return SSE.parseNode(str) ; }
+
+    @Test public void rdfconnection_fuseki_1() {
+        // Tests all run, in order, on one connection.
+        Triple triple = SSE.parseTriple("(:s :p <_:b3456>)");
+        // Goes in as URI! (pre this PR)
+        Model model = ModelFactory.createDefaultModel();
+        model.getGraph().add(triple);
+        
+        int PORT = FusekiLib.choosePort();
+        FusekiServer server = createFusekiServer(PORT).build().start();
+        try {
+            String dsURL = "http://localhost:"+PORT+"/ds" ;
+            assertTrue(FusekiLib.isFuseki(dsURL)); 
+
+            RDFConnectionRemoteBuilder builder = RDFConnectionFuseki.create().destination(dsURL);
+
+            try (RDFConnectionFuseki conn = (RDFConnectionFuseki)builder.build()) {
+                assertTrue(FusekiLib.isFuseki(conn));
+                // GSP
+                conn.put(model);
+                checkModel(conn, "b3456");
+
+                // Query forms.
+                conn.querySelect("SELECT * {?s ?p ?o}", x-> {
+                    Node obj = x.get("o").asNode();
+                    assertTrue(obj.isBlank());
+                    assertEquals("b3456", obj.getBlankNodeLabel());
+                });
+
+                try(QueryExecution qExec = conn.query("ASK {?s ?p <_:b3456>}")){
+                    boolean bool = qExec.execAsk();
+                    assertTrue(bool);
+                }
+
+                try (QueryExecution qExec = conn.query("CONSTRUCT { ?s ?p ?o } WHERE { ?s ?p ?o . FILTER (sameTerm(?o, <_:b3456>)) }")){
+                    Model model2 = qExec.execConstruct() ;
+                    checkModel(model2, "b3456");
+                }
+
+                try(QueryExecution qExec = conn.query("DESCRIBE ?s WHERE { ?s ?p <_:b3456>}")){
+                    Model model2 = qExec.execConstruct() ;
+                    checkModel(model2, "b3456");
+                }
+
+                // Update
+                conn.update("CLEAR DEFAULT" );
+                conn.update("INSERT DATA { <x:s> <x:p> <_:b789> }" );
+                checkModel(conn, "b789");
+                conn.update("CLEAR DEFAULT" );
+                conn.update("INSERT DATA { <x:s> <x:p> <_:6789> }" );
+                checkModel(conn, "6789");
+            }
+        } finally { server.stop(); } 
+    }
+
+    private void checkModel(RDFConnectionFuseki conn, String label) {
+        Model model2 = conn.fetch();
+        checkModel(model2, label);
+    }
+
+    private void checkModel(Model model2, String label) {
+        assertEquals(1, model2.size());
+        Node n = model2.listStatements().next().getObject().asNode();
+        assertTrue(n.isBlank());
+        assertEquals(label, n.getBlankNodeLabel());
+    }
+
+    
+    private static FusekiServer.Builder createFusekiServer(int PORT) {
+        DatasetGraph dsg = DatasetGraphFactory.createTxnMem();
+        return  
+            FusekiServer.create()
+                .setPort(PORT)
+                //.setStaticFileBase("/home/afs/ASF/jena-fuseki-cmds/sparqler")
+                .add("/ds", dsg)
+                //.setVerbose(true)
+                ;
+    }               
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionRemote.java
----------------------------------------------------------------------
diff --git a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionRemote.java b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionRemote.java
index 4b2615f..ef86755 100644
--- a/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionRemote.java
+++ b/jena-integration-tests/src/test/java/org/apache/jena/test/rdfconnection/TestRDFConnectionRemote.java
@@ -20,6 +20,7 @@ package org.apache.jena.test.rdfconnection;
 
 import org.apache.jena.atlas.logging.LogCtl ;
 import org.apache.jena.fuseki.Fuseki ;
+import org.apache.jena.fuseki.FusekiLib;
 import org.apache.jena.fuseki.embedded.FusekiServer ;
 import org.apache.jena.rdfconnection.AbstractTestRDFConnection;
 import org.apache.jena.rdfconnection.RDFConnection;
@@ -34,11 +35,12 @@ import org.junit.BeforeClass ;
 public class TestRDFConnectionRemote extends AbstractTestRDFConnection {
     private static FusekiServer server ;
     private static DatasetGraph serverdsg = DatasetGraphFactory.createTxnMem() ;
+    protected static int PORT  = FusekiLib.choosePort(); 
     
     @BeforeClass
     public static void beforeClass() {
         server = FusekiServer.create()
-            .setPort(2244)
+            .setPort(PORT)
             .add("/ds", serverdsg)
             .build() ;
         LogCtl.setLevel(Fuseki.serverLogName,  "WARN");
@@ -69,7 +71,7 @@ public class TestRDFConnectionRemote extends AbstractTestRDFConnection {
 
     @Override
     protected RDFConnection connection() {
-        return RDFConnectionFactory.connect("http://localhost:2244/ds");
+        return RDFConnectionFactory.connect("http://localhost:"+PORT+"/ds");
     }
 }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
index 3049812..83b1442 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFactory.java
@@ -144,21 +144,21 @@ public class RDFConnectionFactory {
     }
 
     /** Create a connection to a remote Fuseki server by URL.
-         * This is the URL for the dataset.
-         * 
-         * Each service is then specified by a URL which is relative to the {@code datasetURL}.
-         * 
-         * @param datasetURL
-         * @param queryServiceEndpoint
-         * @param updateServiceEndpoint
-         * @param graphStoreProtocolEndpoint
-         * @return RDFConnectionFuseki
-    s     */
-        public static RDFConnectionFuseki connectFuseki(String datasetURL, 
-                                                        String queryServiceEndpoint, 
-                                                        String updateServiceEndpoint,
-                                                        String graphStoreProtocolEndpoint) {
-            return (RDFConnectionFuseki)RDFConnectionFuseki.create()
+     * This is the URL for the dataset.
+     * 
+     * Each service is then specified by a URL which is relative to the {@code datasetURL}.
+     * 
+     * @param datasetURL
+     * @param queryServiceEndpoint
+     * @param updateServiceEndpoint
+     * @param graphStoreProtocolEndpoint
+     * @return RDFConnectionFuseki
+     */
+    public static RDFConnectionFuseki connectFuseki(String datasetURL, 
+                                                    String queryServiceEndpoint, 
+                                                    String updateServiceEndpoint,
+                                                    String graphStoreProtocolEndpoint) {
+        return (RDFConnectionFuseki)RDFConnectionFuseki.create()
                 .destination(datasetURL)
                 .queryEndpoint(queryServiceEndpoint)
                 .updateEndpoint(updateServiceEndpoint)

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
index ece1987..81fab04 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionFuseki.java
@@ -18,27 +18,12 @@
 
 package org.apache.jena.rdfconnection;
 
-import java.util.stream.Stream;
-
 import org.apache.http.client.HttpClient;
 import org.apache.http.protocol.HttpContext;
-import org.apache.jena.graph.Graph;
-import org.apache.jena.graph.Node;
-import org.apache.jena.graph.Triple;
-import org.apache.jena.query.Dataset;
-import org.apache.jena.query.QueryExecution;
-import org.apache.jena.rdf.model.Model;
 import org.apache.jena.riot.Lang;
 import org.apache.jena.riot.RDFFormat;
-import org.apache.jena.riot.WebContent;
 import org.apache.jena.riot.resultset.ResultSetLang;
-import org.apache.jena.riot.web.HttpOp;
-import org.apache.jena.sparql.core.DatasetGraph;
-import org.apache.jena.sparql.core.Quad;
 import org.apache.jena.sparql.core.Transactional;
-import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
-import org.apache.jena.update.UpdateFactory;
-import org.apache.jena.update.UpdateRequest;
 
 /** 
  * Implementation of the {@link RDFConnection} interface for connecting to an Apache Jena Fuseki.
@@ -83,6 +68,7 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
                 .acceptHeaderSelectQuery(ResultSetLang.SPARQLResultSetThrift.getHeaderString())
                 .acceptHeaderAskQuery(ResultSetLang.SPARQLResultSetJSON.getHeaderString())
                 .acceptHeaderQuery(acceptHeaderSPARQL)
+                .parseCheckSPARQL(false)
                 // Create object of this class.
                 .creator((b)->fusekiMaker(b));
     }
@@ -100,89 +86,52 @@ public class RDFConnectionFuseki extends RDFConnectionRemote {
             base.destination, base.queryURL, base.updateURL, base.gspURL,
             base.outputQuads, base.outputTriples,
             base.acceptDataset, base.acceptGraph,
-            base.acceptSparqlResults, base.acceptSelectResult, base.acceptAskResult);
+            base.acceptSparqlResults, base.acceptSelectResult, base.acceptAskResult,
+            base.parseCheckQueries, base.parseCheckUpdates);
     }
     
     protected RDFConnectionFuseki(Transactional txnLifecycle, HttpClient httpClient, HttpContext httpContext, String destination,
                                   String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
                                   String acceptDataset, String acceptGraph, 
-                                  String acceptSparqlResults, String acceptSelectResult, String acceptAskResult) {
+                                  String acceptSparqlResults, String acceptSelectResult, String acceptAskResult,
+                                  boolean parseCheckQueries, boolean parseCheckUpdates) {
         super(txnLifecycle, httpClient, httpContext, 
               destination, queryURL, updateURL, gspURL,
               outputQuads, outputTriples, 
               acceptDataset, acceptGraph,
-              acceptSparqlResults, acceptSelectResult, acceptAskResult);
+              acceptSparqlResults, acceptSelectResult, acceptAskResult, parseCheckQueries, parseCheckUpdates);
     }
     
     // Fuseki specific operations.
     
-    @Override
-    public void update(String updateString) {
-        checkUpdate();
-        if ( true ) {
-            // XXX Parse local, use original string.
-            UpdateRequest req = UpdateFactory.create(updateString);
-        }
-        exec(()->HttpOp.execHttpPost(svcUpdate, WebContent.contentTypeSPARQLUpdate, updateString, this.httpClient, this.httpContext));
-//        update(UpdateFactory.create(updateString));
-    }
-    
-//    @Override
-//    public void querySelect(String query, Consumer<QuerySolution> rowAction) {
-//        try ( QueryExecution qExec = query(query) ) {
-//            qExec.execSelect().forEachRemaining(rowAction);
-//        }
-//    }
-    
-    // Make sure all query goes through query(String) or query(Query) 
-    
-    @Override
-    public QueryExecution query(String queryString) {
-        checkQuery();
-        return exec(()-> {
-            QueryExecution qExec = new QueryEngineHTTP(svcQuery, queryString, httpClient, httpContext);
-            QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
-            // We do not know the kind of query unless we parse it locally.
-            if ( acceptSparqlResults != null )
-                qEngine.setAcceptHeader(super.acceptSparqlResults);
-            else {
-                qEngine.setSelectContentType(acceptSelectResult);
-                qEngine.setAskContentType(acceptAskResult);
-                qEngine.setModelContentType(acceptGraph);
-                qEngine.setDatasetContentType(acceptDataset);
-            }
-            return qEngine ;
-        });
-    }
-
-    /**
-     * Return a {@link Model} that is proxy for a remote model in a Fuseki server. This
-     * support the model operations of accessing statements and changing the model.
-     * <p>
-     * This provide low level access to the remote data. The application will be working
-     * with and manipulating the remote model directly which may involve a significant
-     * overhead for every {@code Model} API operation.
-     * <p>
-     * <b><em>Warning</em>:</b> This is <b>not</b> performant for bulk changes. 
-     * <p>
-     * Getting the model, using {@link #fetch()}, which copies the whole model into a local
-     * {@code Model} object, maniupulating it and putting it back with {@link #put(Model)}
-     * provides another way to work with remote data.
-     * 
-     * @return Model
-     */
-    public Model getModelProxy() { return null; }
-    public Model getModelProxy(String graphName) { return null; }
-    
-    public Graph getGraphProxy() { return null; }
-    public Graph getGraphProxy(String graphName) { return null; }
-
-    public Dataset getDatasetProxy() { return null; }
-    public DatasetGraph getDatasetGraphProxy() { return null; }
-
-    // Or remote RDFStorage?
-    public Stream<Triple> findStream(Node s, Node p , Node o) { return null; }
-    public Stream<Quad> findStream(Node g, Node s, Node p , Node o) { return null; }
+//    /**
+//     * Return a {@link Model} that is proxy for a remote model in a Fuseki server. This
+//     * support the model operations of accessing statements and changing the model.
+//     * <p>
+//     * This provide low level access to the remote data. The application will be working
+//     * with and manipulating the remote model directly which may involve a significant
+//     * overhead for every {@code Model} API operation.
+//     * <p>
+//     * <b><em>Warning</em>:</b> This is <b>not</b> performant for bulk changes. 
+//     * <p>
+//     * Getting the model, using {@link #fetch()}, which copies the whole model into a local
+//     * {@code Model} object, maniupulating it and putting it back with {@link #put(Model)}
+//     * provides another way to work with remote data.
+//     * 
+//     * @return Model
+//     */
+//    public Model getModelProxy() { return null; }
+//    public Model getModelProxy(String graphName) { return null; }
+//    
+//    public Graph getGraphProxy() { return null; }
+//    public Graph getGraphProxy(String graphName) { return null; }
+//
+//    public Dataset getDatasetProxy() { return null; }
+//    public DatasetGraph getDatasetGraphProxy() { return null; }
+//
+//    // Or remote RDFStorage?
+//    public Stream<Triple> findStream(Node s, Node p , Node o) { return null; }
+//    public Stream<Quad> findStream(Node g, Node s, Node p , Node o) { return null; }
 
     // Send Patch 
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
index d0a797a..7af0ca7 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemote.java
@@ -22,6 +22,7 @@ import static java.util.Objects.requireNonNull;
 
 import java.io.File;
 import java.io.InputStream;
+import java.util.Objects;
 import java.util.function.Supplier;
 
 import org.apache.http.HttpEntity;
@@ -29,14 +30,13 @@ import org.apache.http.client.HttpClient;
 import org.apache.http.entity.EntityTemplate;
 import org.apache.http.protocol.HttpContext;
 import org.apache.jena.atlas.io.IO;
+import org.apache.jena.atlas.lib.InternalErrorException;
 import org.apache.jena.atlas.web.HttpException;
 import org.apache.jena.atlas.web.TypedInputStream;
 import org.apache.jena.graph.Graph;
 import org.apache.jena.query.*;
 import org.apache.jena.rdf.model.Model;
 import org.apache.jena.rdf.model.ModelFactory;
-import org.apache.jena.rdfconnection.RDFConnection;
-import org.apache.jena.rdfconnection.RDFConnectionFactory;
 import org.apache.jena.riot.*;
 import org.apache.jena.riot.web.HttpCaptureResponse;
 import org.apache.jena.riot.web.HttpOp;
@@ -47,8 +47,7 @@ import org.apache.jena.sparql.core.Transactional;
 import org.apache.jena.sparql.core.TransactionalLock;
 import org.apache.jena.sparql.engine.http.QueryEngineHTTP;
 import org.apache.jena.system.Txn;
-import org.apache.jena.update.UpdateExecutionFactory;
-import org.apache.jena.update.UpdateProcessor;
+import org.apache.jena.update.UpdateFactory;
 import org.apache.jena.update.UpdateRequest;
 import org.apache.jena.web.HttpSC;
 
@@ -81,6 +80,11 @@ public class RDFConnectionRemote implements RDFConnection {
     protected final String acceptSelectResult;
     protected final String acceptAskResult;
     
+    // Whether to check SPARQL queries given as strings by parsing them.
+    protected final boolean parseCheckQueries;
+    // Whether to check SPARQL updates given as strings by parsing them.
+    protected final boolean parseCheckUpdates;
+
     /** Create a {@link RDFConnectionRemoteBuilder}. */
     public static RDFConnectionRemoteBuilder create() {
         return new RDFConnectionRemoteBuilder();
@@ -160,7 +164,8 @@ public class RDFConnectionRemote implements RDFConnection {
             RDFFormat.NQUADS, RDFFormat.NTRIPLES,
             WebContent.defaultGraphAcceptHeader, WebContent.defaultDatasetAcceptHeader,
             null,
-            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader());
+            QueryEngineHTTP.defaultSelectHeader(), QueryEngineHTTP.defaultAskHeader(),
+            true, true);
     }
 
     // Used by the builder.
@@ -168,7 +173,8 @@ public class RDFConnectionRemote implements RDFConnection {
                                    String queryURL, String updateURL, String gspURL, RDFFormat outputQuads, RDFFormat outputTriples,
                                    String acceptDataset, String acceptGraph,
                                    String acceptSparqlResults,
-                                   String acceptSelectResult, String acceptAskResult) {
+                                   String acceptSelectResult, String acceptAskResult,
+                                   boolean parseCheckQueries, boolean parseCheckUpdates) {
         this.httpClient = httpClient;
         this.httpContext = httpContext;
         this.destination = destination;
@@ -185,6 +191,8 @@ public class RDFConnectionRemote implements RDFConnection {
         this.acceptSparqlResults = acceptSparqlResults;
         this.acceptSelectResult = acceptSelectResult;
         this.acceptAskResult = acceptAskResult;
+        this.parseCheckQueries = parseCheckQueries;
+        this.parseCheckUpdates = parseCheckUpdates;
     }
 
     /** Return the {@link HttpClient} in-use. */ 
@@ -203,14 +211,34 @@ public class RDFConnectionRemote implements RDFConnection {
     }
 
     @Override
+    public QueryExecution query(String queryString) {
+        Objects.requireNonNull(queryString);
+        return queryExec(null, queryString);
+    }
+    
+    @Override
     public QueryExecution query(Query query) {
+        Objects.requireNonNull(query);
+        return queryExec(query, null);
+    }
+    
+    private QueryExecution queryExec(Query query, String queryString) {
         checkQuery();
+        if ( query == null && queryString == null )
+            throw new InternalErrorException("Both query and query string are null"); 
+        if ( query == null ) {
+            if ( parseCheckQueries )
+                QueryFactory.create(queryString);
+        }
+        
+        // Use the query string as provided if possible, otherwise serialize the query.
+        String queryStringToSend = ( queryString != null ) ?  queryString : query.toString();
+        
         return exec(()-> {
-            QueryExecution qExec = QueryExecutionFactory.sparqlService(svcQuery, query, this.httpClient, this.httpContext);
+            QueryExecution qExec = new QueryEngineHTTP(svcQuery, queryStringToSend, httpClient, httpContext);
             QueryEngineHTTP qEngine = (QueryEngineHTTP)qExec;
-            if ( acceptSparqlResults != null )
-                qEngine.setAcceptHeader(acceptSparqlResults);
-            else {
+            // Set the accept header - use the most specific method. 
+            if ( query != null ) {
                 if ( query.isSelectType() && acceptSelectResult != null )
                     qEngine.setAcceptHeader(acceptSelectResult);
                 if ( query.isAskType() && acceptAskResult != null )
@@ -220,15 +248,39 @@ public class RDFConnectionRemote implements RDFConnection {
                 if ( query.isConstructQuad() )
                     qEngine.setDatasetContentType(acceptDataset);
             }
+            // Use the general one.
+            if ( qEngine.getAcceptHeader() == null && acceptSparqlResults != null )
+                qEngine.setAcceptHeader(acceptSparqlResults);
+            // Makre sure it was set somehow.
+            if ( qEngine.getAcceptHeader() == null )
+                throw new JenaConnectionException("No Accept header");   
             return qExec ;
         });
     }
 
     @Override
+    public void update(String updateString) {
+        Objects.requireNonNull(updateString);
+        updateExec(null, updateString);
+    }
+    
+    @Override
     public void update(UpdateRequest update) {
+        Objects.requireNonNull(update);
+        updateExec(update, null);
+    }
+    
+    private void updateExec(UpdateRequest update, String updateString ) {
         checkUpdate();
-        UpdateProcessor proc = UpdateExecutionFactory.createRemote(update, svcUpdate, this.httpClient, this.httpContext);
-        exec(()->proc.execute());
+        if ( update == null && updateString == null )
+            throw new InternalErrorException("Both update request and update string are null"); 
+        if ( update == null ) {
+            if ( parseCheckUpdates )
+                UpdateFactory.create(updateString);
+        }
+        // Use the query string as provided if possible, otherwise serialize the query.
+        String updateStringToSend = ( updateString != null ) ? updateString  : update.toString();
+        exec(()->HttpOp.execHttpPost(svcUpdate, WebContent.contentTypeSPARQLUpdate, updateString, this.httpClient, this.httpContext));
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
index 3b5de7f..617290b 100644
--- a/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
+++ b/jena-rdfconnection/src/main/java/org/apache/jena/rdfconnection/RDFConnectionRemoteBuilder.java
@@ -56,7 +56,10 @@ public class RDFConnectionRemoteBuilder {
     protected String        acceptSelectResult = QueryEngineHTTP.defaultSelectHeader();
     protected String        acceptAskResult    = QueryEngineHTTP.defaultAskHeader();
     // All-purpose head that works for any query type (but is quite long!)
-    protected String        acceptSparqlResults = null;
+    protected String        acceptSparqlResults = acceptSelectResult+","+acceptGraph;
+    // Whether to parse SPARQL Queries and Updates for checkign purposes.
+    protected boolean       parseCheckQueries   = true;
+    protected boolean       parseCheckUpdates   = true;
 
     RDFConnectionRemoteBuilder() { 
         // Default settings are the meber declarations.
@@ -81,6 +84,8 @@ public class RDFConnectionRemoteBuilder {
         
         acceptSelectResult  = base.acceptSelectResult;
         acceptAskResult     = base.acceptAskResult;
+        parseCheckQueries   = base.parseCheckQueries;
+        parseCheckUpdates   = base.parseCheckUpdates;
     }
     
     /** URL of the remote SPARQL endpoint.
@@ -260,7 +265,16 @@ public class RDFConnectionRemoteBuilder {
         this.acceptSparqlResults = acceptHeader;
         return this;
     }
-
+    
+    /**
+     * Set the flag for whether to check SPARQL queries and SPARQL updates provided as a string.   
+     */
+    public RDFConnectionRemoteBuilder parseCheckSPARQL(boolean parseCheck) {
+        this.parseCheckQueries = parseCheck;
+        this.parseCheckUpdates = parseCheck;
+        return this;
+    }
+    
     private Function<RDFConnectionRemoteBuilder, RDFConnection> creator = null;
     /** Provide an alternative function to make the {@link RDFConnection} object.
      * <p>
@@ -304,6 +318,7 @@ public class RDFConnectionRemoteBuilder {
                                         destination, queryURL, updateURL, gspURL,
                                         outputQuads, outputTriples,
                                         acceptDataset, acceptGraph,
-                                        acceptSparqlResults, acceptSelectResult, acceptAskResult);
+                                        acceptSparqlResults, acceptSelectResult, acceptAskResult,
+                                        parseCheckQueries, parseCheckUpdates);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TS_RDFConnection.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TS_RDFConnection.java b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TS_RDFConnection.java
index 82af6a9..e4697af 100644
--- a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TS_RDFConnection.java
+++ b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TS_RDFConnection.java
@@ -23,11 +23,11 @@ import org.junit.runners.Suite;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
-    // Other tests, especifically for RDFCommectionRemote are in jena-integration-tests
+    // Other tests, for RDFConnectionRemote and RDFConnectionFuseki, are in jena-integration-tests
     TestRDFConnectionLocalTxnMem.class
     , TestRDFConnectionLocalMRSW.class
     , TestLocalIsolation.class
-    , TestRDFConn.class
+    , TestLibRDFConn.class
 })
 
 public class TS_RDFConnection {}

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestLibRDFConn.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestLibRDFConn.java b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestLibRDFConn.java
new file mode 100644
index 0000000..37c4280
--- /dev/null
+++ b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestLibRDFConn.java
@@ -0,0 +1,127 @@
+/*
+ * 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.rdfconnection;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+public class TestLibRDFConn {
+    
+    @Test public void service_url_01() {
+        testServiceName(null, "XYZ", "XYZ"); 
+    }
+    
+    @Test public void service_url_02() {
+        testServiceName("http://example/", "XYZ", "http://example/XYZ"); 
+    }
+
+    @Test public void service_url_03() {
+        testServiceName("http://example/abc", "XYZ", "http://example/abc/XYZ"); 
+    }
+
+    @Test public void service_url_04() {
+        testServiceName("http://example/abc/", "XYZ", "http://example/abc/XYZ"); 
+    }
+    
+    @Test public void service_url_05() {
+        testServiceName("http://example/abc?param=value", "XYZ", "http://example/abc/XYZ?param=value"); 
+    }
+
+    @Test public void service_url_06() {
+        testServiceName("http://example/dataset", "http://other/abc/", "http://other/abc/"); 
+    }
+
+    @Test public void service_url_07() {
+        testServiceName("http://example/dataset", "http://example/abc/XYZ?param=value", "http://example/abc/XYZ?param=value"); 
+    }
+    
+    private static void testServiceName(String destination, String service, String expected) {
+        String x = LibRDFConn.formServiceURL(destination, service);
+        assertEquals(expected, x);
+    }
+    
+    // Assumes service name constructed correctly (see above). 
+    
+    @Test public void gsp_url_01() {
+        testGSP("http://example/", null, "http://example/?default");  
+    }
+
+    @Test public void gsp_url_02() {
+        testGSP("http://example/", "default", "http://example/?default");  
+    }
+
+    @Test public void gsp_url_03() {
+        testGSP("http://example/dataset", null, "http://example/dataset?default");  
+    }
+
+    @Test public void gsp_url_04() {
+        testGSP("http://example/dataset", "default", "http://example/dataset?default");  
+    }
+    
+    @Test public void gsp_url_05() {
+        testGSP("http://example/dataset?param=value", "default", "http://example/dataset?param=value&default");  
+    }
+    
+    @Test public void gsp_url_06() {
+        testGSP("http://example/?param=value", "default", "http://example/?param=value&default");  
+    }
+
+    @Test public void gsp_url_07() {
+        testGSP("http://example/dataset?param=value", "default", "http://example/dataset?param=value&default");  
+    }
+    
+    @Test public void gsp_url_08() {
+        testGSP("http://example/dataset/?param=value", "default", "http://example/dataset/?param=value&default");  
+    }
+
+    @Test public void gsp_url_11() {
+        testGSP("http://example/dataset", "name", "http://example/dataset?graph=name");  
+    }
+
+    @Test public void gsp_url_12() {
+        testGSP("http://example/", "name", "http://example/?graph=name");  
+    }
+    
+    @Test public void gsp_url_13() {
+        testGSP("http://example/dataset/", "name", "http://example/dataset/?graph=name");  
+    }
+
+    @Test public void gsp_url_20() {
+        testGSP("http://example/dataset?param=value", null, "http://example/dataset?param=value&default");  
+    }
+
+    @Test public void gsp_url_21() {
+        testGSP("http://example/?param=value", null, "http://example/?param=value&default");  
+    }
+
+    @Test public void gsp_url_16() {
+        testGSP("http://example/dataset?param=value", "name", "http://example/dataset?param=value&graph=name");  
+    }
+
+    @Test public void gsp_url_17() {
+        testGSP("http://example/?param=value", "name", "http://example/?param=value&graph=name");  
+    }
+
+    private void testGSP(String gsp, String graphName, String expected) {
+        String x = LibRDFConn.urlForGraph(gsp, graphName);
+        assertEquals(expected, x);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/d7c1e970/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
----------------------------------------------------------------------
diff --git a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java b/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
deleted file mode 100644
index 46e2159..0000000
--- a/jena-rdfconnection/src/test/java/org/apache/jena/rdfconnection/TestRDFConn.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * 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.rdfconnection;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-public class TestRDFConn {
-    
-    @Test public void service_url_01() {
-        testServiceName(null, "XYZ", "XYZ"); 
-    }
-    
-    @Test public void service_url_02() {
-        testServiceName("http://example/", "XYZ", "http://example/XYZ"); 
-    }
-
-    @Test public void service_url_03() {
-        testServiceName("http://example/abc", "XYZ", "http://example/abc/XYZ"); 
-    }
-
-    @Test public void service_url_04() {
-        testServiceName("http://example/abc/", "XYZ", "http://example/abc/XYZ"); 
-    }
-    
-    @Test public void service_url_05() {
-        testServiceName("http://example/abc?param=value", "XYZ", "http://example/abc/XYZ?param=value"); 
-    }
-
-    @Test public void service_url_06() {
-        testServiceName("http://example/dataset", "http://other/abc/", "http://other/abc/"); 
-    }
-
-    @Test public void service_url_07() {
-        testServiceName("http://example/dataset", "http://example/abc/XYZ?param=value", "http://example/abc/XYZ?param=value"); 
-    }
-    
-    private static void testServiceName(String destination, String service, String expected) {
-        String x = LibRDFConn.formServiceURL(destination, service);
-        assertEquals(expected, x);
-    }
-    
-    // Assumes service name constructed correctly (see above). 
-    
-    @Test public void gsp_url_01() {
-        testGSP("http://example/", null, "http://example/?default");  
-    }
-
-    @Test public void gsp_url_02() {
-        testGSP("http://example/", "default", "http://example/?default");  
-    }
-
-    @Test public void gsp_url_03() {
-        testGSP("http://example/dataset", null, "http://example/dataset?default");  
-    }
-
-    @Test public void gsp_url_04() {
-        testGSP("http://example/dataset", "default", "http://example/dataset?default");  
-    }
-    
-    @Test public void gsp_url_05() {
-        testGSP("http://example/dataset?param=value", "default", "http://example/dataset?param=value&default");  
-    }
-    
-    @Test public void gsp_url_06() {
-        testGSP("http://example/?param=value", "default", "http://example/?param=value&default");  
-    }
-
-    @Test public void gsp_url_07() {
-        testGSP("http://example/dataset?param=value", "default", "http://example/dataset?param=value&default");  
-    }
-    
-    @Test public void gsp_url_08() {
-        testGSP("http://example/dataset/?param=value", "default", "http://example/dataset/?param=value&default");  
-    }
-
-    @Test public void gsp_url_11() {
-        testGSP("http://example/dataset", "name", "http://example/dataset?graph=name");  
-    }
-
-    @Test public void gsp_url_12() {
-        testGSP("http://example/", "name", "http://example/?graph=name");  
-    }
-    
-    @Test public void gsp_url_13() {
-        testGSP("http://example/dataset/", "name", "http://example/dataset/?graph=name");  
-    }
-
-    @Test public void gsp_url_20() {
-        testGSP("http://example/dataset?param=value", null, "http://example/dataset?param=value&default");  
-    }
-
-    @Test public void gsp_url_21() {
-        testGSP("http://example/?param=value", null, "http://example/?param=value&default");  
-    }
-
-    @Test public void gsp_url_16() {
-        testGSP("http://example/dataset?param=value", "name", "http://example/dataset?param=value&graph=name");  
-    }
-
-    @Test public void gsp_url_17() {
-        testGSP("http://example/?param=value", "name", "http://example/?param=value&graph=name");  
-    }
-
-    private void testGSP(String gsp, String graphName, String expected) {
-        String x = LibRDFConn.urlForGraph(gsp, graphName);
-        assertEquals(expected, x);
-    }
-    
-}


[14/24] jena git commit: Add source(Model) and source(Dataset)

Posted by an...@apache.org.
Add source(Model) and source(Dataset)

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/6550960a
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/6550960a
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/6550960a

Branch: refs/heads/master
Commit: 6550960a23bc275762bf4540b27f77e8330f7981
Parents: 7890326
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 08:24:21 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 08:24:21 2018 +0000

----------------------------------------------------------------------
 jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/6550960a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java
index f20f8c2..7cf7759 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriter.java
@@ -200,21 +200,21 @@ public class RDFWriter {
 
     private void write$(OutputStream out, Graph graph, RDFFormat serialization) {
         WriterGraphRIOT w = createGraphWriter$(serialization);
-        w.write(out, graph, RiotLib.prefixMap(graph), null, context);
+        w.write(out, graph, RiotLib.prefixMap(graph), baseURI, context);
     }
 
     private void write$(OutputStream out, DatasetGraph dataset, RDFFormat serialization) {
         WriterDatasetRIOT w = createDatasetWriter$(serialization);
-        w.write(out, dataset, RiotLib.prefixMap(dataset), null, context);
+        w.write(out, dataset, RiotLib.prefixMap(dataset), baseURI, context);
     }
 
     private void write$(Writer out, Graph graph, RDFFormat serialization) {
         WriterGraphRIOT w = createGraphWriter$(serialization);
-        w.write(out, graph, RiotLib.prefixMap(graph), null,context);
+        w.write(out, graph, RiotLib.prefixMap(graph), baseURI,context);
     }
 
     private void write$(Writer out, DatasetGraph dataset, RDFFormat serialization) {
         WriterDatasetRIOT w = createDatasetWriter$(serialization);
-        w.write(out, dataset, RiotLib.prefixMap(dataset), null, context);
+        w.write(out, dataset, RiotLib.prefixMap(dataset), baseURI, context);
     }
 }


[04/24] jena git commit: <_:Label> bnodes in SSE.

Posted by an...@apache.org.
<_:Label> bnodes in SSE.

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/fc3c3c2c
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/fc3c3c2c
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/fc3c3c2c

Branch: refs/heads/master
Commit: fc3c3c2c6880cf01da0371885bcac3af413140b5
Parents: 0a73699
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Feb 20 13:55:10 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Feb 20 13:55:10 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/fc3c3c2c/jena-arq/src/main/java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java b/jena-arq/src/main/java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java
index 6a9be97..3fb679d 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/sse/lang/ParseHandlerPlain.java
@@ -27,6 +27,7 @@ import org.apache.jena.datatypes.TypeMapper ;
 import org.apache.jena.graph.Node ;
 import org.apache.jena.graph.NodeFactory ;
 import org.apache.jena.riot.lang.LabelToNode;
+import org.apache.jena.riot.system.RiotLib;
 import org.apache.jena.sparql.ARQConstants ;
 import org.apache.jena.sparql.core.Var ;
 import org.apache.jena.sparql.core.VarAlloc ;
@@ -171,7 +172,7 @@ public class ParseHandlerPlain implements ParseHandler
     @Override
     public void emitIRI(int line, int column, String iriStr)
     {
-        Node n = NodeFactory.createURI(iriStr) ;
+        Node n = RiotLib.createIRIorBNode(iriStr) ;
         Item item = Item.createNode(n, line, column) ;
         listAdd(item) ;
     }


[19/24] jena git commit: Parse query string in getQuery

Posted by an...@apache.org.
Parse query string in getQuery


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/27c4e479
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/27c4e479
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/27c4e479

Branch: refs/heads/master
Commit: 27c4e4799362d0a4037221b2200ba7eb0cfa00cd
Parents: d7c1e97
Author: Andy Seaborne <an...@apache.org>
Authored: Sun Feb 25 16:41:19 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sun Feb 25 16:41:19 2018 +0000

----------------------------------------------------------------------
 .../sparql/engine/http/QueryEngineHTTP.java     | 21 +++++++++++++++++++-
 1 file changed, 20 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/27c4e479/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
index 165108d..7c7491d 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/http/QueryEngineHTTP.java
@@ -552,7 +552,26 @@ public class QueryEngineHTTP implements QueryExecution {
     // extensions to the far end.
     @Override
     public Query getQuery() {
-        return query;
+        if ( query != null )
+            return query;
+        if ( queryString != null ) {
+            // Object not created with a Query object, may be because there is forgein
+            // syntax in the query or may be because the queystrign was available and the app
+            // didn't want the overhead of parsing it everytime. 
+            // Try to parse it else return null;
+            try { return QueryFactory.create(queryString, Syntax.syntaxARQ); }
+            catch (QueryParseException ex) {}
+            return null ;
+        }
+        return null;
+    }
+
+    /**
+     * Return the query string. If this was supplied in a constructor, there is no
+     * guaranttee this is legal SPARQL syntax.
+     */
+    public String getQueryString() {
+        return queryString;
     }
 
     @Override


[22/24] jena git commit: Revise javadoc for RDF/XML properties

Posted by an...@apache.org.
Revise javadoc for RDF/XML properties


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/88de3ebb
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/88de3ebb
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/88de3ebb

Branch: refs/heads/master
Commit: 88de3ebb13ddb558b70db42f4ed2779fb32429c5
Parents: c88316f
Author: Andy Seaborne <an...@apache.org>
Authored: Mon Feb 26 16:26:08 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Mon Feb 26 16:26:08 2018 +0000

----------------------------------------------------------------------
 jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/88de3ebb/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java b/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
index 49611d6..9a4622d 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/SysRIOT.java
@@ -19,6 +19,7 @@
 package org.apache.jena.riot;
 
 import org.apache.jena.atlas.lib.IRILib ;
+import org.apache.jena.rdf.model.RDFReader;
 import org.apache.jena.riot.system.IRIResolver ;
 import org.apache.jena.sparql.util.Context;
 import org.apache.jena.sparql.util.Symbol ;
@@ -51,7 +52,7 @@ public class SysRIOT
     /**
      * Context key for old style RDFWriter properties. The value of this in a
      * {@link Context} must be a {@code Map<String, Object>}. The entries of the
-     * map are used to set writer properties before the Jena legacy
+     * map are used to set writer properties before the
      * {@link RDFWriter} is called. Only has any effect on RDF/XML and
      * RDF/XML-ABBREV.
      */
@@ -60,8 +61,8 @@ public class SysRIOT
     /**
      * Context key for old style RDFReader properties. The value of this in a
      * {@link Context} must be a {@code Map<String, Object>}. The entries of the
-     * map are used to set reader properties before the Jena legalacy
-     * {@link RDFWriter} is called. Only has any effect on RDF/XML,
+     * map are used to set reader properties before the
+     * {@link RDFReader} is called. Only has any effect on RDF/XML,
      */
     public static final Symbol sysRdfReaderProperties      = Symbol.create(riotBase+"rdfReader_properties") ;
 


[13/24] jena git commit: Add parse(Model) and parse(Dataset)

Posted by an...@apache.org.
Add parse(Model) and parse(Dataset)

Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/78903268
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/78903268
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/78903268

Branch: refs/heads/master
Commit: 78903268f7e48ab6672a183b5db46eeff963e6ea
Parents: c6e13b7
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Feb 24 08:23:29 2018 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Feb 24 08:23:40 2018 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/riot/RDFParser.java    | 20 +++++++++++++++-
 .../org/apache/jena/riot/RDFParserBuilder.java  | 25 ++++++++++++++++++++
 2 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/78903268/jena-arq/src/main/java/org/apache/jena/riot/RDFParser.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFParser.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFParser.java
index 61d8724..ae5e675 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFParser.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFParser.java
@@ -36,6 +36,8 @@ import org.apache.jena.atlas.lib.InternalErrorException;
 import org.apache.jena.atlas.web.ContentType;
 import org.apache.jena.atlas.web.TypedInputStream;
 import org.apache.jena.graph.Graph;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.rdf.model.Model;
 import org.apache.jena.riot.process.normalize.StreamCanonicalLangTag;
 import org.apache.jena.riot.process.normalize.StreamCanonicalLiterals;
 import org.apache.jena.riot.system.*;
@@ -235,7 +237,15 @@ public class RDFParser {
         parse(StreamRDFLib.graph(graph));
     }
     
-
+    /**
+     * Parse the source, sending the results to a {@link Model}.
+     * The source must be for triples; any quads are discarded.
+     * This method is equivalent to {@code parse(model.getGraph())}. 
+     */
+    public void parse(Model model) {
+        parse(model.getGraph());
+    }
+    
     /**
      * Parse the source, sending the results to a {@link DatasetGraph}.
      */
@@ -243,6 +253,14 @@ public class RDFParser {
         parse(StreamRDFLib.dataset(dataset));
     }
     
+    /**
+     * Parse the source, sending the results to a {@link Dataset}.
+     * This method is equivalent to {@code parse(dataset.asDatasetGraph())}. 
+     */
+    public void parse(Dataset dataset) {
+        parse(dataset.asDatasetGraph());
+    }
+    
     /** 
      * Parse the source, sending the results to a {@link StreamRDF}.
      */

http://git-wip-us.apache.org/repos/asf/jena/blob/78903268/jena-arq/src/main/java/org/apache/jena/riot/RDFParserBuilder.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFParserBuilder.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFParserBuilder.java
index 8efdfd6..6a596bf 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFParserBuilder.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFParserBuilder.java
@@ -30,6 +30,8 @@ import org.apache.http.message.BasicHeader;
 import org.apache.jena.atlas.lib.IRILib;
 import org.apache.jena.graph.BlankNodeId;
 import org.apache.jena.graph.Graph;
+import org.apache.jena.query.Dataset;
+import org.apache.jena.rdf.model.Model;
 import org.apache.jena.riot.RDFParser.LangTagForm;
 import org.apache.jena.riot.lang.LabelToNode;
 import org.apache.jena.riot.system.*;
@@ -511,6 +513,18 @@ public class RDFParserBuilder {
     }
 
     /**
+     * Parse the source, sending the results to a {@link Model}.
+     * The source must be for triples; any quads are discarded. 
+     * Short form for {@code build().parse(model)}
+     * which sends triples and prefixes to the {@code Model}.
+     * 
+     * @param model
+     */
+    public void parse(Model model) {
+        build().parse(model);
+    }
+
+    /**
      * Parse the source, sending the results to a {@link DatasetGraph}.
      * Short form for {@code build().parse(dataset)}
      * which sends triples and prefixes to the {@code DatasetGraph}.
@@ -521,6 +535,17 @@ public class RDFParserBuilder {
         build().parse(dataset);
     }
 
+    /**
+     * Parse the source, sending the results to a {@link Dataset}.
+     * Short form for {@code build().parse(dataset)}
+     * which sends triples and prefixes to the {@code Dataset}.
+     * 
+     * @param dataset
+     */
+    public void parse(Dataset dataset) {
+        build().parse(dataset);
+    }
+
     /** Build an {@link RDFParser}. The parser takes it's configuration from this builder and can not then be changed.
      * The source must be set.
      * When a parser is used, it is takes the source and sends output to an {@link StreamRDF}.