You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by pu...@apache.org on 2016/06/16 16:47:15 UTC

incubator-rya git commit: Adding Visibilties to Rya Web

Repository: incubator-rya
Updated Branches:
  refs/heads/develop 3032690ee -> 54e6c4741


Adding Visibilties to Rya Web

Modified the Rya Web Rest service to take in visibilites; renamed
integration test to run as unit test; updated Example to use visibilities
and auths


Project: http://git-wip-us.apache.org/repos/asf/incubator-rya/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-rya/commit/54e6c474
Tree: http://git-wip-us.apache.org/repos/asf/incubator-rya/tree/54e6c474
Diff: http://git-wip-us.apache.org/repos/asf/incubator-rya/diff/54e6c474

Branch: refs/heads/develop
Commit: 54e6c4741626c0e768ff5fcec3f1f9c90d3d7466
Parents: 3032690
Author: Aaron Mihalik <mi...@alum.mit.edu>
Authored: Thu Jun 2 11:51:34 2016 -0400
Committer: pujav65 <pu...@gmail.com>
Committed: Thu Jun 16 11:21:05 2016 -0400

----------------------------------------------------------------------
 .../src/main/java/RyaDirectExample.java         |  74 +++++++
 .../rya/rdftriplestore/RdfCloudTripleStore.java |   2 +-
 .../rya/RdfCloudTripleStoreConnectionTest.java  |  89 +++++++-
 .../mvm/cloud/rdf/web/sail/RdfController.java   |  41 ++--
 web/web.rya/src/main/webapp/sparqlQuery.jsp     |   9 +-
 .../rdf/web/sail/RdfControllerAccumuloTest.java | 219 +++++++++++++++++++
 .../web/sail/RdfControllerIntegrationTest.java  |  86 --------
 .../controllerIntegrationTest-root.xml          |   1 +
 .../controllerIntegrationTest-security.xml      |  34 +++
 9 files changed, 447 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/extras/indexingExample/src/main/java/RyaDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/RyaDirectExample.java b/extras/indexingExample/src/main/java/RyaDirectExample.java
index c4faa69..1c303c6 100644
--- a/extras/indexingExample/src/main/java/RyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/RyaDirectExample.java
@@ -36,6 +36,7 @@ import org.openrdf.model.URI;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
 import org.openrdf.model.vocabulary.RDF;
 import org.openrdf.model.vocabulary.RDFS;
 import org.openrdf.query.BindingSet;
