You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by rv...@apache.org on 2013/04/09 00:16:00 UTC

svn commit: r1465811 - in /jena/Experimental/jena-jdbc: jena-jdbc-core/src/main/java/org/apache/jena/jdbc/ jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/ jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/ jena-jdbc-driver-remo...

Author: rvesse
Date: Mon Apr  8 22:16:00 2013
New Revision: 1465811

URL: http://svn.apache.org/r1465811
Log:
Start wiring up support for specifying default and named graph URIs in the Remote Endpoint driver

Added:
    jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/test/java/org/apache/jena/jdbc/remote/connections/TestRemoteEndpointConnectionWithGraphUris.java
Modified:
    jena/Experimental/jena-jdbc/jena-jdbc-core/src/main/java/org/apache/jena/jdbc/JenaDriver.java
    jena/Experimental/jena-jdbc/jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/TestUtils.java
    jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/RemoteEndpointDriver.java
    jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/connections/RemoteEndpointConnection.java
    jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/statements/RemoteEndpointStatement.java

Modified: jena/Experimental/jena-jdbc/jena-jdbc-core/src/main/java/org/apache/jena/jdbc/JenaDriver.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-core/src/main/java/org/apache/jena/jdbc/JenaDriver.java?rev=1465811&r1=1465810&r2=1465811&view=diff
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-core/src/main/java/org/apache/jena/jdbc/JenaDriver.java (original)
+++ jena/Experimental/jena-jdbc/jena-jdbc-core/src/main/java/org/apache/jena/jdbc/JenaDriver.java Mon Apr  8 22:16:00 2013
@@ -147,7 +147,7 @@ public abstract class JenaDriver impleme
 
     @SuppressWarnings("unchecked")
     @Override
-    public Connection connect(String url, Properties props) throws SQLException {
+    public final Connection connect(String url, Properties props) throws SQLException {
         // Make sure to return null if the URL is not supported
         if (!this.acceptsURL(url))
             return null;
@@ -387,4 +387,25 @@ public abstract class JenaDriver impleme
         return false;
     }
 
+    /**
+     * Helper method which attempts to return the value for a parameter that may allow multiple values
+     * @param props Properties
+     * @param key Parameter
+     * @return List of values (may be empty)
+     * @throws SQLException Thrown if the parameter has a value of an incompatible type
+     */
+    @SuppressWarnings("unchecked")
+    protected List<String> getValues(Properties props, String key) throws SQLException {
+        Object obj = props.get(key);
+        if (obj == null) return new ArrayList<String>();
+        if (obj instanceof List<?>) return (List<String>)obj;
+        if (obj instanceof String) {
+            List<String> values = new ArrayList<String>();
+            values.add(obj.toString());
+            return values;
+        } else {
+            throw new SQLException("Value given for parameter " + key + " was not a string/list of strings");
+        }
+    }
+
 }

Modified: jena/Experimental/jena-jdbc/jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/TestUtils.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/TestUtils.java?rev=1465811&r1=1465810&r2=1465811&view=diff
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/TestUtils.java (original)
+++ jena/Experimental/jena-jdbc/jena-jdbc-core/src/test/java/org/apache/jena/jdbc/utils/TestUtils.java Mon Apr  8 22:16:00 2013
@@ -130,4 +130,24 @@ public class TestUtils {
         }
     }
 
+    /**
+     * Renames a model within a dataset
+     * @param ds Dataset
+     * @param oldUri Old URI
+     * @param newUri New URI
+     */
+    public static void renameGraph(Dataset ds, String oldUri, String newUri) {
+        Model m = oldUri != null ? ds.getNamedModel(oldUri) : ds.getDefaultModel();
+        if (oldUri == null) {
+            ds.setDefaultModel(ModelFactory.createDefaultModel());
+        } else {
+            ds.removeNamedModel(oldUri);
+        }
+        if (newUri == null) {
+            ds.setDefaultModel(m);
+        } else {
+            ds.addNamedModel(newUri, m);
+        }
+    }
+
 }