@@ -99,6 +100,8 @@ public class RyaDirectExample {
 			final long start = System.currentTimeMillis();
 			log.info("Running SPARQL Example: Add and Delete");
 			testAddAndDelete(conn);
+            log.info("Running SPARQL Example: Add with Visibilities and Query with Authorizations");
+            testAddAndQueryWithAuths(conf);
 			log.info("Running SAIL/SPARQL Example: PCJ Search");
 			testPCJSearch(conn);
 			log.info("Running SAIL/SPARQL Example: Add and Temporal Search");
@@ -215,6 +218,77 @@ public class RyaDirectExample {
 		Validate.isTrue(resultHandler.getCount() == 0);
 	}
 
+	private static void testAddAndQueryWithAuths(Configuration conf) throws Exception {
+        Sail s;
+        SailRepository r;
+        SailRepositoryConnection conn;
+
+        // Create a connection that writes data with "A&B" visibilities
+	    ValueFactory vf = new ValueFactoryImpl();
+	    
+	    Configuration confAB = new Configuration(conf);
+        confAB.set(RdfCloudTripleStoreConfiguration.CONF_CV, "A&B");
+
+        s = RyaSailFactory.getInstance(confAB);
+        r = new SailRepository(s);
+        r.initialize();
+        conn = r.getConnection();
+
+        conn.add(vf.createStatement(vf.createURI("u:S_AB"), vf.createURI("u:pred"), vf.createURI("u:O_AB")));
+
+        closeQuietly(r);
+
+        // Create a connection that writes data with "B&C" visibilities
+        Configuration confBC = new Configuration(conf);
+        confBC.set(RdfCloudTripleStoreConfiguration.CONF_CV, "B&C");
+
+        s = RyaSailFactory.getInstance(confBC);
+        r = new SailRepository(s);
+        r.initialize();
+        conn = r.getConnection();
+
+        conn.add(vf.createStatement(vf.createURI("u:S_BC"), vf.createURI("u:pred"), vf.createURI("u:O_BC")));
+
+        closeQuietly(r);
+
+        //Query
+        s = RyaSailFactory.getInstance(conf);
+        r = new SailRepository(s);
+        r.initialize();
+        conn = r.getConnection();
+
+        String query;
+        
+        // Query with No Auths
+        query = "select ?s ?o { ?s <u:pred> ?o . }";
+        final CountingResultHandler resultHandler = new CountingResultHandler();
+        TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+        tupleQuery.evaluate(resultHandler);
+        log.info("Result count : " + resultHandler.getCount());
+
+        Validate.isTrue(resultHandler.getCount() == 0);
+        resultHandler.resetCount();
+        
+        // Query with A B Auths
+        // Note: Alternatively, you can set this value on the conf used to create the conn
+        tupleQuery.setBinding(RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, vf.createLiteral("A,B"));
+        tupleQuery.evaluate(resultHandler);
+        log.info("Result count : " + resultHandler.getCount());
+
+        Validate.isTrue(resultHandler.getCount() == 1);
+        resultHandler.resetCount();
+
+        // Query with A B C Auths
+        tupleQuery.setBinding(RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, vf.createLiteral("A,B,C"));
+        tupleQuery.evaluate(resultHandler);
+        log.info("Result count : " + resultHandler.getCount());
+
+        Validate.isTrue(resultHandler.getCount() == 2);
+        resultHandler.resetCount();
+
+        closeQuietly(r);
+	}
+	
 	private static void testPCJSearch(SailRepositoryConnection conn)
 			throws Exception {
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStore.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStore.java b/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStore.java
index 4fcc726..6b47c0c 100644
--- a/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStore.java
+++ b/sail/src/main/java/mvm/rya/rdftriplestore/RdfCloudTripleStore.java
@@ -53,7 +53,7 @@ public class RdfCloudTripleStore extends SailBase {
 
     @Override
     protected SailConnection getConnectionInternal() throws SailException {
-        return new RdfCloudTripleStoreConnection(this, conf, vf);
+        return new RdfCloudTripleStoreConnection(this, conf.clone(), vf);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/sail/src/test/java/mvm/rya/RdfCloudTripleStoreConnectionTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/mvm/rya/RdfCloudTripleStoreConnectionTest.java b/sail/src/test/java/mvm/rya/RdfCloudTripleStoreConnectionTest.java
index 31efa3a..0ea6e63 100644
--- a/sail/src/test/java/mvm/rya/RdfCloudTripleStoreConnectionTest.java
+++ b/sail/src/test/java/mvm/rya/RdfCloudTripleStoreConnectionTest.java
@@ -39,11 +39,12 @@ import mvm.rya.rdftriplestore.namespace.NamespaceManager;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.mock.MockInstance;
-import org.junit.Ignore;
 import org.openrdf.model.Literal;
+import org.openrdf.model.Model;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
+import org.openrdf.model.Value;
 import org.openrdf.model.impl.StatementImpl;
 import org.openrdf.model.impl.URIImpl;
 import org.openrdf.model.impl.ValueFactoryImpl;
@@ -62,6 +63,7 @@ import org.openrdf.repository.RepositoryConnection;
 import org.openrdf.repository.RepositoryResult;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.rio.RDFFormat;
+import org.openrdf.rio.Rio;
 
 /**
  * Class RdfCloudTripleStoreConnectionTest
@@ -1013,6 +1015,91 @@ public class RdfCloudTripleStoreConnectionTest extends TestCase {
 //        conn.close();
 //    }
 
+    private static String escape(Value r) {
+        if (r instanceof URI)
+            return "<" + r.toString() +">";
+        return r.toString();
+    }
+    
+    private static String getSparqlUpdate() throws Exception {
+        InputStream stream = Thread.currentThread().getContextClassLoader().getResourceAsStream("namedgraphs.trig");
+        assertNotNull(stream);
+
+        Model m = Rio.parse(stream, "", RDFFormat.TRIG);
+
+        StringBuffer updateStr = new StringBuffer();
+        updateStr.append("INSERT DATA {\n");
+        for (Statement s : m){
+            if (s.getContext() != null) {
+                updateStr.append("graph ");
+                updateStr.append(escape(s.getContext()));
+                updateStr.append("{ ");
+            }
+
+            updateStr.append(escape(s.getSubject()));
+            updateStr.append(" ");
+            updateStr.append(escape(s.getPredicate()));
+            updateStr.append(" ");
+            updateStr.append(escape(s.getObject()));
+            if (s.getContext() != null){
+                updateStr.append("}");
+            }
+            updateStr.append(" . \n");
+        }
+        updateStr.append("}");
+        return updateStr.toString();
+    }
+
+    // Set the persistence visibilites on the config
+    public void testUpdateWAuthOnConfig() throws Exception {
+        String sparqlUpdate = getSparqlUpdate();
+        
+        RdfCloudTripleStore tstore = new MockRdfCloudStore();
+        NamespaceManager nm = new NamespaceManager(tstore.getRyaDAO(), tstore.getConf());
+        tstore.setNamespaceManager(nm);
+        SailRepository repo = new SailRepository(tstore);
+        tstore.getRyaDAO().getConf().setCv("1|2");
+        repo.initialize();
+
+        RepositoryConnection conn = repo.getConnection();
+        Update u = conn.prepareUpdate(QueryLanguage.SPARQL, sparqlUpdate);
+        u.execute();
+        
+        String query = "PREFIX  ex:  <http://www.example.org/exampleDocument#>\n" +
+                "PREFIX  voc:  <http://www.example.org/vocabulary#>\n" +
+                "PREFIX  foaf:  <http://xmlns.com/foaf/0.1/>\n" +
+                "PREFIX  rdfs:  <http://www.w3.org/2000/01/rdf-schema#>\n" +
+                "\n" +
+                "SELECT * \n" +
+//                "FROM NAMED <http://www.example.org/exampleDocument#G1>\n" +
+                "WHERE\n" +
+                "{\n" +
+                "  GRAPH ex:G1\n" +
+                "  {\n" +
+                "    ?m voc:name ?name ;\n" +
+                "           voc:homepage ?hp .\n" +
+                "  } .\n" +
+                " GRAPH ex:G2\n" +
+                "  {\n" +
+                "    ?m voc:hasSkill ?skill .\n" +
+                "  } .\n" +
+                "}";
+        TupleQuery tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query);
+        tupleQuery.setBinding(RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, vf.createLiteral("2"));
+        CountTupleHandler tupleHandler = new CountTupleHandler();
+        tupleQuery.evaluate(tupleHandler);
+        assertEquals(1, tupleHandler.getCount());
+
+        tupleQuery = conn.prepareTupleQuery(QueryLanguage.SPARQL, query); //no auth
+        tupleHandler = new CountTupleHandler();
+        tupleQuery.evaluate(tupleHandler);
+        assertEquals(0, tupleHandler.getCount());
+
+        conn.close();
+
+        repo.shutDown();
+    }
+
     public void testNamedGraphLoadWAuth() throws Exception {
         InputStream stream = Thread.currentThread().getContextClassLoader().getResourceAsStream("namedgraphs.trig");
         assertNotNull(stream);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/main/java/mvm/cloud/rdf/web/sail/RdfController.java
----------------------------------------------------------------------
diff --git a/web/web.rya/src/main/java/mvm/cloud/rdf/web/sail/RdfController.java b/web/web.rya/src/main/java/mvm/cloud/rdf/web/sail/RdfController.java
index bc6272a..41be580 100644
--- a/web/web.rya/src/main/java/mvm/cloud/rdf/web/sail/RdfController.java
+++ b/web/web.rya/src/main/java/mvm/cloud/rdf/web/sail/RdfController.java
@@ -21,7 +21,6 @@ package mvm.cloud.rdf.web.sail;
 
 
 
-import static mvm.rya.api.RdfCloudTripleStoreConstants.AUTH_NAMESPACE;
 import static mvm.rya.api.RdfCloudTripleStoreConstants.VALUE_FACTORY;
 
 import java.io.IOException;
@@ -37,6 +36,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import mvm.rya.api.security.SecurityProvider;
 import mvm.rya.api.RdfCloudTripleStoreConfiguration;
+import mvm.rya.rdftriplestore.RdfCloudTripleStoreConnection;
 
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -58,9 +58,10 @@ import org.openrdf.query.parser.ParsedUpdate;
 import org.openrdf.query.parser.QueryParserUtil;
 import org.openrdf.query.resultio.sparqljson.SPARQLResultsJSONWriter;
 import org.openrdf.query.resultio.sparqlxml.SPARQLResultsXMLWriter;
-import org.openrdf.repository.Repository;
 import org.openrdf.repository.RepositoryConnection;
 import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.rio.RDFFormat;
 import org.openrdf.rio.RDFHandler;
 import org.openrdf.rio.RDFHandlerException;
@@ -85,7 +86,7 @@ public class RdfController {
 	private static final int QUERY_TIME_OUT_SECONDS = 120;
 
     @Autowired
-    Repository repository;
+    SailRepository repository;
     
     @Autowired   
     SecurityProvider provider;
@@ -93,6 +94,7 @@ public class RdfController {
     @RequestMapping(value = "/queryrdf", method = {RequestMethod.GET, RequestMethod.POST})
     public void queryRdf(@RequestParam("query") String query,
                          @RequestParam(value = RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, required = false) String auth,
+                         @RequestParam(value = RdfCloudTripleStoreConfiguration.CONF_CV, required = false) String vis,
                          @RequestParam(value = RdfCloudTripleStoreConfiguration.CONF_INFER, required = false) String infer,
                          @RequestParam(value = "nullout", required = false) String nullout,
                          @RequestParam(value = RdfCloudTripleStoreConfiguration.CONF_RESULT_FORMAT, required = false) String emit,
@@ -100,7 +102,7 @@ public class RdfController {
                          @RequestParam(value = "callback", required = false) String callback,
                          HttpServletRequest request,
                          HttpServletResponse response) {
-        RepositoryConnection conn = null;
+        SailRepositoryConnection conn = null;
 		final Thread queryThread = Thread.currentThread();
 		auth = StringUtils.arrayToCommaDelimitedString(provider.getUserAuths(request));
 		Timer timer = new Timer();
@@ -130,12 +132,12 @@ public class RdfController {
 
             if (!isBlankQuery) {
             	if (operation instanceof ParsedGraphQuery) {
-            		// Perform Tupple Query
+            		// Perform Graph Query
                     RDFHandler handler = new RDFXMLWriter(os);
                     response.setContentType("text/xml");
                     performGraphQuery(query, conn, auth, infer, nullout, handler);
                 } else if (operation instanceof ParsedTupleQuery) {
-                    // Perform Tupple Query
+                    // Perform Tuple Query
                     TupleQueryResultHandler handler;
 
                     if (requestedFormat && emit.equalsIgnoreCase("json")) {
@@ -149,7 +151,7 @@ public class RdfController {
                     performQuery(query, conn, auth, infer, nullout, handler);
                 } else if (operation instanceof ParsedUpdate) {
                     // Perform Update Query
-                    performUpdate(query, conn, os, auth, infer);
+                    performUpdate(query, conn, os, infer, vis);
                 } else {
                     throw new MalformedQueryException("Cannot process query. Query type not supported.");
                 }
@@ -253,13 +255,16 @@ public class RdfController {
         }
 
     }
-    private void performUpdate(String query, RepositoryConnection conn, ServletOutputStream os, String auth, String infer) throws RepositoryException, MalformedQueryException, IOException {
+    private void performUpdate(String query, SailRepositoryConnection conn, ServletOutputStream os, String infer, String vis) throws RepositoryException, MalformedQueryException, IOException {
         Update update = conn.prepareUpdate(QueryLanguage.SPARQL, query);
-        if (auth != null && auth.length() > 0)
-            update.setBinding(RdfCloudTripleStoreConfiguration.CONF_QUERY_AUTH, VALUE_FACTORY.createLiteral(auth));
         if (infer != null && infer.length() > 0)
             update.setBinding(RdfCloudTripleStoreConfiguration.CONF_INFER, VALUE_FACTORY.createLiteral(Boolean.parseBoolean(infer)));
 
+        if (conn.getSailConnection() instanceof RdfCloudTripleStoreConnection && vis != null) {
+            RdfCloudTripleStoreConnection sailConnection = (RdfCloudTripleStoreConnection) conn.getSailConnection();
+            sailConnection.getConf().set(RdfCloudTripleStoreConfiguration.CONF_CV, vis);
+        }
+
         long startTime = System.currentTimeMillis();
 
         try {
@@ -324,16 +329,16 @@ public class RdfController {
         if (graph != null) {
         	authList.add(VALUE_FACTORY.createURI(graph));
         }
-        RepositoryConnection conn = null;
+        SailRepositoryConnection conn = null;
         try {
             conn = repository.getConnection();
-            if (cv != null && cv.length() > 0) {
-                String[] auths = cv.split("\\|");
-                for (String auth : auths) {
-                    authList.add(VALUE_FACTORY.createURI(AUTH_NAMESPACE, auth));
-                }
-            } 
-            conn.add(new StringReader(body), "", format_r, authList.toArray(new Resource[authList.size()]));
+            
+            if (conn.getSailConnection() instanceof RdfCloudTripleStoreConnection && cv != null) {
+                RdfCloudTripleStoreConnection sailConnection = (RdfCloudTripleStoreConnection) conn.getSailConnection();
+                sailConnection.getConf().set(RdfCloudTripleStoreConfiguration.CONF_CV, cv);
+            }
+
+            conn.add(new StringReader(body), "", format_r);
             conn.commit();
         } finally {
             if (conn != null) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/main/webapp/sparqlQuery.jsp
----------------------------------------------------------------------
diff --git a/web/web.rya/src/main/webapp/sparqlQuery.jsp b/web/web.rya/src/main/webapp/sparqlQuery.jsp
index d026a50..a787b81 100644
--- a/web/web.rya/src/main/webapp/sparqlQuery.jsp
+++ b/web/web.rya/src/main/webapp/sparqlQuery.jsp
@@ -23,12 +23,13 @@ under the License.
     String sparql=request.getParameter("sparql");
     String infer=request.getParameter("infer");
     String auth=request.getParameter("auth");
+    String vis=request.getParameter("vis");
 	String resultFormat = request.getParameter("emit");
     String padding = request.getParameter("padding");
 
     if(sparql != null){
         String sparqlEnc = URLEncoder.encode(sparql,"UTF-8");
-        String urlTo = "queryrdf?query.infer="+infer+"&query.auth="+auth+"&query.resultformat="+resultFormat+"&padding="+padding+"&query="+sparqlEnc;
+        String urlTo = "queryrdf?query.infer="+infer+"&query.auth="+auth+"&conf.cv="+vis+"&query.resultformat="+resultFormat+"&padding="+padding+"&query="+sparqlEnc;
         response.sendRedirect(urlTo);
     }
 %>
@@ -53,7 +54,11 @@ Enter Sparql query here
     <tr>
         <td>Authorization</td>
         <td><INPUT TYPE=TEXT NAME="auth" SIZE="20"></td>
-      </tr>
+    </tr>
+    <tr>
+        <td>visibility</td>
+        <td><INPUT TYPE=TEXT NAME="vis" SIZE="20"></td>
+    </tr>
 		<tr>
 			<td>Result Format</td>
 			<td><select name="emit">

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerAccumuloTest.java
----------------------------------------------------------------------
diff --git a/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerAccumuloTest.java b/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerAccumuloTest.java
new file mode 100644
index 0000000..3bf5726
--- /dev/null
+++ b/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerAccumuloTest.java
@@ -0,0 +1,219 @@
+package mvm.cloud.rdf.web.sail;
+
+/*
+ * 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.
+ */
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
+import static org.springframework.test.web.servlet.setup.MockMvcBuilders.standaloneSetup;
+
+import java.io.ByteArrayInputStream;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.TupleQueryResult;
+import org.openrdf.query.resultio.QueryResultIO;
+import org.openrdf.query.resultio.TupleQueryResultFormat;
+import org.openrdf.repository.Repository;
+import org.openrdf.repository.RepositoryConnection;
+import org.openrdf.rio.RDFFormat;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.http.MediaType;
+import org.springframework.mock.web.MockHttpServletResponse;
+import org.springframework.test.context.ContextConfiguration;
+import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
+import org.springframework.test.context.web.WebAppConfiguration;
+import org.springframework.test.web.servlet.MockMvc;
+import org.springframework.test.web.servlet.ResultActions;
+
+@RunWith(SpringJUnit4ClassRunner.class)
+@WebAppConfiguration
+@ContextConfiguration({"/controllerIntegrationTest-accumulo.xml", "/controllerIntegrationTest-root.xml"})
+public class RdfControllerAccumuloTest {
+
+    private MockMvc mockMvc;
+
+    @Autowired
+    private RdfController controller;
+
+    @Autowired
+    private Repository repository;
+
+    @Rule
+    public ExpectedException thrown = ExpectedException.none();
+
+    @Before
+    public void setup() {
+        this.mockMvc = standaloneSetup(controller).build();
+        try {
+            RepositoryConnection con = repository.getConnection();
+            con.add(getClass().getResourceAsStream("/test.nt"), "", RDFFormat.NTRIPLES);
+            con.close();
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Test
+    public void emptyQuery() throws Exception {
+        mockMvc.perform(get("/queryrdf?query="))
+                .andExpect(status().isOk());
+    }
+
+    @Test
+    public void emptyQueryXMLFormat() throws Exception {
+        this.mockMvc.perform(get("/queryrdf")
+                .param("query", "SELECT * WHERE { ?s a <http://mynamespace/ProductType> . }")
+                .param("query.resultformat", "xml"))
+                .andExpect(status().isOk())
+                .andExpect(content().contentType(MediaType.TEXT_XML));
+        }
+
+    @Test
+    public void loadDataWithVisibilities() throws Exception {
+        this.mockMvc.perform(post("/loadrdf")
+                .content("<http://loadDataWithVisibilities/AB> <http://loadDataWithVisibilities#pred1> \"loadDataWithVisibilities_AB\" . ")
+                .param("format", "N-Triples")
+                .param("conf.cv", "A&B"))
+                .andExpect(status().isOk());
+
+        this.mockMvc.perform(post("/loadrdf")
+                .content("<http://loadDataWithVisibilities/BC> <http://loadDataWithVisibilities#pred1> \"loadDataWithVisibilities_BC\" . ")
+                .param("format", "N-Triples")
+                .param("conf.cv", "B&C"))
+                .andExpect(status().isOk());
+        
+        ResultActions actions;
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://loadDataWithVisibilities#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 0);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://loadDataWithVisibilities#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 0);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A,B")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://loadDataWithVisibilities#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 1);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "B,C")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://loadDataWithVisibilities#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 1);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A,B,C")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://loadDataWithVisibilities#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 2);
+
+    }
+
+    private static void validateCount(MockHttpServletResponse response, int count) throws Exception {
+
+        String rstString = response.getContentAsString();
+        TupleQueryResult result = QueryResultIO.parse(new ByteArrayInputStream(rstString.getBytes()), TupleQueryResultFormat.SPARQL);
+        
+        assertEquals(1, result.getBindingNames().size());
+        String binding = result.getBindingNames().get(0);
+        
+        assertTrue(result.hasNext());
+        BindingSet bs = result.next();
+        assertEquals(Integer.toString(count), bs.getBinding(binding).getValue().stringValue());
+    }        
+
+    @Test
+    public void updateQueryWithVisibilities() throws Exception {
+        this.mockMvc.perform(get("/queryrdf")
+                .param("query", "INSERT DATA { <http://mynamespace/ProductType1_AB> <http://mynamespace#pred1> \"test_AB\" }")
+                .param("conf.cv", "A&B"))
+                .andExpect(status().isOk());
+        this.mockMvc.perform(get("/queryrdf")
+                .param("query", "INSERT DATA { <http://mynamespace/ProductType1_BC> <http://mynamespace#pred1> \"test_BC\" }")
+                .param("conf.cv", "B&C"))
+                .andExpect(status().isOk());
+
+        ResultActions actions;
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://mynamespace#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 0);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://mynamespace#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 0);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A,B")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://mynamespace#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 1);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "B,C")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://mynamespace#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 1);
+
+        actions = this.mockMvc.perform(get("/queryrdf")
+                .param("query.resultformat", "xml")
+                .param("query.auth", "A,B,C")
+                .param("query", "SELECT (COUNT(?s) as ?c) WHERE {?s <http://mynamespace#pred1> ?o}"))
+                .andExpect(status().isOk());
+
+        validateCount(actions.andReturn().getResponse(), 2);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerIntegrationTest.java b/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerIntegrationTest.java
deleted file mode 100644
index eea0bad..0000000
--- a/web/web.rya/src/test/java/mvm/cloud/rdf/web/sail/RdfControllerIntegrationTest.java
+++ /dev/null
@@ -1,86 +0,0 @@
-package mvm.cloud.rdf.web.sail;
-
-/*
- * 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.
- */
-
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.openrdf.repository.Repository;
-import org.openrdf.repository.RepositoryConnection;
-import org.openrdf.rio.RDFFormat;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.http.MediaType;
-import org.springframework.test.context.ContextConfiguration;
-import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
-import org.springframework.test.context.web.WebAppConfiguration;
-import org.springframework.test.web.servlet.MockMvc;
-
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
-import static org.springframework.test.web.servlet.setup.MockMvcBuilders.standaloneSetup;
-
-@RunWith(SpringJUnit4ClassRunner.class)
-@WebAppConfiguration
-@ContextConfiguration({"/controllerIntegrationTest-accumulo.xml", "/controllerIntegrationTest-root.xml"})
-public class RdfControllerIntegrationTest {
-
-    private MockMvc mockMvc;
-
-    @Autowired
-    private RdfController controller;
-
-    @Autowired
-    private Repository repository;
-
-    @Rule
-    public ExpectedException thrown = ExpectedException.none();
-
-    @Before
-    public void setup() {
-        this.mockMvc = standaloneSetup(controller).build();
-        try {
-            RepositoryConnection con = repository.getConnection();
-            con.add(getClass().getResourceAsStream("/test.nt"), "", RDFFormat.NTRIPLES);
-            con.close();
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Test
-    public void emptyQuery() throws Exception {
-        mockMvc.perform(get("/queryrdf?query="))
-                .andExpect(status().isOk());
-    }
-
-    @Test
-    public void emptyQueryXMLFormat() throws Exception {
-        this.mockMvc.perform(get("/queryrdf")
-                .param("query", "SELECT * WHERE { ?s a <http://mynamespace/ProductType> . }")
-                .param("query.resultformat", "xml"))
-                .andExpect(status().isOk())
-                .andExpect(content().contentType(MediaType.TEXT_XML));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/test/resources/controllerIntegrationTest-root.xml
----------------------------------------------------------------------
diff --git a/web/web.rya/src/test/resources/controllerIntegrationTest-root.xml b/web/web.rya/src/test/resources/controllerIntegrationTest-root.xml
index 7fab80d..2928387 100644
--- a/web/web.rya/src/test/resources/controllerIntegrationTest-root.xml
+++ b/web/web.rya/src/test/resources/controllerIntegrationTest-root.xml
@@ -29,6 +29,7 @@ under the License.
 	<context:component-scan base-package="mvm.cloud.rdf.web.sail" />
 	
 	<import resource="controllerIntegrationTest-accumulo.xml"/>
+    <import resource="controllerIntegrationTest-security.xml"/>
 
 	<bean id="inferenceEngine" class="mvm.rya.rdftriplestore.inference.InferenceEngine" init-method="init" destroy-method="destroy">
         <property name="ryaDAO" ref="ryaDAO"/>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/54e6c474/web/web.rya/src/test/resources/controllerIntegrationTest-security.xml
----------------------------------------------------------------------
diff --git a/web/web.rya/src/test/resources/controllerIntegrationTest-security.xml b/web/web.rya/src/test/resources/controllerIntegrationTest-security.xml
new file mode 100644
index 0000000..c15e9d3
--- /dev/null
+++ b/web/web.rya/src/test/resources/controllerIntegrationTest-security.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:p="http://www.springframework.org/schema/p"
+	xmlns:context="http://www.springframework.org/schema/context"
+	xmlns:oxm="http://www.springframework.org/schema/oxm"
+	xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd
+				http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-2.5.xsd
+				http://www.springframework.org/schema/oxm http://www.springframework.org/schema/oxm/spring-oxm-3.0.xsd">
+
+	<context:annotation-config/>
+
+	<bean id="provider" class="mvm.cloud.rdf.web.sail.SecurityProviderImpl"/>
+
+</beans>