Modified: jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/RemoteEndpointDriver.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/RemoteEndpointDriver.java?rev=1465811&r1=1465810&r2=1465811&view=diff
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/RemoteEndpointDriver.java (original)
+++ jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/RemoteEndpointDriver.java Mon Apr  8 22:16:00 2013
@@ -20,6 +20,7 @@ package org.apache.jena.jdbc.remote;
 
 import java.sql.DriverManager;
 import java.sql.SQLException;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.jena.jdbc.JenaDriver;
@@ -74,6 +75,34 @@ public class RemoteEndpointDriver extend
     public static final String PARAM_UPDATE_ENDPOINT = "update";
 
     /**
+     * Constant for the connection URL parameter that sets a default graph URI
+     * for SPARQL queries, may be specified multiple times to use specify
+     * multiple default graphs to use
+     */
+    public static final String PARAM_DEFAULT_GRAPH_URI = "default-graph-uri";
+
+    /**
+     * Constant for the connection URL parameter that sets a named graph URI for
+     * SPARQL queries, may be specified multiple times to use specify multiple
+     * named graphs to use
+     */
+    public static final String PARAM_NAMED_GRAPH_URI = "named-graph-uri";
+
+    /**
+     * Constant for the connection URL parameter that sets a default graph URI
+     * for SPARQL updates, may be specified multiple times to use specify
+     * multiple default graphs to use
+     */
+    public static final String PARAM_USING_GRAPH_URI = "using-graph-uri";
+
+    /**
+     * Constant for the connection URL parameter that sets a named graph URI for
+     * SPARQL updates, may be specified multiple times to use specify multiple
+     * named graphs to use
+     */
+    public static final String PARAM_USING_NAMED_GRAPH_URI = "using-named-graph-uri";
+
+    /**
      * Creates a new driver
      */
     public RemoteEndpointDriver() {
@@ -100,8 +129,24 @@ public class RemoteEndpointDriver extend
             throw new SQLException("At least one of the " + PARAM_QUERY_ENDPOINT + " or " + PARAM_UPDATE_ENDPOINT
                     + " connection parameters must be specified to make a remote connection");
 
+        // Gather dataset related parameters
+        List<String> defaultGraphs = this.getValues(props, PARAM_DEFAULT_GRAPH_URI);
+        List<String> namedGraphs = this.getValues(props, PARAM_NAMED_GRAPH_URI);
+        List<String> usingGraphs = this.getValues(props, PARAM_USING_GRAPH_URI);
+        List<String> usingNamedGraphs = this.getValues(props, PARAM_USING_NAMED_GRAPH_URI);
+
         // Create connection
-        return new RemoteEndpointConnection(queryEndpoint, updateEndpoint, JenaConnection.DEFAULT_HOLDABILITY, compatibilityLevel);
+        return new RemoteEndpointConnection(queryEndpoint, updateEndpoint, defaultGraphs, namedGraphs, usingGraphs,
+                usingNamedGraphs, JenaConnection.DEFAULT_HOLDABILITY, compatibilityLevel);
     }
 
+    @Override
+    protected boolean allowsMultipleValues(String key) {
+        if (PARAM_DEFAULT_GRAPH_URI.equals(key) || PARAM_NAMED_GRAPH_URI.equals(key) || PARAM_USING_GRAPH_URI.equals(key)
+                || PARAM_USING_NAMED_GRAPH_URI.equals(key)) {
+            return true;
+        } else {
+            return super.allowsMultipleValues(key);
+        }
+    }
 }

Modified: jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/connections/RemoteEndpointConnection.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/connections/RemoteEndpointConnection.java?rev=1465811&r1=1465810&r2=1465811&view=diff
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/connections/RemoteEndpointConnection.java (original)
+++ jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/connections/RemoteEndpointConnection.java Mon Apr  8 22:16:00 2013
@@ -24,6 +24,8 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
+import java.util.List;
+
 import org.apache.jena.jdbc.JdbcCompatibility;
 import org.apache.jena.jdbc.connections.JenaConnection;
 import org.apache.jena.jdbc.remote.metadata.RemoteEndpointMetadata;
@@ -39,6 +41,10 @@ public class RemoteEndpointConnection ex
     private String queryService, updateService;
     private boolean closed = false;
     private boolean readonly = false;
+    private List<String> defaultGraphUris;
+    private List<String> namedGraphUris;
+    private List<String> usingGraphUris;
+    private List<String> usingNamedGraphUris;
 
     /**
      * Creates a new dataset connection
@@ -56,6 +62,34 @@ public class RemoteEndpointConnection ex
      */
     public RemoteEndpointConnection(String queryEndpoint, String updateEndpoint, int holdability, int compatibilityLevel)
             throws SQLException {
+        this(queryEndpoint, updateEndpoint, null, null, null, null, holdability, compatibilityLevel);
+    }
+
+    /**
+     * Creates a new dataset connection
+     * 
+     * @param queryEndpoint
+     *            SPARQL Query Endpoint
+     * @param updateEndpoint
+     *            SPARQL Update Endpoint
+     * @param defaultGraphUris
+     *            Default Graph URIs for SPARQL queries
+     * @param namedGraphUris
+     *            Named Graph URIs for SPARQL queries
+     * @param usingGraphUris
+     *            Default Graph URIs for SPARQL updates
+     * @param usingNamedGraphUris
+     *            Named Graph URIs for SPARQL updates
+     * @param holdability
+     *            Result Set holdability
+     * @param compatibilityLevel
+     *            JDBC compatibility level, see {@link JdbcCompatibility}
+     * @throws SQLException
+     *             Thrown if there is a problem creating the connection
+     */
+    public RemoteEndpointConnection(String queryEndpoint, String updateEndpoint, List<String> defaultGraphUris,
+            List<String> namedGraphUris, List<String> usingGraphUris, List<String> usingNamedGraphUris, int holdability,
+            int compatibilityLevel) throws SQLException {
         super(holdability, true, Connection.TRANSACTION_NONE, compatibilityLevel);
         if (queryEndpoint == null && updateEndpoint == null)
             throw new SQLException("Must specify one/both of a query endpoint and update endpoint");
@@ -81,6 +115,38 @@ public class RemoteEndpointConnection ex
     public String getUpdateEndpoint() {
         return this.updateService;
     }
+    
+    /**
+     * Gets the default graphs for SPARQL queries (may be null if none were set)
+     * @return Default graphs
+     */
+    public List<String> getDefaultGraphURIs() {
+        return this.defaultGraphUris;
+    }
+    
+    /**
+     * Gets the named graphs for SPARQL queries (may be null if none were set)
+     * @return Named graphs
+     */
+    public List<String> getNamedGraphURIs() {
+        return this.namedGraphUris;
+    }
+    
+    /**
+     * Gets the default graphs for SPARQL updates (may be null if none were set)
+     * @return Default graphs
+     */
+    public List<String> getUsingGraphURIs() {
+        return this.usingGraphUris;
+    }
+    
+    /**
+     * Gets the named graphs for SPARQL updates (may be null if none were set)
+     * @return Named graphs
+     */
+    public List<String> getUsingNamedGraphURIs() {
+        return this.usingNamedGraphUris;
+    }
 
     @Override
     protected void closeInternal() throws SQLException {

Modified: jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/statements/RemoteEndpointStatement.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/statements/RemoteEndpointStatement.java?rev=1465811&r1=1465810&r2=1465811&view=diff
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/statements/RemoteEndpointStatement.java (original)
+++ jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/main/java/org/apache/jena/jdbc/remote/statements/RemoteEndpointStatement.java Mon Apr  8 22:16:00 2013
@@ -29,6 +29,7 @@ import com.hp.hpl.jena.query.Query;
 import com.hp.hpl.jena.query.QueryExecution;
 import com.hp.hpl.jena.query.QueryExecutionFactory;
 import com.hp.hpl.jena.query.ReadWrite;
+import com.hp.hpl.jena.sparql.engine.http.QueryEngineHTTP;
 import com.hp.hpl.jena.update.UpdateExecutionFactory;
 import com.hp.hpl.jena.update.UpdateProcessor;
 import com.hp.hpl.jena.update.UpdateRequest;
@@ -76,7 +77,16 @@ public class RemoteEndpointStatement ext
     @Override
     protected QueryExecution createQueryExecution(Query q) throws SQLException {
         if (this.remoteConn.getQueryEndpoint() == null) throw new SQLException("This statement is backed by a write-only connection, read operations are not supported");
-        return QueryExecutionFactory.sparqlService(this.remoteConn.getQueryEndpoint(), q);
+        
+        // Create basic execution
+        QueryEngineHTTP exec = (QueryEngineHTTP) QueryExecutionFactory.sparqlService(this.remoteConn.getQueryEndpoint(), q);
+        
+        // Apply default and named graphs if appropriate
+        if (this.remoteConn.getDefaultGraphURIs() != null) exec.setDefaultGraphURIs(this.remoteConn.getDefaultGraphURIs());
+        if (this.remoteConn.getNamedGraphURIs() != null) exec.setNamedGraphURIs(this.remoteConn.getNamedGraphURIs());
+        
+        // Return execution
+        return exec;
     }
 
     @Override

Added: jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/test/java/org/apache/jena/jdbc/remote/connections/TestRemoteEndpointConnectionWithGraphUris.java
URL: http://svn.apache.org/viewvc/jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/test/java/org/apache/jena/jdbc/remote/connections/TestRemoteEndpointConnectionWithGraphUris.java?rev=1465811&view=auto
==============================================================================
--- jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/test/java/org/apache/jena/jdbc/remote/connections/TestRemoteEndpointConnectionWithGraphUris.java (added)
+++ jena/Experimental/jena-jdbc/jena-jdbc-driver-remote/src/test/java/org/apache/jena/jdbc/remote/connections/TestRemoteEndpointConnectionWithGraphUris.java Mon Apr  8 22:16:00 2013
@@ -0,0 +1,92 @@
+/**
+ * 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.jdbc.remote.connections;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.jena.fuseki.BaseServerTest;
+import org.apache.jena.fuseki.ServerTest;
+import org.apache.jena.jdbc.JdbcCompatibility;
+import org.apache.jena.jdbc.connections.AbstractJenaConnectionTests;
+import org.apache.jena.jdbc.connections.JenaConnection;
+import org.apache.jena.jdbc.remote.connections.RemoteEndpointConnection;
+import org.apache.jena.jdbc.utils.TestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import com.hp.hpl.jena.query.Dataset;
+
+/**
+ * Tests for the {@link RemoteEndpointConnection} where we force the default graph to be a named graph and use connection URL parameters to ensure desired default graph is used
+ *
+ */
+public class TestRemoteEndpointConnectionWithGraphUris extends AbstractJenaConnectionTests {
+    
+    /**
+     * Constant used for default graph URI in these tests
+     */
+    private static final String DEFAULT_GRAPH_URI = "http://example.org/defaultGraph";
+        
+    /**
+     * Setup for the tests by allocating a Fuseki instance to work with
+     */
+    @BeforeClass
+    public static void setup() {
+        ServerTest.allocServer();
+    }
+    
+    /**
+     * Clean up after each test by resetting the Fuseki instance
+     */
+    @After
+    public void cleanupTest() {
+        ServerTest.resetServer();
+    }
+    
+    /**
+     * Clean up after tests by de-allocating the Fuseki instance
+     */
+    @AfterClass
+    public static void cleanup() {
+        ServerTest.freeServer();
+    }
+
+    @Override
+    protected JenaConnection getConnection() throws SQLException {
+        List<String> defaultGraphs = new ArrayList<String>();
+        defaultGraphs.add(DEFAULT_GRAPH_URI);
+        return new RemoteEndpointConnection(BaseServerTest.serviceQuery, BaseServerTest.serviceUpdate, defaultGraphs, null, null, null, JenaConnection.DEFAULT_HOLDABILITY, JdbcCompatibility.DEFAULT);
+    }
+
+    @Override
+    protected JenaConnection getConnection(Dataset ds) throws SQLException {
+        List<String> defaultGraphs = new ArrayList<String>();
+        defaultGraphs.add(DEFAULT_GRAPH_URI);
+        
+        // Set up the dataset
+        TestUtils.renameGraph(ds, null, DEFAULT_GRAPH_URI);
+        TestUtils.copyToRemoteDataset(ds, BaseServerTest.serviceREST);
+        return new RemoteEndpointConnection(BaseServerTest.serviceQuery, BaseServerTest.serviceUpdate, JenaConnection.DEFAULT_HOLDABILITY, JdbcCompatibility.DEFAULT);
+    }
+
+
+}