You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by ca...@apache.org on 2017/08/25 16:44:50 UTC

[1/5] incubator-rya git commit: RYA-289 Mongo Singleton tests. Closes #216.

Repository: incubator-rya
Updated Branches:
  refs/heads/master 8431dfb78 -> 9c12630bb


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java
index 4ac7707..1c712ca 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java
@@ -29,10 +29,8 @@ import org.apache.rya.api.resolver.RyaToRdfConversions;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoRyaTestBase;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
@@ -46,24 +44,13 @@ import com.google.common.collect.Sets;
 
 import info.aduna.iteration.CloseableIteration;
 
-public class MongoFreeTextIndexerTest extends MongoRyaTestBase {
+public class MongoFreeTextIndexerTest extends MongoTestBase {
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
 
-    MongoDBRdfConfiguration conf;
-
-    @Before
-    public void before() throws Exception {
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.setMongoDBName(getDbName());
-        conf.setCollectionName("rya_");
-        conf.setTablePrefix("another_");
-    }
-    
     @Test
     public void testSearch() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -87,7 +74,7 @@ public class MongoFreeTextIndexerTest extends MongoRyaTestBase {
     @Test
     public void testDelete() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -137,7 +124,7 @@ public class MongoFreeTextIndexerTest extends MongoRyaTestBase {
         conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, "pred:1,pred:2");
 
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             // These should not be stored because they are not in the predicate list
             f.storeStatement(new RyaStatement(new RyaURI("foo:subj1"), new RyaURI(RDFS.LABEL.toString()), new RyaType("invalid")));
@@ -170,7 +157,7 @@ public class MongoFreeTextIndexerTest extends MongoRyaTestBase {
     @Test
     public void testContextSearch() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final URI subject = new URIImpl("foo:subj");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java
index 9069fdc..215e64c 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java
@@ -39,9 +39,7 @@ import org.apache.rya.indexing.TemporalInterval;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoRyaTestBase;
-import org.junit.After;
-import org.junit.AfterClass;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Statement;
@@ -81,10 +79,9 @@ import info.aduna.iteration.CloseableIteration;
  * And a few more.
  *
  */
-public final class MongoTemporalIndexerTest extends MongoRyaTestBase {
-    MongoDBRdfConfiguration conf;
-    MongoTemporalIndexer tIndexer;
-    DBCollection collection;
+public final class MongoTemporalIndexerTest extends MongoTestBase {
+    private MongoTemporalIndexer tIndexer;
+    private DBCollection collection;
 
     private static final String URI_PROPERTY_EVENT_TIME = "Property:event:time";
     private static final String URI_PROPERTY_CIRCA = "Property:circa";
@@ -163,24 +160,10 @@ public final class MongoTemporalIndexerTest extends MongoRyaTestBase {
         for (int i = 0; i < seriesTs.length; i++) {
             seriesSpo[i] = new StatementImpl(vf.createURI("foo:event0" + i), pred1_atTime, vf.createLiteral(seriesTs[i].getAsReadable()));
         }
-
-    }
-
-    /**
-     * @throws java.lang.Exception
-     */
-    @AfterClass
-    public static void tearDownAfterClass() throws Exception {
     }
 
     @Before
     public void before() throws Exception {
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.setTablePrefix("isthisused_");
-        
         // This is from http://linkedevents.org/ontology
         // and http://motools.sourceforge.net/event/event.html
         conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
@@ -189,20 +172,12 @@ public final class MongoTemporalIndexerTest extends MongoRyaTestBase {
                 + URI_PROPERTY_EVENT_TIME);
 
         tIndexer = new MongoTemporalIndexer();
-        tIndexer.initIndexer(conf, mongoClient);
-
+        tIndexer.initIndexer(conf, super.getMongoClient());
 
         final String dbName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
-        final DB db = mongoClient.getDB(dbName);
+        final DB db = super.getMongoClient().getDB(dbName);
         collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
    }
-    /**
-     * @throws java.lang.Exception
-     */
-    @After
-    public void tearDown() throws Exception {
-        tIndexer.close();
-    }
 
     /**
      * Test method for {@link MongoTemporalIndexer#storeStatement(convertStatement(org.openrdf.model.Statement)}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java
index 3385285..c7e73ed 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java
@@ -58,11 +58,11 @@ import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageExcepti
 import org.apache.rya.indexing.entity.storage.TypeStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage.TypeStorageException;
 import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
-import org.apache.rya.indexing.entity.storage.mongo.MongoITBase;
 import org.apache.rya.indexing.entity.storage.mongo.MongoTypeStorage;
 import org.apache.rya.indexing.mongodb.update.RyaObjectStorage.ObjectStorageException;
 import org.apache.rya.indexing.smarturi.SmartUriException;
 import org.apache.rya.indexing.smarturi.duplication.conf.DuplicateDataConfig;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.joda.time.DateTime;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
@@ -76,7 +76,7 @@ import com.google.common.collect.Lists;
 /**
  * Tests the methods of {@link DuplicateDataDetector}.
  */
-public class DuplicateDataDetectorTest extends MongoITBase {
+public class DuplicateDataDetectorTest extends MongoTestBase {
     private static final String RYA_INSTANCE_NAME = "testInstance";
 
     private static final String NAMESPACE = RyaSchema.NAMESPACE;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataIT.java
index 3ca37a6..2f3214d 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataIT.java
@@ -1,5 +1,3 @@
-package org.apache.rya.indexing.statement.metadata;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -8,9 +6,9 @@ package org.apache.rya.indexing.statement.metadata;
  * 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
@@ -18,7 +16,8 @@ package org.apache.rya.indexing.statement.metadata;
  * specific language governing permissions and limitations
  * under the License.
  */
-import java.io.IOException;
+package org.apache.rya.indexing.statement.metadata;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -30,13 +29,9 @@ import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.api.domain.StatementMetadata;
 import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.apache.rya.sail.config.RyaSailFactory;
-import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -53,15 +48,8 @@ import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
-import com.mongodb.MongoClient;
-
-import de.flapdoodle.embed.mongo.distribution.Version;
-
-public class MongoStatementMetadataIT {
+public class MongoStatementMetadataIT extends MongoTestBase {
 
-    protected MockMongoFactory testsFactory;
-    protected MongoClient mongoClient;
-    private MongoDBRdfConfiguration conf;
     private Sail sail;
     private SailRepository repo;
     private SailRepositoryConnection conn;
@@ -75,51 +63,36 @@ public class MongoStatementMetadataIT {
 
     @Before
     public void init() throws Exception {
-        testsFactory = MockMongoFactory.with(Version.Main.PRODUCTION);
-        mongoClient = testsFactory.newMongoClient();
-        conf = getConf();
+        final Set<RyaURI> propertySet = new HashSet<RyaURI>(
+                Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+        conf.setUseStatementMetadata(true);
+        conf.setStatementMetadataProperties(propertySet);
+
         sail = RyaSailFactory.getInstance(conf);
         repo = new SailRepository(sail);
         conn = repo.getConnection();
 
-        dao = new MongoDBRyaDAO(conf, mongoClient);
+        dao = new MongoDBRyaDAO(conf, super.getMongoClient());
         dao.init();
     }
 
-    @After
-    public void close() throws Exception {
-        conn.close();
-        repo.shutDown();
-        sail.shutDown();
-        sail.shutDown();
-        dao.destroy();
-
-        if (mongoClient != null) {
-            mongoClient.close();
-        }
-        if (testsFactory != null) {
-            testsFactory.shutdown();
-        }
-        MongoConnectorFactory.closeMongoClient();
-    }
-
     @Test
     public void simpleQueryWithoutBindingSet() throws Exception {
-        StatementMetadata metadata = new StatementMetadata();
+        final StatementMetadata metadata = new StatementMetadata();
         metadata.addMetadata(new RyaURI("http://createdBy"), new RyaType("Joe"));
         metadata.addMetadata(new RyaURI("http://createdOn"), new RyaType(XMLSchema.DATE, "2017-01-04"));
 
-        RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+        final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
                 new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
         dao.add(statement);
 
-        TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-        QueryBindingSet bs = new QueryBindingSet();
+        final QueryBindingSet bs = new QueryBindingSet();
         bs.addBinding("x", new LiteralImpl("CoffeeShop"));
         bs.addBinding("y", new LiteralImpl("Joe"));
 
-        List<BindingSet> bsList = new ArrayList<>();
+        final List<BindingSet> bsList = new ArrayList<>();
         while (result.hasNext()) {
             bsList.add(result.next());
         }
@@ -134,24 +107,24 @@ public class MongoStatementMetadataIT {
      * Tests if results are filtered correctly using the metadata properties. In
      * this case, the date for the ingested RyaStatement differs from the date
      * specified in the query.
-     * 
+     *
      * @throws MalformedQueryException
      * @throws QueryEvaluationException
      * @throws RyaDAOException
      */
     @Test
     public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
-        StatementMetadata metadata = new StatementMetadata();
+        final StatementMetadata metadata = new StatementMetadata();
         metadata.addMetadata(new RyaURI("http://createdBy"), new RyaType("Doug"));
         metadata.addMetadata(new RyaURI("http://createdOn"), new RyaType(XMLSchema.DATE, "2017-02-15"));
 
-        RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+        final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
                 new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
         dao.add(statement);
 
-        TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-        List<BindingSet> bsList = new ArrayList<>();
+        final List<BindingSet> bsList = new ArrayList<>();
         while (result.hasNext()) {
             bsList.add(result.next());
         }
@@ -162,30 +135,30 @@ public class MongoStatementMetadataIT {
     @Test
     public void simpleQueryWithBindingSet() throws Exception {
 
-        StatementMetadata metadata = new StatementMetadata();
+        final StatementMetadata metadata = new StatementMetadata();
         metadata.addMetadata(new RyaURI("http://createdBy"), new RyaType("Joe"));
         metadata.addMetadata(new RyaURI("http://createdOn"), new RyaType(XMLSchema.DATE, "2017-01-04"));
 
-        RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+        final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
                 new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+        final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
                 new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
         dao.add(statement1);
         dao.add(statement2);
 
-        TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-        Set<BindingSet> expected = new HashSet<>();
-        QueryBindingSet expected1 = new QueryBindingSet();
+        final Set<BindingSet> expected = new HashSet<>();
+        final QueryBindingSet expected1 = new QueryBindingSet();
         expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
         expected1.addBinding("y", new LiteralImpl("Joe"));
-        QueryBindingSet expected2 = new QueryBindingSet();
+        final QueryBindingSet expected2 = new QueryBindingSet();
         expected2.addBinding("x", new LiteralImpl("HardwareStore"));
         expected2.addBinding("y", new LiteralImpl("Joe"));
         expected.add(expected1);
         expected.add(expected2);
 
-        Set<BindingSet> bsSet = new HashSet<>();
+        final Set<BindingSet> bsSet = new HashSet<>();
         while (result.hasNext()) {
             bsSet.add(result.next());
         }
@@ -212,36 +185,36 @@ public class MongoStatementMetadataIT {
     @Test
     public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
 
-        StatementMetadata metadata1 = new StatementMetadata();
+        final StatementMetadata metadata1 = new StatementMetadata();
         metadata1.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://Doug"));
         metadata1.addMetadata(new RyaURI("http://createdOn"), new RyaType(XMLSchema.DATE, "2017-01-04"));
-        StatementMetadata metadata2 = new StatementMetadata();
+        final StatementMetadata metadata2 = new StatementMetadata();
         metadata2.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://Bob"));
         metadata2.addMetadata(new RyaURI("http://createdOn"), new RyaType(XMLSchema.DATE, "2017-02-04"));
 
-        RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+        final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
                 new RyaURI("http://BurgerShack"), new RyaURI("http://context"), "", metadata1);
-        RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
+        final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
                 new RyaURI("http://Betty"), new RyaURI("http://context"), "", metadata1);
-        RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
+        final RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
                 new RyaURI("http://Amanda"), new RyaURI("http://context"), "", metadata1);
-        RyaStatement statement4 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
+        final RyaStatement statement4 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
                 new RyaURI("http://Wanda"), new RyaURI("http://context"), "", metadata2);
         dao.add(statement1);
         dao.add(statement2);
         dao.add(statement3);
         dao.add(statement4);
 
-        TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
+        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
 
-        Set<BindingSet> expected = new HashSet<>();
-        QueryBindingSet expected1 = new QueryBindingSet();
+        final Set<BindingSet> expected = new HashSet<>();
+        final QueryBindingSet expected1 = new QueryBindingSet();
         expected1.addBinding("b", new URIImpl("http://Betty"));
         expected1.addBinding("a", new URIImpl("http://Joe"));
         expected1.addBinding("c", new URIImpl("http://Doug"));
         expected.add(expected1);
 
-        Set<BindingSet> bsSet = new HashSet<>();
+        final Set<BindingSet> bsSet = new HashSet<>();
         while (result.hasNext()) {
             bsSet.add(result.next());
         }
@@ -253,23 +226,4 @@ public class MongoStatementMetadataIT {
         dao.delete(statement3, conf);
         dao.delete(statement4, conf);
     }
-
-    private MongoDBRdfConfiguration getConf() throws IOException {
-
-        String host = mongoClient.getServerAddressList().get(0).getHost();
-        int port = mongoClient.getServerAddressList().get(0).getPort();
-        Set<RyaURI> propertySet = new HashSet<RyaURI>(
-                Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-        MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.setMongoInstance(host);
-        conf.setMongoPort(Integer.toString(port));
-        conf.setMongoDBName("local");
-        conf.setCollectionName("rya");
-        conf.setTablePrefix("rya_");
-        conf.setUseStatementMetadata(true);
-        conf.setStatementMetadataProperties(propertySet);
-        return conf;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java
index 36091ea..d005ce7 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java
@@ -17,7 +17,6 @@ package org.apache.rya.indexing.statement.metadata;
  * specific language governing permissions and limitations
  * under the License.
  */
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -29,12 +28,10 @@ import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.api.domain.StatementMetadata;
 import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.statement.metadata.matching.StatementMetadataNode;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -51,44 +48,23 @@ import org.openrdf.query.algebra.helpers.StatementPatternCollector;
 import org.openrdf.query.parser.ParsedQuery;
 import org.openrdf.query.parser.sparql.SPARQLParser;
 
-import com.mongodb.MongoClient;
-
-import de.flapdoodle.embed.mongo.distribution.Version;
 import info.aduna.iteration.CloseableIteration;
 
-public class MongoStatementMetadataNodeTest {
-
-    protected MockMongoFactory testsFactory;
-    protected MongoClient mongoClient;
-    private MongoDBRdfConfiguration conf;
+public class MongoStatementMetadataNodeTest extends MongoTestBase {
     private MongoDBRyaDAO dao;
     private final String query = "prefix owl: <http://www.w3.org/2002/07/owl#> prefix ano: <http://www.w3.org/2002/07/owl#annotated> prefix rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#> select ?x ?y where {_:blankNode rdf:type owl:Annotation; ano:Source <http://Joe>; "
             + "ano:Property <http://worksAt>; ano:Target ?x; <http://createdBy> ?y; <http://createdOn> \'2017-01-04\'^^xsd:date }";
     private final String query2 = "prefix owl: <http://www.w3.org/2002/07/owl#> prefix ano: <http://www.w3.org/2002/07/owl#annotated> prefix rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#> select ?x ?y where {_:blankNode rdf:type owl:Annotation; ano:Source ?x; "
             + "ano:Property <http://worksAt>; ano:Target ?y; <http://createdBy> ?x; <http://createdOn> \'2017-01-04\'^^xsd:date }";
 
-    
     @Before
     public void init() throws Exception {
-        testsFactory = MockMongoFactory.with(Version.Main.PRODUCTION);
-        mongoClient = testsFactory.newMongoClient();
-        conf = getConf();
-
-        dao = new MongoDBRyaDAO(conf, mongoClient);
-        dao.init();
-    }
-
-    @After
-    public void close() throws RyaDAOException {
-        dao.destroy();
+        final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+        conf.setUseStatementMetadata(true);
+        conf.setStatementMetadataProperties(propertySet);
         
-        if (mongoClient != null) {
-            mongoClient.close();
-        }
-        if (testsFactory != null) {
-            testsFactory.shutdown();
-        }
-        MongoConnectorFactory.closeMongoClient();
+        dao = new MongoDBRyaDAO(conf, super.getMongoClient());
+        dao.init();
     }
 
     @Test
@@ -378,22 +354,4 @@ public class MongoStatementMetadataNodeTest {
         dao.delete(statement1, conf);
         dao.delete(statement2, conf);
     }
-
-    private MongoDBRdfConfiguration getConf() throws IOException {
-        
-        String host = mongoClient.getServerAddressList().get(0).getHost();
-        int port = mongoClient.getServerAddressList().get(0).getPort();
-        Set<RyaURI> propertySet = new HashSet<RyaURI>(
-                Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-        MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.setMongoInstance(host);
-        conf.setMongoPort(Integer.toString(port));
-        conf.setMongoDBName("local");
-        conf.setCollectionName("rya");
-        conf.setTablePrefix("rya_");
-        conf.setUseStatementMetadata(true);
-        conf.setStatementMetadataProperties(propertySet);
-        return conf;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
index a2611d0..7f0d308 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -63,6 +63,7 @@ import com.mongodb.ServerAddress;
 
 import info.aduna.iteration.Iterations;
 
+
 public class MongoRyaDirectExample {
     private static final Logger log = Logger.getLogger(MongoRyaDirectExample.class);
 
@@ -128,9 +129,6 @@ public class MongoRyaDirectExample {
             log.info("Shutting down");
             closeQuietly(conn);
             closeQuietly(repository);
-            if (mock != null) {
-                mock.shutdown();
-            }
             MongoConnectorFactory.closeMongoClient();
         }
     }
@@ -290,15 +288,13 @@ public class MongoRyaDirectExample {
         Validate.isTrue(tupleHandler.getCount() == 2);
     }
 
-    private static MockMongoFactory mock = null;
     private static Configuration getConf() throws IOException {
 
         MongoDBIndexingConfigBuilder builder = MongoIndexingConfiguration.builder()
             .setUseMockMongo(USE_MOCK).setUseInference(USE_INFER).setAuths("U");
 
         if (USE_MOCK) {
-            mock = MockMongoFactory.newFactory();
-            final MongoClient c = mock.newMongoClient();
+            final MongoClient c = MockMongoFactory.newFactory().newMongoClient();
             final ServerAddress address = c.getAddress();
             final String url = address.getHost();
             final String port = Integer.toString(address.getPort());
@@ -765,7 +761,7 @@ public class MongoRyaDirectExample {
         }
 
         public void resetCount() {
-            this.count = 0;
+            count = 0;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
index d9e0294..f35eeb7 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
@@ -32,10 +32,9 @@ import org.apache.rya.indexing.IndexingFunctionRegistry;
 import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
 import org.apache.rya.indexing.TemporalInstant;
 import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
 import org.apache.rya.indexing.geotemporal.mongo.MongoEventStorage;
+import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
-import org.apache.rya.mongodb.MockMongoFactory;
 import org.junit.Test;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -51,7 +50,6 @@ import org.openrdf.query.algebra.ValueExpr;
 import org.openrdf.query.algebra.Var;
 import org.openrdf.query.impl.MapBindingSet;
 
-import com.mongodb.MongoClient;
 import com.vividsolutions.jts.geom.Coordinate;
 import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
@@ -62,7 +60,7 @@ import info.aduna.iteration.CloseableIteration;
 /**
  * Unit tests the methods of {@link EventQueryNode}.
  */
-public class EventQueryNodeTest extends GeoTemporalTestBase {
+public class EventQueryNodeTest extends MongoITBase {
     private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();
 
@@ -113,8 +111,7 @@ public class EventQueryNodeTest extends GeoTemporalTestBase {
 
     @Test
     public void evaluate_constantSubject() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EventStorage storage = new MongoEventStorage(client, "testDB");
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), "testDB");
         RyaURI subject = new RyaURI("urn:event-1111");
         final Geometry geo = GF.createPoint(new Coordinate(1, 1));
         final TemporalInstant temp = new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0);
@@ -163,8 +160,7 @@ public class EventQueryNodeTest extends GeoTemporalTestBase {
 
     @Test
     public void evaluate_variableSubject() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EventStorage storage = new MongoEventStorage(client, "testDB");
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), "testDB");
         RyaURI subject = new RyaURI("urn:event-1111");
         Geometry geo = GF.createPoint(new Coordinate(1, 1));
         final TemporalInstant temp = new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0);
@@ -218,8 +214,7 @@ public class EventQueryNodeTest extends GeoTemporalTestBase {
 
     @Test
     public void evaluate_variableSubject_existingBindingset() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EventStorage storage = new MongoEventStorage(client, "testDB");
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), "testDB");
         RyaURI subject = new RyaURI("urn:event-1111");
         Geometry geo = GF.createPoint(new Coordinate(1, 1));
         final TemporalInstant temp = new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0);
@@ -271,8 +266,7 @@ public class EventQueryNodeTest extends GeoTemporalTestBase {
 
     @Test
     public void evaluate_variableSubject_existingBindingsetWrongFilters() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EventStorage storage = new MongoEventStorage(client, "testDB");
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), "testDB");
         RyaURI subject = new RyaURI("urn:event-1111");
         Geometry geo = GF.createPoint(new Coordinate(1, 1));
         final TemporalInstant temp = new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
index 802e8c1..f2d0868 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
@@ -23,16 +23,12 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.Optional;
 
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.TemporalInstant;
 import org.apache.rya.indexing.geotemporal.model.Event;
-import org.apache.rya.indexing.geotemporal.mongo.MongoGeoTemporalIndexer;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -42,24 +38,17 @@ import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.StatementImpl;
 import org.openrdf.model.impl.ValueFactoryImpl;
 
-import com.mongodb.MongoClient;
 import com.vividsolutions.jts.geom.Geometry;
 
 /**
  * Integration tests the methods of {@link MongoGeoTemporalIndexer}.
  */
 public class MongoGeoTemporalIndexerIT extends MongoITBase {
-
     private MongoGeoTemporalIndexer indexer;
 
     @Before
     public void makeTestIndexer() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
         indexer = new MongoGeoTemporalIndexer();
-        conf.setMongoDBName("GEO_TEMPORAL_INDEXER_TEST");
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "GEO_TEMPORAL_INDEXER_TEST");
-        conf.setMongoClient(client);
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
         indexer.setConf(conf);
         indexer.init();
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
index bd7b5db..7488572 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
@@ -18,16 +18,11 @@
  */
 package org.apache.rya.indexing.geotemporal.mongo;
 
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
+import org.apache.rya.mongodb.MockMongoSingleton;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
 
 import com.mongodb.MongoClient;
@@ -37,41 +32,29 @@ import com.mongodb.MongoClient;
  * use the JUnit framework.
  */
 public class MongoITBase extends GeoTemporalTestBase {
-    protected final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
 
-    private MongoClient mongoClient = null;
-    private Set<String> originalDbNames = null;
+    private static MongoClient mongoClient = null;
+    protected static MongoIndexingConfiguration conf;
 
     @Before
     public void setupTest() throws Exception {
-        conf.setMongoDBName("testDB");
-        conf.setMongoInstance("testDB");
-        conf.setMongoPort("27017");
-
-        mongoClient = MockMongoFactory.newFactory().newMongoClient();
-        conf.setMongoClient(mongoClient);;
-        // Store the names of the DBs that are present before running the test.
-        originalDbNames = new HashSet<>();
-        for(final String name : mongoClient.listDatabaseNames()) {
-            originalDbNames.add(name);
-        }
+        mongoClient = MockMongoSingleton.getInstance();
+        conf = MongoIndexingConfiguration.builder()
+            .setMongoCollectionPrefix("test_")
+            .setMongoDBName("testDB")
+            .build();
+        conf.setBoolean(ConfigUtils.USE_MONGO, true);
+        conf.setMongoClient(mongoClient);
     }
 
     @After
     public void cleanupTest() {
         // Remove any DBs that were created by the test.
         for(final String dbName : mongoClient.listDatabaseNames()) {
-            if(!originalDbNames.contains(dbName)) {
-                mongoClient.dropDatabase(dbName);
-            }
+            mongoClient.dropDatabase(dbName);
         }
     }
 
-    @AfterClass
-    public static void shutdown() {
-        MongoConnectorFactory.closeMongoClient();
-    }
-
     /**
      * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
      */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
index 0eefd3f..d05524f 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
@@ -22,7 +22,6 @@ import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
 
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
@@ -31,9 +30,8 @@ import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.OptionalConfigUtils;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
 import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoRyaTestBase;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -47,23 +45,19 @@ import org.openrdf.model.impl.ValueFactoryImpl;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.vividsolutions.jts.geom.Coordinate;
 import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
 import com.vividsolutions.jts.geom.LineString;
-import com.vividsolutions.jts.geom.LinearRing;
 import com.vividsolutions.jts.geom.Point;
 import com.vividsolutions.jts.geom.Polygon;
 import com.vividsolutions.jts.geom.PrecisionModel;
-import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
 
 import info.aduna.iteration.CloseableIteration;
 
 /**
  * Tests all of the "simple functions" of the geoindexer.
  */
-public class MongoGeoIndexerSfTest extends MongoRyaTestBase {
-    private MongoDBRdfConfiguration conf;
+public class MongoGeoIndexerSfTest extends MongoITBase {
     private static GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
     private static MongoGeoIndexer g;
 
@@ -107,17 +101,11 @@ public class MongoGeoIndexerSfTest extends MongoRyaTestBase {
 
     @Before
     public void before() throws Exception {
-        System.out.println(UUID.randomUUID().toString());
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
         conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         conf.set(OptionalConfigUtils.USE_GEO, "true");
-        conf.setTablePrefix("rya_");
 
         g = new MongoGeoIndexer();
-        g.initIndexer(conf, mongoClient);
+        g.initIndexer(conf, super.getMongoClient());
         g.storeStatement(statement(A));
         g.storeStatement(statement(B));
         g.storeStatement(statement(C));
@@ -135,24 +123,6 @@ public class MongoGeoIndexerSfTest extends MongoRyaTestBase {
 
     }
 
-    private static Point point(final double x, final double y) {
-        return gf.createPoint(new Coordinate(x, y));
-    }
-
-    private static LineString line(final double x1, final double y1, final double x2, final double y2) {
-        return new LineString(new PackedCoordinateSequence.Double(new double[] { x1, y1, x2, y2 }, 2), gf);
-    }
-
-    private static Polygon poly(final double[] arr) {
-        final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(arr, 2));
-        final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-        return p1;
-    }
-
-    private static double[] bbox(final double x1, final double y1, final double x2, final double y2) {
-        return new double[] { x1, y1, x1, y2, x2, y2, x2, y1, x1, y1 };
-    }
-
     public void compare(final CloseableIteration<Statement, ?> actual, final Geometry... expected) throws Exception {
         final Set<Statement> expectedSet = Sets.newHashSet();
         for (final Geometry geo : expected) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
index 024c126..3506f5d 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
@@ -31,9 +31,8 @@ import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.OptionalConfigUtils;
 import org.apache.rya.indexing.StatementConstraints;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
 import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoRyaTestBase;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,29 +54,21 @@ import com.vividsolutions.jts.geom.Polygon;
 import com.vividsolutions.jts.geom.PrecisionModel;
 import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
 
-public class MongoGeoIndexerTest extends MongoRyaTestBase {
-
+public class MongoGeoIndexerTest extends MongoITBase {
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-
-    MongoDBRdfConfiguration conf;
     GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
 
     @Before
     public void before() throws Exception {
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
         conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         conf.set(OptionalConfigUtils.USE_GEO, "true");
-        conf.setTablePrefix("rya_");
     }
 
     @Test
     public void testRestrictPredicatesSearch() throws Exception {
         conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -116,7 +107,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     @Test
     public void testPrimeMeridianSearch() throws Exception {
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -160,7 +151,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDcSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -189,7 +180,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDeleteSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -228,7 +219,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDcSearchWithContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -257,7 +248,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDcSearchWithSubject() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -285,7 +276,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDcSearchWithSubjectAndContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -318,7 +309,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testDcSearchWithPredicate() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -348,7 +339,7 @@ public class MongoGeoIndexerTest extends MongoRyaTestBase {
     public void testAntiMeridianSearch() throws Exception {
         // verify that a search works if the bounding box crosses the anti meridian
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, mongoClient);
+            f.initIndexer(conf, super.getMongoClient());
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource context = vf.createURI("foo:context");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
index 4b66b5b..7473261 100644
--- a/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
+++ b/extras/rya.geoindexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
@@ -30,8 +30,7 @@ import org.apache.rya.indexing.TemporalInstant;
 import org.apache.rya.indexing.TemporalInstantRfc3339;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
+import org.apache.rya.mongodb.MockMongoSingleton;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -64,15 +63,14 @@ public class MongoIndexerDeleteIT {
     @Before
     public void before() throws Exception {
         final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
-            .setMongoCollectionPrefix("rya_")
-            .setMongoDBName("indexerTests")
+                .setMongoCollectionPrefix("rya_").setMongoDBName("indexerTests")
             .setUseMongoFreetextIndex(true)
             .setUseMongoTemporalIndex(true)
             .setMongoFreeTextPredicates(RDFS.LABEL.stringValue())
             .setMongoTemporalPredicates("Property:atTime")
             .build();
 
-        client = MockMongoFactory.newFactory().newMongoClient();
+        client = MockMongoSingleton.getInstance();
         indxrConf.setBoolean(OptionalConfigUtils.USE_GEO, true);
         indxrConf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         indxrConf.setBoolean(ConfigUtils.USE_MONGO, true);
@@ -85,9 +83,6 @@ public class MongoIndexerDeleteIT {
 
     @After
     public void after() throws Exception {
-        if (client != null) {
-            MongoConnectorFactory.closeMongoClient();
-        }
         if (conn != null) {
             conn.clear();
         }


[2/5] incubator-rya git commit: RYA-289 Mongo Singleton tests. Closes #216.

Posted by ca...@apache.org.
RYA-289 Mongo Singleton tests. Closes #216.

Created a mongo client singleton to be used
for tests.
Updated existing tests to use the singleton


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

Branch: refs/heads/master
Commit: e5738966d5df9112da44df3447fc70945c49319c
Parents: 8431dfb
Author: isper3at <sm...@gmail.com>
Authored: Tue Aug 22 18:34:26 2017 -0400
Committer: Caleb Meier <ca...@parsons.com>
Committed: Fri Aug 25 09:43:28 2017 -0700

----------------------------------------------------------------------
 .../org/apache/rya/mongodb/MongoDBRyaDAO.java   |   2 +-
 .../apache/rya/mongodb/MockMongoSingleton.java  |  82 +++++
 .../rya/mongodb/MongoDBQueryEngineTest.java     |  41 +--
 .../org/apache/rya/mongodb/MongoDBRyaDAOIT.java |  61 ++--
 .../apache/rya/mongodb/MongoDBRyaDAOTest.java   |  38 +--
 .../org/apache/rya/mongodb/MongoTestBase.java   |  60 ++++
 .../instance/MongoRyaDetailsRepositoryIT.java   |  27 +-
 .../indexing/mongodb/AbstractMongoIndexer.java  |   1 -
 .../rya/indexing/mongodb/MongoDbSmartUri.java   |   8 -
 .../entity/query/EntityQueryNodeTest.java       |  14 +-
 .../storage/mongo/MongoEntityStorageIT.java     |   3 +-
 .../entity/storage/mongo/MongoITBase.java       |  80 -----
 .../storage/mongo/MongoTypeStorageIT.java       |   3 +-
 .../update/mongo/MongoEntityIndexerIT.java      |   4 +-
 .../external/PrecompJoinOptimizerTest2.java     |   3 +-
 .../rya/indexing/mongo/MongoDbSmartUriIT.java   | 309 +++++++++++++++++
 .../rya/indexing/mongo/MongoDbSmartUriTest.java | 332 -------------------
 .../rya/indexing/mongo/MongoEntityIndexIT.java  |  33 +-
 .../indexing/mongo/MongoEntityIndexTest.java    |  17 +-
 .../mongo/MongoFreeTextIndexerTest.java         |  25 +-
 .../mongo/MongoTemporalIndexerTest.java         |  37 +--
 .../duplication/DuplicateDataDetectorTest.java  |   4 +-
 .../metadata/MongoStatementMetadataIT.java      | 126 +++----
 .../MongoStatementMetadataNodeTest.java         |  56 +---
 .../src/main/java/MongoRyaDirectExample.java    |  10 +-
 .../geotemporal/model/EventQueryNodeTest.java   |  18 +-
 .../mongo/MongoGeoTemporalIndexerIT.java        |  11 -
 .../indexing/geotemporal/mongo/MongoITBase.java |  43 +--
 .../indexing/mongo/MongoGeoIndexerSfTest.java   |  36 +-
 .../rya/indexing/mongo/MongoGeoIndexerTest.java |  31 +-
 .../indexing/mongo/MongoIndexerDeleteIT.java    |  11 +-
 31 files changed, 632 insertions(+), 894 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java
index a32651d..fe0f6f9 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java
@@ -102,7 +102,7 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
     @Override
     public void setConf(final MongoDBRdfConfiguration conf) {
         this.conf = conf;
-        this.auths = conf.getAuthorizations();
+        auths = conf.getAuthorizations();
     }
 
     public MongoClient getMongoClient(){

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MockMongoSingleton.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MockMongoSingleton.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MockMongoSingleton.java
new file mode 100644
index 0000000..c7860af
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MockMongoSingleton.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rya.mongodb;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mongodb.MongoClient;
+
+/**
+ * To be used for tests. Creates a singleton {@link MongoClient} to be used
+ * throughout all of the MongoDB related tests. Without the singleton, the
+ * embedded mongo factory ends up orphaning processes, consuming resources.
+ */
+public class MockMongoSingleton {
+    public static MongoClient getInstance() {
+        return InstanceHolder.SINGLETON.instance;
+    }
+
+    private MockMongoSingleton() {
+        // hiding implicit default constructor
+    }
+
+    private enum InstanceHolder {
+
+        SINGLETON;
+
+        private final Logger log;
+        private MongoClient instance;
+
+        InstanceHolder() {
+            log = LoggerFactory.getLogger(MockMongoSingleton.class);
+            instance = null;
+            try {
+                instance = MockMongoFactory.newFactory().newMongoClient();
+                // JUnit does not have an overall lifecycle event for tearing down
+                // this kind of resource, but shutdown hooks work alright in practice
+                // since this should only be used during testing
+
+                // The only other alternative for lifecycle management is to use a
+                // suite lifecycle to enclose the tests that need this resource.
+                // In practice this becomes unwieldy.
+                Runtime.getRuntime().addShutdownHook(new Thread() {
+                    @Override
+                    public void run() {
+                        try {
+                            instance.close();
+                        } catch (final Throwable t) {
+                            // logging frameworks will likely be shut down
+                            t.printStackTrace(System.err);
+                        }
+                    }
+                });
+
+            } catch (final IOException e) {
+                log.error("Unexpected error while starting mongo client", e);
+            } catch (final Throwable e) {
+                // catching throwable because failure to construct an enum
+                // instance will lead to another error being thrown downstream
+                log.error("Unexpected throwable while starting mongo client", e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java
index 4187c85..a8f2d88 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java
@@ -23,8 +23,6 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.RdfCloudTripleStoreUtils;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
@@ -37,32 +35,27 @@ import org.openrdf.query.BindingSet;
 import org.openrdf.query.impl.MapBindingSet;
 
 import com.google.common.collect.Lists;
+import com.mongodb.MongoClient;
 
 import info.aduna.iteration.CloseableIteration;
 
-public class MongoDBQueryEngineTest extends MongoRyaTestBase {
-
-    // private dao;
-    // private configuration;
+public class MongoDBQueryEngineTest extends MongoTestBase {
+    private MongoClient client;
+    private MongoDBRyaDAO dao;
 
     private MongoDBQueryEngine engine;
-    private MongoDBRdfConfiguration configuration;
+
+    private static final String DB_NAME = "testInstance";
 
     @Before
     public void setUp() throws Exception {
-        // Set up Mongo/Rya
-        final Configuration conf = new Configuration();
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
-        configuration = new MongoDBRdfConfiguration(conf);
-        final int port = mongoClient.getServerAddressList().get(0).getPort();
-        configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, Integer.toString(port));
+        client = super.getMongoClient();
+        conf.setAuths("A", "B", "C");
 
-        engine = new MongoDBQueryEngine(configuration, mongoClient);
+        engine = new MongoDBQueryEngine(conf, client);
 
         // Add Data
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO(configuration, mongoClient);
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO(conf, client);
         dao.add(getStatement("u:a", "u:tt", "u:b"));
         dao.add(getStatement("u:a", "u:tt", "u:c"));
     }
@@ -93,7 +86,7 @@ public class MongoDBQueryEngineTest extends MongoRyaTestBase {
     @Test
     public void statementQuery() throws Exception {
         final RyaStatement s = getStatement("u:a", null, null);
-        Assert.assertEquals(2, size(engine.query(s, configuration)));
+        Assert.assertEquals(2, size(engine.query(s, conf)));
     }
 
     @SuppressWarnings("unchecked")
@@ -106,7 +99,7 @@ public class MongoDBQueryEngineTest extends MongoRyaTestBase {
 
         final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s1, bs1);
         final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
-        Assert.assertEquals(1, size(engine.queryWithBindingSet(stmts1, configuration)));
+        Assert.assertEquals(1, size(engine.queryWithBindingSet(stmts1, conf)));
 
 
         final MapBindingSet bs2 = new MapBindingSet();
@@ -117,14 +110,14 @@ public class MongoDBQueryEngineTest extends MongoRyaTestBase {
         final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s2, bs2);
 
         final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
-        Assert.assertEquals(2, size(engine.queryWithBindingSet(stmts2, configuration)));
+        Assert.assertEquals(2, size(engine.queryWithBindingSet(stmts2, conf)));
 
 
         final Map.Entry<RyaStatement, BindingSet> e3 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s2, bs1);
         final Map.Entry<RyaStatement, BindingSet> e4 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s1, bs2);
-        
+
         final Collection<Entry<RyaStatement, BindingSet>> stmts3 = Lists.newArrayList(e1, e2, e3, e4);
-        Assert.assertEquals(4, size(engine.queryWithBindingSet(stmts3, configuration)));
+        Assert.assertEquals(4, size(engine.queryWithBindingSet(stmts3, conf)));
 }
     @SuppressWarnings("unchecked")
     @Test
@@ -136,7 +129,7 @@ public class MongoDBQueryEngineTest extends MongoRyaTestBase {
 
         final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s, bs1);
         final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
-        Assert.assertEquals(2, size(engine.queryWithBindingSet(stmts1, configuration)));
+        Assert.assertEquals(2, size(engine.queryWithBindingSet(stmts1, conf)));
 
 
         final MapBindingSet bs2 = new MapBindingSet();
@@ -145,6 +138,6 @@ public class MongoDBQueryEngineTest extends MongoRyaTestBase {
         final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s, bs2);
 
         final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
-        Assert.assertEquals(4, size(engine.queryWithBindingSet(stmts2, configuration)));
+        Assert.assertEquals(4, size(engine.queryWithBindingSet(stmts2, conf)));
 }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java
index dc3e026..5f3605e 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOIT.java
@@ -27,8 +27,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
 import org.apache.rya.api.domain.RyaURI;
@@ -41,26 +39,20 @@ import org.calrissian.mango.collect.CloseableIterable;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.mongodb.MongoClient;
 import com.mongodb.MongoException;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 
-public class MongoDBRyaDAOIT extends MongoRyaTestBase {
-
+public class MongoDBRyaDAOIT extends MongoTestBase {
+    private MongoClient client;
     private MongoDBRyaDAO dao;
-    private MongoDBRdfConfiguration configuration;
 
     @Before
     public void setUp() throws IOException, RyaDAOException{
-        final Configuration conf = new Configuration();
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
-        configuration = new MongoDBRdfConfiguration(conf);
-        configuration.setAuths("A", "B", "C");
-        final int port = mongoClient.getServerAddressList().get(0).getPort();
-        configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, Integer.toString(port));
-        dao = new MongoDBRyaDAO(configuration, mongoClient);
+        conf.setAuths("A", "B", "C");
+        client = super.getMongoClient();
+        dao = new MongoDBRyaDAO(conf, client);
     }
 
     @Test
@@ -68,7 +60,7 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         final RyaStatementBuilder builder = new RyaStatementBuilder();
         builder.setPredicate(new RyaURI("http://temp.com"));
         builder.setColumnVisibility(new DocumentVisibility("A").flatten());
-        dao.delete(builder.build(), configuration);
+        dao.delete(builder.build(), conf);
     }
 
     @Test
@@ -79,8 +71,8 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         builder.setObject(new RyaURI("http://object.com"));
         builder.setColumnVisibility(new DocumentVisibility("B").flatten());
 
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(builder.build());
 
@@ -99,16 +91,16 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         builder.setObject(new RyaURI("http://object.com"));
         builder.setColumnVisibility(new DocumentVisibility("C").flatten());
         final RyaStatement statement = builder.build();
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(statement);
 
-        assertEquals(coll.count(),1);
+        assertEquals(1, coll.count());
 
-        dao.delete(statement, configuration);
+        dao.delete(statement, conf);
 
-        assertEquals(coll.count(),0);
+        assertEquals(0, coll.count());
 
     }
 
@@ -122,12 +114,12 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         builder.setColumnVisibility(new DocumentVisibility("A&B&C").flatten());
         final RyaStatement statement = builder.build();
 
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(statement);
 
-        assertEquals(coll.count(),1);
+        assertEquals(1, coll.count());
 
         final RyaStatementBuilder builder2 = new RyaStatementBuilder();
         builder2.setPredicate(new RyaURI("http://temp.com"));
@@ -135,9 +127,9 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         builder2.setContext(new RyaURI("http://context3.com"));
         final RyaStatement query = builder2.build();
 
-        dao.delete(query, configuration);
+        dao.delete(query, conf);
 
-        assertEquals(coll.count(),1);
+        assertEquals(1, coll.count());
     }
 
     @Test
@@ -286,7 +278,8 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "E" = User can view
         assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B", "E")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "C" and "D" and "E" = User can view
+        // Doc requires ("A" and "B")mongoClient or ("C" and "D") and user has
+        // "C" and "D" and "E" = User can view
         assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("C", "D", "E")));
 
         // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "C" = User CANNOT view
@@ -505,8 +498,8 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
      * @throws RyaDAOException
      */
     private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         final RyaStatement statement = buildVisibilityTestRyaStatement(documentVisibility);
 
@@ -514,18 +507,18 @@ public class MongoDBRyaDAOIT extends MongoRyaTestBase {
         dao.add(statement);
         dao.getConf().setAuths(AuthorizationsUtil.getAuthorizationsStringArray(userAuthorizations != null ? userAuthorizations : Authorizations.EMPTY));
 
-        assertEquals(coll.count(), 1);
+        assertEquals(1, coll.count());
 
         final MongoDBQueryEngine queryEngine = (MongoDBQueryEngine) dao.getQueryEngine();
-        queryEngine.setConf(configuration);
+        queryEngine.setConf(conf);
         final CloseableIterable<RyaStatement> iter = queryEngine.query(new RyaQuery(statement));
 
         // Check if user has authorization to view document based on its visibility
         final boolean hasNext = iter.iterator().hasNext();
 
         // Reset
-        dao.delete(statement, configuration);
-        assertEquals(coll.count(), 0);
+        dao.delete(statement, conf);
+        assertEquals(0, coll.count());
         dao.getConf().setAuths(AuthorizationsUtil.getAuthorizationsStringArray(Authorizations.EMPTY));
 
         return hasNext;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java
index c7efbf1..8cf75f1 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java
@@ -21,10 +21,7 @@ package org.apache.rya.mongodb;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
 import org.apache.rya.api.domain.RyaURI;
@@ -33,33 +30,26 @@ import org.bson.Document;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.mongodb.MongoClient;
 import com.mongodb.MongoException;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 
-public class MongoDBRyaDAOTest extends MongoRyaTestBase {
-
+public class MongoDBRyaDAOTest extends MongoTestBase {
+    private MongoClient client;
     private MongoDBRyaDAO dao;
-    private MongoDBRdfConfiguration configuration;
 
     @Before
     public void setUp() throws IOException, RyaDAOException{
-        final Configuration conf = new Configuration();
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, getDbName());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
-        configuration = new MongoDBRdfConfiguration(conf);
-        final int port = mongoClient.getServerAddressList().get(0).getPort();
-        configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
-        dao = new MongoDBRyaDAO(configuration, mongoClient);
+        client = super.getMongoClient();
+        dao = new MongoDBRyaDAO(conf, client);
     }
 
-
     @Test
     public void testDeleteWildcard() throws RyaDAOException {
         final RyaStatementBuilder builder = new RyaStatementBuilder();
         builder.setPredicate(new RyaURI("http://temp.com"));
-        dao.delete(builder.build(), configuration);
+        dao.delete(builder.build(), conf);
     }
 
 
@@ -70,8 +60,8 @@ public class MongoDBRyaDAOTest extends MongoRyaTestBase {
         builder.setSubject(new RyaURI("http://subject.com"));
         builder.setObject(new RyaURI("http://object.com"));
 
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(builder.build());
 
@@ -87,14 +77,14 @@ public class MongoDBRyaDAOTest extends MongoRyaTestBase {
         builder.setObject(new RyaURI("http://object.com"));
         final RyaStatement statement = builder.build();
 
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(statement);
 
         assertEquals(coll.count(),1);
 
-        dao.delete(statement, configuration);
+        dao.delete(statement, conf);
 
         assertEquals(coll.count(),0);
 
@@ -109,8 +99,8 @@ public class MongoDBRyaDAOTest extends MongoRyaTestBase {
         builder.setContext(new RyaURI("http://context.com"));
         final RyaStatement statement = builder.build();
 
-        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
+        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         dao.add(statement);
 
@@ -122,7 +112,7 @@ public class MongoDBRyaDAOTest extends MongoRyaTestBase {
         builder2.setContext(new RyaURI("http://context3.com"));
         final RyaStatement query = builder2.build();
 
-        dao.delete(query, configuration);
+        dao.delete(query, conf);
 
         assertEquals(coll.count(),1);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java
new file mode 100644
index 0000000..e4578f2
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java
@@ -0,0 +1,60 @@
+/**
+ * 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.rya.mongodb;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.After;
+import org.junit.Before;
+
+import com.mongodb.MongoClient;
+
+/**
+ * A base class that may be used when implementing Mongo DB tests that use the
+ * JUnit framework.
+ */
+public class MongoTestBase {
+
+    private static MongoClient mongoClient = null;
+    protected static MongoDBRdfConfiguration conf;
+
+    @Before
+    public void setupTest() throws Exception {
+        conf = new MongoDBRdfConfiguration( new Configuration() );
+        conf.setBoolean("sc.useMongo", true);
+        conf.setTablePrefix("test_");
+        conf.setMongoDBName("testDB");
+        mongoClient = MockMongoSingleton.getInstance();
+        conf.setMongoClient(mongoClient);
+    }
+
+    @After
+    public void cleanupTest() {
+        // Remove any DBs that were created by the test.
+        for(final String dbName : mongoClient.listDatabaseNames()) {
+            mongoClient.dropDatabase(dbName);
+        }
+    }
+
+    /**
+     * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
+     */
+    public MongoClient getMongoClient() {
+        return mongoClient;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/instance/MongoRyaDetailsRepositoryIT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/instance/MongoRyaDetailsRepositoryIT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/instance/MongoRyaDetailsRepositoryIT.java
index 8d1694b..cb276cb 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/instance/MongoRyaDetailsRepositoryIT.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/instance/MongoRyaDetailsRepositoryIT.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.util.Date;
 
 import org.apache.rya.api.instance.RyaDetails;
@@ -41,38 +40,22 @@ import org.apache.rya.api.instance.RyaDetailsRepository.AlreadyInitializedExcept
 import org.apache.rya.api.instance.RyaDetailsRepository.ConcurrentUpdateException;
 import org.apache.rya.api.instance.RyaDetailsRepository.NotInitializedException;
 import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
-import org.junit.AfterClass;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Optional;
 import com.mongodb.MongoClient;
-import com.mongodb.MongoException;
-
-import de.flapdoodle.embed.mongo.tests.MongodForTestsFactory;
 
 /**
  * Tests the methods of {@link AccumuloRyaDetailsRepository} by using a {@link MiniAccumuloCluster}.
  */
-public class MongoRyaDetailsRepositoryIT {
-
-    private static MongoClient client = null;
-
-    @BeforeClass
-    public static void startMiniAccumulo() throws MongoException, IOException {
-        final MongodForTestsFactory mongoFactory = new MongodForTestsFactory();
-        client = mongoFactory.newMongo();
-    }
+public class MongoRyaDetailsRepositoryIT extends MongoTestBase {
+    private MongoClient client;
 
     @Before
-    public void clearLastTest() {
-        client.dropDatabase("testInstance");
-    }
-
-    @AfterClass
-    public static void stopMiniAccumulo() throws IOException, InterruptedException {
-        client.close();
+    public void startMongo() {
+        client = super.getMongoClient();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
index 69ca274..f8ab40f 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/AbstractMongoIndexer.java
@@ -121,7 +121,6 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
 
     @Override
     public void close() throws IOException {
-        mongoClient.close();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoDbSmartUri.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoDbSmartUri.java b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoDbSmartUri.java
index b40c9b6..5d353b4 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoDbSmartUri.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/MongoDbSmartUri.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Entity;
@@ -159,13 +158,6 @@ public class MongoDbSmartUri implements SmartUriStorage {
     }
 
     /**
-     * Shutdown the MongoDB client.
-     */
-    public void shutdown() {
-        IOUtils.closeQuietly(mongoClient);
-    }
-
-    /**
      * @return the {@link EntityStorage}.
      */
     public EntityStorage getEntityStorage() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
index d2d9a54..874a40e 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
@@ -32,7 +32,7 @@ import org.apache.rya.indexing.entity.model.Property;
 import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
-import org.apache.rya.mongodb.MockMongoFactory;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
@@ -46,14 +46,13 @@ import org.openrdf.query.impl.MapBindingSet;
 import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import com.google.common.collect.ImmutableSet;
-import com.mongodb.MongoClient;
 
 import info.aduna.iteration.CloseableIteration;
 
 /**
  * Unit tests the methods of {@link EntityQueryNode}.
  */
-public class EntityQueryNodeTest {
+public class EntityQueryNodeTest extends MongoTestBase {
 
     private static final Type PERSON_TYPE =
             new Type(new RyaURI("urn:person"),
@@ -152,8 +151,7 @@ public class EntityQueryNodeTest {
 
     @Test
     public void evaluate_constantSubject() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EntityStorage storage = new MongoEntityStorage(client, "testDB");
+        final EntityStorage storage = new MongoEntityStorage(super.getMongoClient(), "testDB");
         final ValueFactory vf = ValueFactoryImpl.getInstance();
         final RyaURI subject = new RyaURI("urn:SSN:111-11-1111");
         final Entity entity = Entity.builder()
@@ -188,8 +186,7 @@ public class EntityQueryNodeTest {
 
     @Test
     public void evaluate_variableSubject() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EntityStorage storage = new MongoEntityStorage(client, "testDB");
+        final EntityStorage storage = new MongoEntityStorage(super.getMongoClient(), "testDB");
         final ValueFactory vf = ValueFactoryImpl.getInstance();
         RyaURI subject = new RyaURI("urn:SSN:111-11-1111");
         final Entity bob = Entity.builder()
@@ -242,8 +239,7 @@ public class EntityQueryNodeTest {
 
     @Test
     public void evaluate_constantObject() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        final EntityStorage storage = new MongoEntityStorage(client, "testDB");
+        final EntityStorage storage = new MongoEntityStorage(super.getMongoClient(), "testDB");
         final ValueFactory vf = ValueFactoryImpl.getInstance();
         final RyaURI subject = new RyaURI("urn:SSN:111-11-1111");
         final Entity entity = Entity.builder()

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoEntityStorageIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoEntityStorageIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoEntityStorageIT.java
index 5d26bc0..d8856fe 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoEntityStorageIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoEntityStorageIT.java
@@ -36,6 +36,7 @@ import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.EntityStorage.EntityAlreadyExistsException;
 import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
 import org.apache.rya.indexing.entity.storage.EntityStorage.StaleUpdateException;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.XMLSchema;
 
@@ -45,7 +46,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoEntityStorage}.
  */
-public class MongoEntityStorageIT extends MongoITBase {
+public class MongoEntityStorageIT extends MongoTestBase {
 
     private static final String RYA_INSTANCE_NAME = "testInstance";
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
deleted file mode 100644
index 692d1aa..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
+++ /dev/null
@@ -1,80 +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.rya.indexing.entity.storage.mongo;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-
-import com.mongodb.MongoClient;
-
-/**
- * A base class that may be used when implementing Mongo DB integration tests that
- * use the JUnit framework.
- */
-public class MongoITBase {
-
-    private MongoClient mongoClient = null;
-    private Set<String> originalDbNames = null;
-    protected MongoDBRdfConfiguration conf;
-
-    @Before
-    public void setupTest() throws Exception {
-        conf = new MongoDBRdfConfiguration( new Configuration() );
-        conf.setMongoDBName("testDB");
-        mongoClient = MockMongoFactory.newFactory().newMongoClient();
-        conf.setMongoClient(mongoClient);
-
-
-        // Store the names of the DBs that are present before running the test.
-        originalDbNames = new HashSet<>();
-        for(final String name : mongoClient.listDatabaseNames()) {
-            originalDbNames.add(name);
-        }
-    }
-
-    @After
-    public void cleanupTest() {
-        // Remove any DBs that were created by the test.
-        for(final String dbName : mongoClient.listDatabaseNames()) {
-            if(!originalDbNames.contains(dbName)) {
-                mongoClient.dropDatabase(dbName);
-            }
-        }
-    }
-
-    @AfterClass
-    public static void shutdown() {
-        MongoConnectorFactory.closeMongoClient();
-    }
-
-    /**
-     * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
-     */
-    public MongoClient getMongoClient() {
-        return mongoClient;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoTypeStorageIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoTypeStorageIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoTypeStorageIT.java
index 56c10c0..cff4f26 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoTypeStorageIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoTypeStorageIT.java
@@ -30,6 +30,7 @@ import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage.TypeStorageException;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
@@ -38,7 +39,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoTypeStorage}.
  */
-public class MongoTypeStorageIT extends MongoITBase {
+public class MongoTypeStorageIT extends MongoTestBase {
 
     private static final String RYA_INSTANCE_NAME = "testInstance";
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexerIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexerIT.java
index a33ddd4..875d54e 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexerIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexerIT.java
@@ -31,8 +31,8 @@ import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
 import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
-import org.apache.rya.indexing.entity.storage.mongo.MongoITBase;
 import org.apache.rya.indexing.entity.storage.mongo.MongoTypeStorage;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.RDF;
@@ -44,7 +44,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoEntityIndexer}.
  */
-public class MongoEntityIndexerIT extends MongoITBase {
+public class MongoEntityIndexerIT extends MongoTestBase {
 
     private static final String RYA_INSTANCE_NAME = "testDB";
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerTest2.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerTest2.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerTest2.java
index ba26a4b..35e4ca3 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerTest2.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerTest2.java
@@ -686,7 +686,8 @@ public class PrecompJoinOptimizerTest2 {
 					.getTupleExpr()));
 		}
 
-		Assert.assertTrue(qSet.containsAll(set) && eTupSet.size() == 1);
+		Assert.assertTrue(qSet.containsAll(set));
+		Assert.assertEquals(3, eTupSet.size());
 	}
 
 	@Test

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriIT.java
new file mode 100644
index 0000000..4dc0e8e
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriIT.java
@@ -0,0 +1,309 @@
+/*
+ * 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.rya.indexing.mongo;
+
+import static org.apache.rya.api.domain.RyaTypeUtils.booleanRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.byteRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.dateRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.doubleRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.floatRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.intRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.longRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.shortRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.stringRyaType;
+import static org.apache.rya.api.domain.RyaTypeUtils.uriRyaType;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.net.URISyntaxException;
+import java.util.Date;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaSchema;
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.indexing.entity.model.Entity;
+import org.apache.rya.indexing.entity.model.Property;
+import org.apache.rya.indexing.entity.model.Type;
+import org.apache.rya.indexing.entity.model.TypedEntity;
+import org.apache.rya.indexing.entity.query.EntityQueryNode;
+import org.apache.rya.indexing.entity.storage.mongo.ConvertingCursor;
+import org.apache.rya.indexing.mongodb.MongoDbSmartUri;
+import org.apache.rya.indexing.smarturi.SmartUriAdapter;
+import org.apache.rya.indexing.smarturi.SmartUriException;
+import org.apache.rya.mongodb.MongoTestBase;
+import org.joda.time.DateTime;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+import org.openrdf.query.algebra.helpers.StatementPatternCollector;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.ImmutableSet;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Tests for MongoDB based Smart URI.
+ */
+public class MongoDbSmartUriIT extends MongoTestBase {
+    private static final String NAMESPACE = RyaSchema.NAMESPACE;
+    private static final ValueFactory VALUE_FACTORY = ValueFactoryImpl.getInstance();
+
+    // People
+    private static final RyaURI BOB = createRyaUri("Bob");
+
+    // Attributes
+    private static final RyaURI HAS_WEIGHT = createRyaUri("hasWeight");
+    private static final RyaURI HAS_HEIGHT = createRyaUri("hasHeight");
+    private static final RyaURI HAS_SSN = createRyaUri("hasSSN");
+    private static final RyaURI HAS_AGE = createRyaUri("hasAge");
+    private static final RyaURI HAS_INCOME = createRyaUri("hasIncome");
+    private static final RyaURI HAS_NUMBER_OF_CHILDREN = createRyaUri("hasNumberOfChildren");
+    private static final RyaURI HAS_LICENSE_NUMBER = createRyaUri("hasLicenseNumber");
+    private static final RyaURI HAS_EYE_COLOR = createRyaUri("hasEyeColor");
+    private static final RyaURI HAS_HAIR_COLOR = createRyaUri("hasHairColor");
+    private static final RyaURI HAS_DATE_OF_BIRTH = createRyaUri("hasDateOfBirth");
+    private static final RyaURI HAS_EXPIRATION_DATE = createRyaUri("hasExpirationDate");
+    private static final RyaURI HAS_GLASSES = createRyaUri("hasGlasses");
+    private static final RyaURI HAS_EMAIL_ADDRESS = createRyaUri("hasEmailAddress");
+    private static final RyaURI HAS_ATTRIBUTE_SPACE = createRyaUri("has Attribute Space");
+    private static final RyaURI HAS_MOTTO = createRyaUri("hasMotto");
+    private static final RyaURI HAS_BLOOD_TYPE = createRyaUri("hasBloodType");
+    private static final RyaURI HAS_SEX = createRyaUri("hasSex");
+    private static final RyaURI HAS_ADDRESS = createRyaUri("hasAddress");
+    private static final RyaURI HAS_POSITION_TITLE = createRyaUri("hasPositionTitle");
+    private static final RyaURI HAS_WORK_ADDRESS = createRyaUri("hasWorkAddress");
+    private static final RyaURI HAS_EXTENSION = createRyaUri("hasExtension");
+    private static final RyaURI HAS_OFFICE_ROOM_NUMBER = createRyaUri("hasOfficeRoomNumber");
+
+    // Type URIs
+    private static final RyaURI PERSON_TYPE_URI = new RyaURI("urn:example/person");
+    private static final RyaURI EMPLOYEE_TYPE_URI = new RyaURI("urn:example/employee");
+
+    // Entities
+    private static final Entity BOB_ENTITY = createBobEntity();
+
+    // Types
+    private static final Type PERSON_TYPE = createPersonType();
+    private static final Type EMPLOYEE_TYPE = createEmployeeType();
+
+    private static MongoDbSmartUri smartUriConverter;
+
+    @Before
+    public void setup() throws Exception {
+        smartUriConverter = new MongoDbSmartUri(conf);
+    }
+
+    /**
+     * Creates a {@link RyaURI} for the specified local name.
+     * @param localName the URI's local name.
+     * @return the {@link RyraURI}.
+     */
+    private static RyaURI createRyaUri(final String localName) {
+        return createRyaUri(NAMESPACE, localName);
+    }
+
+    /**
+     * Creates a {@link RyaURI} for the specified local name.
+     * @param namespace the namespace.
+     * @param localName the URI's local name.
+     * @return the {@link RyraURI}.
+     */
+    private static RyaURI createRyaUri(final String namespace, final String localName) {
+        return RdfToRyaConversions.convertURI(VALUE_FACTORY.createURI(namespace, localName));
+    }
+
+    private static Entity createBobEntity() {
+        final Entity bobEntity = Entity.builder()
+            .setSubject(BOB)
+            .setExplicitType(PERSON_TYPE_URI)
+            .setExplicitType(EMPLOYEE_TYPE_URI)
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_WEIGHT, floatRyaType(250.75f)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_HEIGHT, doubleRyaType(72.5)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_SSN, stringRyaType("123-45-6789")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_AGE, shortRyaType((short) 40)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_INCOME, intRyaType(50000)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_NUMBER_OF_CHILDREN, byteRyaType((byte) 2)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_LICENSE_NUMBER, longRyaType(123456789012L)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_EYE_COLOR, stringRyaType("blue")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_HAIR_COLOR, stringRyaType("brown")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_DATE_OF_BIRTH, dateRyaType(new DateTime().minusYears(40))))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_EXPIRATION_DATE, dateRyaType(new Date())))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_GLASSES, booleanRyaType(true)))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_EMAIL_ADDRESS, uriRyaType(new URIImpl("mailto:bob.smitch00@gmail.com"))))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_ATTRIBUTE_SPACE, stringRyaType("attribute space")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_MOTTO, stringRyaType("!@#*\\&%20^ smörgåsbord")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_BLOOD_TYPE, stringRyaType("A+ blood type")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_SEX, stringRyaType("M")))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_ADDRESS, stringRyaType("123 Fake St. Washington, DC 20024")))
+            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_POSITION_TITLE, stringRyaType("Assistant to the Regional Manager")))
+            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_WORK_ADDRESS, stringRyaType("987 Fake Office Rd. Washington, DC 20024")))
+            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_EXTENSION, shortRyaType((short) 555)))
+            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_OFFICE_ROOM_NUMBER, shortRyaType((short) 9999)))
+            .build();
+
+        return bobEntity;
+    }
+
+    private static Type createPersonType() {
+         final Type personType = new Type(PERSON_TYPE_URI,
+            ImmutableSet.<RyaURI>builder()
+                .add(HAS_WEIGHT)
+                .add(HAS_HEIGHT)
+                .add(HAS_SSN)
+                .add(HAS_AGE)
+                .add(HAS_INCOME)
+                .add(HAS_NUMBER_OF_CHILDREN)
+                .add(HAS_LICENSE_NUMBER)
+                .add(HAS_EYE_COLOR)
+                .add(HAS_HAIR_COLOR)
+                .add(HAS_DATE_OF_BIRTH)
+                .add(HAS_EXPIRATION_DATE)
+                .add(HAS_GLASSES)
+                .add(HAS_EMAIL_ADDRESS)
+                .add(HAS_ATTRIBUTE_SPACE)
+                .add(HAS_MOTTO)
+                .add(HAS_BLOOD_TYPE)
+                .add(HAS_SEX)
+                .add(HAS_ADDRESS)
+                .build());
+         return personType;
+    }
+
+    private static Type createEmployeeType() {
+        final Type employeeType = new Type(EMPLOYEE_TYPE_URI,
+            ImmutableSet.<RyaURI>builder()
+                .add(HAS_POSITION_TITLE)
+                .add(HAS_WORK_ADDRESS)
+                .add(HAS_EXTENSION)
+                .add(HAS_OFFICE_ROOM_NUMBER)
+                .build());
+        return employeeType;
+    }
+
+    private static String getRyaUriLocalName(final RyaURI ryaUri) {
+        return new URIImpl(ryaUri.getData()).getLocalName();
+    }
+
+    @Test
+    public void testSerializeDeserialize() throws SmartUriException, URISyntaxException {
+        final URI smartUri = SmartUriAdapter.serializeUriEntity(BOB_ENTITY);
+        final Entity resultEntity = SmartUriAdapter.deserializeUriEntity(smartUri);
+        assertEquals(BOB_ENTITY.getSubject(), resultEntity.getSubject());
+    }
+
+    @Test
+    public void testStorage() throws SmartUriException, MalformedQueryException, RuntimeException, QueryEvaluationException {
+        smartUriConverter.storeEntity(BOB_ENTITY);
+
+        final String sparql = "SELECT * WHERE { " +
+            "<" + BOB.getData() + "> <" + RDF.TYPE + "> <" + PERSON_TYPE.getId().getData() + "> . " +
+            "<" + BOB.getData() + "> <" + HAS_SSN.getData() + "> ?ssn . " +
+            "<" + BOB.getData() + "> <" + HAS_AGE.getData() + "> ?age . " +
+            "<" + BOB.getData() + "> <" + HAS_WEIGHT.getData() + "> ?weight . " +
+            "<" + BOB.getData() + "> <" + HAS_ADDRESS.getData() + "> ?address . " +
+        "}";
+
+        final StatementPatternCollector spCollector = new StatementPatternCollector();
+        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
+        final List<StatementPattern> patterns = spCollector.getStatementPatterns();
+        final EntityQueryNode entityQueryNode = new EntityQueryNode(PERSON_TYPE, patterns, smartUriConverter.getEntityStorage());
+        final QueryBindingSet queryBindingSet = new QueryBindingSet();
+        final Property ssnProperty = BOB_ENTITY.lookupTypeProperty(PERSON_TYPE, HAS_SSN).get();
+        queryBindingSet.addBinding(HAS_SSN.getData(), RyaToRdfConversions.convertValue(ssnProperty.getValue()));
+
+        final CloseableIteration<BindingSet, QueryEvaluationException> iter = entityQueryNode.evaluate(queryBindingSet);
+        int count = 0;
+        // These should match what was used in the SPARQL query.
+        final List<String> queryParamNames = Lists.newArrayList("ssn", "age", "weight", "address");
+        while (iter.hasNext()) {
+            final BindingSet bs = iter.next();
+            assertTrue(bs.getBindingNames().containsAll(queryParamNames));
+            count++;
+        }
+        assertEquals(count, 1);
+    }
+
+    @Test
+    public void testUpdate() throws SmartUriException {
+        smartUriConverter.storeEntity(BOB_ENTITY);
+
+        // New properties to add
+        final RyaURI hasNickName = createRyaUri("hasNickName");
+        final RyaURI hasWindowOffice = createRyaUri("hasWindowOffice");
+
+        final Entity.Builder builder = Entity.builder(BOB_ENTITY);
+        builder.setProperty(PERSON_TYPE_URI, new Property(HAS_AGE, shortRyaType((short) 41)));
+        builder.setProperty(PERSON_TYPE_URI, new Property(hasNickName, stringRyaType("Bobby")));
+        builder.setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_POSITION_TITLE, stringRyaType("Assistant Regional Manager")));
+        builder.setProperty(EMPLOYEE_TYPE_URI, new Property(hasWindowOffice, booleanRyaType(true)));
+        builder.setVersion(BOB_ENTITY.getVersion() + 1);
+        builder.rebuildSmartUri();
+
+        final Entity newBobEntity = builder.build();
+
+        smartUriConverter.updateEntity(BOB_ENTITY, newBobEntity);
+
+        final Entity resultEntity = smartUriConverter.queryEntity(BOB_ENTITY.getSubject());
+        assertEquals(newBobEntity.getVersion(), resultEntity.getVersion());
+        assertEquals(newBobEntity.lookupTypeProperty(PERSON_TYPE, HAS_AGE), resultEntity.lookupTypeProperty(PERSON_TYPE, HAS_AGE));
+        assertEquals(newBobEntity.lookupTypeProperty(PERSON_TYPE, hasNickName), resultEntity.lookupTypeProperty(PERSON_TYPE, hasNickName));
+        assertEquals(newBobEntity.lookupTypeProperty(EMPLOYEE_TYPE, HAS_POSITION_TITLE), resultEntity.lookupTypeProperty(EMPLOYEE_TYPE, HAS_POSITION_TITLE));
+        assertEquals(newBobEntity.lookupTypeProperty(EMPLOYEE_TYPE, hasWindowOffice), resultEntity.lookupTypeProperty(EMPLOYEE_TYPE, hasWindowOffice));
+        assertEquals(newBobEntity.getSmartUri(), resultEntity.getSmartUri());
+        final String resultUriString = resultEntity.getSmartUri().stringValue();
+        assertTrue(resultUriString.contains(getRyaUriLocalName(hasWindowOffice)));
+        assertTrue(resultUriString.contains(getRyaUriLocalName(hasNickName)));
+    }
+
+    @Test
+    public void testQuery() throws SmartUriException {
+        smartUriConverter.storeEntity(BOB_ENTITY);
+
+        // Look up Person Type Entities that match Bob's SSN property
+        final Set<Property> properties = new LinkedHashSet<>();
+        properties.add(BOB_ENTITY.lookupTypeProperty(PERSON_TYPE, HAS_SSN).get());
+        final Map<URI, Value> map = SmartUriAdapter.propertiesToMap(properties);
+
+        final ConvertingCursor<TypedEntity> cursor = smartUriConverter.queryEntity(PERSON_TYPE, map);
+        int count = 0;
+        while (cursor.hasNext()) {
+            final TypedEntity typedEntity = cursor.next();
+            System.out.println(typedEntity);
+            count++;
+        }
+        assertEquals(count, 1);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriTest.java
deleted file mode 100644
index dff271f..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoDbSmartUriTest.java
+++ /dev/null
@@ -1,332 +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.rya.indexing.mongo;
-
-import static org.apache.rya.api.domain.RyaTypeUtils.booleanRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.byteRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.dateRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.doubleRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.floatRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.intRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.longRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.shortRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.stringRyaType;
-import static org.apache.rya.api.domain.RyaTypeUtils.uriRyaType;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.net.URISyntaxException;
-import java.util.Date;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.rya.api.domain.RyaSchema;
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.api.resolver.RdfToRyaConversions;
-import org.apache.rya.api.resolver.RyaToRdfConversions;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.entity.model.Entity;
-import org.apache.rya.indexing.entity.model.Property;
-import org.apache.rya.indexing.entity.model.Type;
-import org.apache.rya.indexing.entity.model.TypedEntity;
-import org.apache.rya.indexing.entity.query.EntityQueryNode;
-import org.apache.rya.indexing.entity.storage.mongo.ConvertingCursor;
-import org.apache.rya.indexing.mongodb.MongoDbSmartUri;
-import org.apache.rya.indexing.smarturi.SmartUriAdapter;
-import org.apache.rya.indexing.smarturi.SmartUriException;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.joda.time.DateTime;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.evaluation.QueryBindingSet;
-import org.openrdf.query.algebra.helpers.StatementPatternCollector;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import com.beust.jcommander.internal.Lists;
-import com.google.common.collect.ImmutableSet;
-import com.mongodb.MongoClient;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Tests for MongoDB based Smart URI.
- */
-public class MongoDbSmartUriTest {
-    private static final String NAMESPACE = RyaSchema.NAMESPACE;
-    private static final ValueFactory VALUE_FACTORY = ValueFactoryImpl.getInstance();
-
-    // People
-    private static final RyaURI BOB = createRyaUri("Bob");
-
-    // Attributes
-    private static final RyaURI HAS_WEIGHT = createRyaUri("hasWeight");
-    private static final RyaURI HAS_HEIGHT = createRyaUri("hasHeight");
-    private static final RyaURI HAS_SSN = createRyaUri("hasSSN");
-    private static final RyaURI HAS_AGE = createRyaUri("hasAge");
-    private static final RyaURI HAS_INCOME = createRyaUri("hasIncome");
-    private static final RyaURI HAS_NUMBER_OF_CHILDREN = createRyaUri("hasNumberOfChildren");
-    private static final RyaURI HAS_LICENSE_NUMBER = createRyaUri("hasLicenseNumber");
-    private static final RyaURI HAS_EYE_COLOR = createRyaUri("hasEyeColor");
-    private static final RyaURI HAS_HAIR_COLOR = createRyaUri("hasHairColor");
-    private static final RyaURI HAS_DATE_OF_BIRTH = createRyaUri("hasDateOfBirth");
-    private static final RyaURI HAS_EXPIRATION_DATE = createRyaUri("hasExpirationDate");
-    private static final RyaURI HAS_GLASSES = createRyaUri("hasGlasses");
-    private static final RyaURI HAS_EMAIL_ADDRESS = createRyaUri("hasEmailAddress");
-    private static final RyaURI HAS_ATTRIBUTE_SPACE = createRyaUri("has Attribute Space");
-    private static final RyaURI HAS_MOTTO = createRyaUri("hasMotto");
-    private static final RyaURI HAS_BLOOD_TYPE = createRyaUri("hasBloodType");
-    private static final RyaURI HAS_SEX = createRyaUri("hasSex");
-    private static final RyaURI HAS_ADDRESS = createRyaUri("hasAddress");
-    private static final RyaURI HAS_POSITION_TITLE = createRyaUri("hasPositionTitle");
-    private static final RyaURI HAS_WORK_ADDRESS = createRyaUri("hasWorkAddress");
-    private static final RyaURI HAS_EXTENSION = createRyaUri("hasExtension");
-    private static final RyaURI HAS_OFFICE_ROOM_NUMBER = createRyaUri("hasOfficeRoomNumber");
-
-    // Type URIs
-    private static final RyaURI PERSON_TYPE_URI = new RyaURI("urn:example/person");
-    private static final RyaURI EMPLOYEE_TYPE_URI = new RyaURI("urn:example/employee");
-
-    // Entities
-    private static final Entity BOB_ENTITY = createBobEntity();
-
-    // Types
-    private static final Type PERSON_TYPE = createPersonType();
-    private static final Type EMPLOYEE_TYPE = createEmployeeType();
-
-    private static MongoDBRdfConfiguration conf;
-    private static MongoDbSmartUri smartUriConverter;
-
-    @BeforeClass
-    public static void setupClass() throws Exception {
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.setTablePrefix("another_");
-    }
-
-    @Before
-    public void setupTest() throws Exception {
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        conf.setMongoClient(client);
-
-        if (smartUriConverter != null) {
-            smartUriConverter.shutdown();
-        }
-        smartUriConverter = new MongoDbSmartUri(conf);
-    }
-
-    /**
-     * Creates a {@link RyaURI} for the specified local name.
-     * @param localName the URI's local name.
-     * @return the {@link RyraURI}.
-     */
-    private static RyaURI createRyaUri(final String localName) {
-        return createRyaUri(NAMESPACE, localName);
-        //return new RyaURI("http://" + localName);
-    }
-
-    /**
-     * Creates a {@link RyaURI} for the specified local name.
-     * @param namespace the namespace.
-     * @param localName the URI's local name.
-     * @return the {@link RyraURI}.
-     */
-    private static RyaURI createRyaUri(final String namespace, final String localName) {
-        return RdfToRyaConversions.convertURI(VALUE_FACTORY.createURI(namespace, localName));
-    }
-
-    private static Entity createBobEntity() {
-        final Entity bobEntity = Entity.builder()
-            .setSubject(BOB)
-            .setExplicitType(PERSON_TYPE_URI)
-            .setExplicitType(EMPLOYEE_TYPE_URI)
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_WEIGHT, floatRyaType(250.75f)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_HEIGHT, doubleRyaType(72.5)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_SSN, stringRyaType("123-45-6789")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_AGE, shortRyaType((short) 40)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_INCOME, intRyaType(50000)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_NUMBER_OF_CHILDREN, byteRyaType((byte) 2)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_LICENSE_NUMBER, longRyaType(123456789012L)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_EYE_COLOR, stringRyaType("blue")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_HAIR_COLOR, stringRyaType("brown")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_DATE_OF_BIRTH, dateRyaType(new DateTime().minusYears(40))))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_EXPIRATION_DATE, dateRyaType(new Date())))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_GLASSES, booleanRyaType(true)))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_EMAIL_ADDRESS, uriRyaType(new URIImpl("mailto:bob.smitch00@gmail.com"))))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_ATTRIBUTE_SPACE, stringRyaType("attribute space")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_MOTTO, stringRyaType("!@#*\\&%20^ smörgåsbord")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_BLOOD_TYPE, stringRyaType("A+ blood type")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_SEX, stringRyaType("M")))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_ADDRESS, stringRyaType("123 Fake St. Washington, DC 20024")))
-            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_POSITION_TITLE, stringRyaType("Assistant to the Regional Manager")))
-            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_WORK_ADDRESS, stringRyaType("987 Fake Office Rd. Washington, DC 20024")))
-            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_EXTENSION, shortRyaType((short) 555)))
-            .setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_OFFICE_ROOM_NUMBER, shortRyaType((short) 9999)))
-            .build();
-
-        return bobEntity;
-    }
-
-    private static Type createPersonType() {
-         final Type personType = new Type(PERSON_TYPE_URI,
-            ImmutableSet.<RyaURI>builder()
-                .add(HAS_WEIGHT)
-                .add(HAS_HEIGHT)
-                .add(HAS_SSN)
-                .add(HAS_AGE)
-                .add(HAS_INCOME)
-                .add(HAS_NUMBER_OF_CHILDREN)
-                .add(HAS_LICENSE_NUMBER)
-                .add(HAS_EYE_COLOR)
-                .add(HAS_HAIR_COLOR)
-                .add(HAS_DATE_OF_BIRTH)
-                .add(HAS_EXPIRATION_DATE)
-                .add(HAS_GLASSES)
-                .add(HAS_EMAIL_ADDRESS)
-                .add(HAS_ATTRIBUTE_SPACE)
-                .add(HAS_MOTTO)
-                .add(HAS_BLOOD_TYPE)
-                .add(HAS_SEX)
-                .add(HAS_ADDRESS)
-                .build());
-         return personType;
-    }
-
-    private static Type createEmployeeType() {
-        final Type employeeType = new Type(EMPLOYEE_TYPE_URI,
-            ImmutableSet.<RyaURI>builder()
-                .add(HAS_POSITION_TITLE)
-                .add(HAS_WORK_ADDRESS)
-                .add(HAS_EXTENSION)
-                .add(HAS_OFFICE_ROOM_NUMBER)
-                .build());
-        return employeeType;
-    }
-
-    private static String getRyaUriLocalName(final RyaURI ryaUri) {
-        return new URIImpl(ryaUri.getData()).getLocalName();
-    }
-
-    @Test
-    public void testSerializeDeserialize() throws SmartUriException, URISyntaxException {
-        final URI smartUri = SmartUriAdapter.serializeUriEntity(BOB_ENTITY);
-        System.out.println(smartUri);
-        final Entity resultEntity = SmartUriAdapter.deserializeUriEntity(smartUri);
-        System.out.println(resultEntity);
-        assertEquals(BOB_ENTITY.getSubject(), resultEntity.getSubject());
-    }
-
-    @Test
-    public void testStorage() throws SmartUriException, MalformedQueryException, RuntimeException, QueryEvaluationException {
-        smartUriConverter.storeEntity(BOB_ENTITY);
-
-        final String sparql = "SELECT * WHERE { " +
-            "<" + BOB.getData() + "> <" + RDF.TYPE + "> <" + PERSON_TYPE.getId().getData() + "> . " +
-            "<" + BOB.getData() + "> <" + HAS_SSN.getData() + "> ?ssn . " +
-            "<" + BOB.getData() + "> <" + HAS_AGE.getData() + "> ?age . " +
-            "<" + BOB.getData() + "> <" + HAS_WEIGHT.getData() + "> ?weight . " +
-            "<" + BOB.getData() + "> <" + HAS_ADDRESS.getData() + "> ?address . " +
-        "}";
-
-        final StatementPatternCollector spCollector = new StatementPatternCollector();
-        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
-        final List<StatementPattern> patterns = spCollector.getStatementPatterns();
-        final EntityQueryNode entityQueryNode = new EntityQueryNode(PERSON_TYPE, patterns, smartUriConverter.getEntityStorage());
-        final QueryBindingSet queryBindingSet = new QueryBindingSet();
-        final Property ssnProperty = BOB_ENTITY.lookupTypeProperty(PERSON_TYPE, HAS_SSN).get();
-        queryBindingSet.addBinding(HAS_SSN.getData(), RyaToRdfConversions.convertValue(ssnProperty.getValue()));
-
-        final CloseableIteration<BindingSet, QueryEvaluationException> iter = entityQueryNode.evaluate(queryBindingSet);
-        int count = 0;
-        // These should match what was used in the SPARQL query.
-        final List<String> queryParamNames = Lists.newArrayList("ssn", "age", "weight", "address");
-        while (iter.hasNext()) {
-            final BindingSet bs = iter.next();
-            assertTrue(bs.getBindingNames().containsAll(queryParamNames));
-            count++;
-        }
-        assertEquals(count, 1);
-    }
-
-    @Test
-    public void testUpdate() throws SmartUriException {
-        smartUriConverter.storeEntity(BOB_ENTITY);
-
-        // New properties to add
-        final RyaURI hasNickName = createRyaUri("hasNickName");
-        final RyaURI hasWindowOffice = createRyaUri("hasWindowOffice");
-
-        final Entity.Builder builder = Entity.builder(BOB_ENTITY);
-        builder.setProperty(PERSON_TYPE_URI, new Property(HAS_AGE, shortRyaType((short) 41)));
-        builder.setProperty(PERSON_TYPE_URI, new Property(hasNickName, stringRyaType("Bobby")));
-        builder.setProperty(EMPLOYEE_TYPE_URI, new Property(HAS_POSITION_TITLE, stringRyaType("Assistant Regional Manager")));
-        builder.setProperty(EMPLOYEE_TYPE_URI, new Property(hasWindowOffice, booleanRyaType(true)));
-        builder.setVersion(BOB_ENTITY.getVersion() + 1);
-        builder.rebuildSmartUri();
-
-        final Entity newBobEntity = builder.build();
-
-        smartUriConverter.updateEntity(BOB_ENTITY, newBobEntity);
-
-        final Entity resultEntity = smartUriConverter.queryEntity(BOB_ENTITY.getSubject());
-        assertEquals(newBobEntity.getVersion(), resultEntity.getVersion());
-        assertEquals(newBobEntity.lookupTypeProperty(PERSON_TYPE, HAS_AGE), resultEntity.lookupTypeProperty(PERSON_TYPE, HAS_AGE));
-        assertEquals(newBobEntity.lookupTypeProperty(PERSON_TYPE, hasNickName), resultEntity.lookupTypeProperty(PERSON_TYPE, hasNickName));
-        assertEquals(newBobEntity.lookupTypeProperty(EMPLOYEE_TYPE, HAS_POSITION_TITLE), resultEntity.lookupTypeProperty(EMPLOYEE_TYPE, HAS_POSITION_TITLE));
-        assertEquals(newBobEntity.lookupTypeProperty(EMPLOYEE_TYPE, hasWindowOffice), resultEntity.lookupTypeProperty(EMPLOYEE_TYPE, hasWindowOffice));
-        assertEquals(newBobEntity.getSmartUri(), resultEntity.getSmartUri());
-        final String resultUriString = resultEntity.getSmartUri().stringValue();
-        assertTrue(resultUriString.contains(getRyaUriLocalName(hasWindowOffice)));
-        assertTrue(resultUriString.contains(getRyaUriLocalName(hasNickName)));
-    }
-
-    @Test
-    public void testQuery() throws SmartUriException {
-        smartUriConverter.storeEntity(BOB_ENTITY);
-
-        // Look up Person Type Entities that match Bob's SSN property
-        final Set<Property> properties = new LinkedHashSet<>();
-        properties.add(BOB_ENTITY.lookupTypeProperty(PERSON_TYPE, HAS_SSN).get());
-        final Map<URI, Value> map = SmartUriAdapter.propertiesToMap(properties);
-
-        final ConvertingCursor<TypedEntity> cursor = smartUriConverter.queryEntity(PERSON_TYPE, map);
-        int count = 0;
-        while (cursor.hasNext()) {
-            final TypedEntity typedEntity = cursor.next();
-            System.out.println(typedEntity);
-            count++;
-        }
-        assertEquals(count, 1);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexIT.java
index 6fac386..019760d 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexIT.java
@@ -25,7 +25,6 @@ import java.util.HashSet;
 import java.util.Optional;
 import java.util.Set;
 
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.entity.model.Entity;
@@ -33,11 +32,8 @@ import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
 import org.apache.rya.indexing.entity.update.mongo.MongoEntityIndexer;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.apache.rya.sail.config.RyaSailFactory;
-import org.bson.Document;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,31 +51,16 @@ import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
 import com.google.common.collect.ImmutableSet;
-import com.mongodb.Block;
-import com.mongodb.MongoClient;
-import com.mongodb.client.MongoDatabase;
 
-import de.flapdoodle.embed.mongo.distribution.Version;
-
-public class MongoEntityIndexIT {
+public class MongoEntityIndexIT extends MongoTestBase {
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();
-    private MongoDBRdfConfiguration conf;
     private SailRepositoryConnection conn;
     private MongoEntityIndexer indexer;
-    private MongoClient mongoClient;
 
     @Before
     public void setUp() throws Exception {
-        mongoClient = MockMongoFactory.with(Version.Main.PRODUCTION).newMongoClient();
-        conf = new MongoDBRdfConfiguration();
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya");
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya");
         conf.setBoolean(ConfigUtils.USE_MONGO, true);
-        conf.setMongoClient(mongoClient);
         conf.setBoolean(ConfigUtils.USE_ENTITY, true);
-        final int port = mongoClient.getServerAddressList().get(0).getPort();
-        conf.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, Integer.toString(port));
 
         final Sail sail = RyaSailFactory.getInstance(conf);
         conn = new SailRepository(sail).getConnection();
@@ -92,9 +73,6 @@ public class MongoEntityIndexIT {
 
     @After
     public void tearDown() throws Exception {
-        if (mongoClient != null) {
-            MongoConnectorFactory.closeMongoClient();
-        }
         if (conn != null) {
             conn.clear();
         }
@@ -108,12 +86,6 @@ public class MongoEntityIndexIT {
         setupTypes();
         addStatements();
 
-        final MongoDatabase db = mongoClient.getDatabase("test");
-        db.listCollections().forEach((Block<Document>)doc -> {
-            System.out.println(doc);
-        });
-
-
         final EntityStorage entities = indexer.getEntityStorage(conf);
         final RyaURI subject = new RyaURI("urn:alice");
         final Optional<Entity> alice = entities.get(subject);
@@ -136,7 +108,6 @@ public class MongoEntityIndexIT {
         final Set<BindingSet> results = new HashSet<>();
         while(rez.hasNext()) {
             final BindingSet bs = rez.next();
-            System.out.println(bs);
             results.add(bs);
         }
         final MapBindingSet expected = new MapBindingSet();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/e5738966/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java
index f4aeaa1..850640f 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java
@@ -22,7 +22,6 @@ import java.util.List;
 
 import org.apache.rya.api.domain.RyaType;
 import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.entity.EntityIndexOptimizer;
 import org.apache.rya.indexing.entity.model.Entity;
 import org.apache.rya.indexing.entity.model.Property;
@@ -30,8 +29,7 @@ import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.query.EntityQueryNode;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
-import org.apache.rya.mongodb.MockMongoFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -45,9 +43,8 @@ import org.openrdf.query.algebra.helpers.StatementPatternCollector;
 import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import com.google.common.collect.ImmutableSet;
-import com.mongodb.MongoClient;
 
-public class MongoEntityIndexTest {
+public class MongoEntityIndexTest extends MongoTestBase {
     private static final Type PERSON_TYPE =
             new Type(new RyaURI("urn:person"),
                 ImmutableSet.<RyaURI>builder()
@@ -57,21 +54,11 @@ public class MongoEntityIndexTest {
                     .build());
     private static final RyaURI RYA_PERSON_TYPE = new RyaURI("urn:person");
 
-    static MongoDBRdfConfiguration conf;
     private static EntityIndexOptimizer optimizer;
     private static EntityStorage entityStorage;
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        conf = new MongoDBRdfConfiguration();
-        conf.set(ConfigUtils.USE_MONGO, "true");
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
-        conf.setTablePrefix("another_");
-
-        final MongoClient client = MockMongoFactory.newFactory().newMongoClient();
-        conf.setMongoClient(client);
-
         optimizer = new EntityIndexOptimizer();
         optimizer.setConf(conf);
 


[5/5] incubator-rya git commit: RYA-332 rename integration tests. Closes #212.

Posted by ca...@apache.org.
RYA-332 rename integration tests.  Closes #212.

Some integration tests were being ignored
since the class names ended with
"IntegrationTest" instead of "IT"


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

Branch: refs/heads/master
Commit: 9c12630bb41ddcd5756b224fe6f0aeba0455fd6a
Parents: e573896
Author: isper3at <sm...@gmail.com>
Authored: Tue Aug 22 15:28:18 2017 -0400
Committer: Caleb Meier <ca...@parsons.com>
Committed: Fri Aug 25 09:44:18 2017 -0700

----------------------------------------------------------------------
 .../pcj/matching/PCJOptimizerUtilities.java     |   25 +-
 .../external/AccumuloConstantPcjIT.java         |  414 +++++
 .../AccumuloConstantPcjIntegrationTest.java     |  414 -----
 .../rya/indexing/external/AccumuloPcjIT.java    | 1440 ++++++++++++++++++
 .../external/AccumuloPcjIntegrationTest.java    | 1440 ------------------
 .../indexing/external/PCJOptionalTestIT.java    |    2 +-
 .../external/PrecompJoinOptimizerIT.java        |  505 ++++++
 .../PrecompJoinOptimizerIntegrationTest.java    |  505 ------
 .../pcj/storage/accumulo/PcjTablesIT.java       |  573 +++++++
 .../accumulo/PcjTablesIntegrationTest.java      |  573 -------
 .../rya/camel/cbsail/CbSailIntegrationTest.java |  117 --
 11 files changed, 2949 insertions(+), 3059 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java
index 909d932..1ad03b6 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJOptimizerUtilities.java
@@ -291,15 +291,22 @@ public class PCJOptimizerUtilities {
 		}
 
 		@Override
-		public void meet(Union union) {
-			if (Sets.intersection(union.getRightArg().getBindingNames(), filterVars).size() > 0) {
-				relocate(filter, union.getRightArg());
-			} else if (Sets.intersection(union.getLeftArg().getBindingNames(), filterVars).size() > 0) {
-				Filter clone = new Filter(filter.getArg(), filter
-						.getCondition().clone());
-				relocate(clone, union.getLeftArg());
-			}
-		}
+        public void meet(Union union) {
+            boolean filterMoved = false;
+            if (Sets.intersection(union.getRightArg().getBindingNames(), filterVars).size() > 0) {
+                relocate(filter, union.getRightArg());
+                filterMoved = true;
+            }
+ 
+            if (Sets.intersection(union.getLeftArg().getBindingNames(), filterVars).size() > 0) {
+                if (filterMoved) {
+                    Filter clone = new Filter(filter.getArg(), filter.getCondition().clone());
+                    relocate(clone, union.getLeftArg());
+                } else {
+                    relocate(filter, union.getLeftArg());
+                }
+            }
+        }
 
 		@Override
 		public void meet(Difference node) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
new file mode 100644
index 0000000..8c55e98
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
@@ -0,0 +1,414 @@
+package org.apache.rya.indexing.external;
+
+import java.net.UnknownHostException;
+
+/*
+ * 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 java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.rya.indexing.pcj.storage.PcjException;
+import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.QueryResultHandlerException;
+import org.openrdf.query.TupleQueryResultHandler;
+import org.openrdf.query.TupleQueryResultHandlerException;
+import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.SailException;
+
+import com.google.common.base.Optional;
+
+import org.apache.rya.api.persist.RyaDAOException;
+import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
+
+public class AccumuloConstantPcjIT {
+
+	private SailRepositoryConnection conn, pcjConn;
+	private SailRepository repo, pcjRepo;
+	private Connector accCon;
+	String prefix = "table_";
+	String tablename = "table_INDEX_";
+	URI obj, obj2, subclass, subclass2, talksTo;
+
+	@Before
+	public void init() throws RepositoryException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, RyaDAOException, InferenceEngineException,
+			NumberFormatException, UnknownHostException, SailException {
+
+		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
+		conn = repo.getConnection();
+
+		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
+		pcjConn = pcjRepo.getConnection();
+
+		final URI sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		conn.add(sub, RDF.TYPE, subclass);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(sub, talksTo, obj);
+
+		final URI sub2 = new URIImpl("uri:entity2");
+		subclass2 = new URIImpl("uri:class2");
+		obj2 = new URIImpl("uri:obj2");
+
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub2, talksTo, obj2);
+
+		accCon = new MockInstance("instance").getConnector("root",new PasswordToken(""));
+
+	}
+
+	@After
+	public void close() throws RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException {
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 2, prefix);
+
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexVarInstantiate1() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			MalformedQueryException, SailException, QueryEvaluationException,
+			TupleQueryResultHandlerException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  <uri:entity> a ?c . "//
+				+ "  <uri:entity> <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  <uri:entity> <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh2);
+
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateThreeIndexVarInstantiate() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			MalformedQueryException, SailException, QueryEvaluationException,
+			TupleQueryResultHandlerException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		final URI sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		final URI howlsAt = new URIImpl("uri:howlsAt");
+		final URI subType = new URIImpl("uri:subType");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, howlsAt, superclass);
+		conn.add(superclass, subType, obj);
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String indexSparqlString3 = ""//
+				+ "SELECT ?wolf ?sheep ?chicken  " //
+				+ "{" //
+				+ "  ?wolf <uri:howlsAt> ?sheep . "//
+				+ "  ?sheep <uri:subType> ?chicken. "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  <uri:entity> a ?c . "//
+				+ "  <uri:entity> <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  <uri:entity> <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "  <uri:entity> <uri:howlsAt> ?f. "//
+				+ "  ?f <uri:subType> <uri:obj>. "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 3,
+				indexSparqlString3,
+				new String[] { "wolf", "sheep", "chicken" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateFilterInstantiate() throws RepositoryException,
+			PcjException, MalformedQueryException, SailException,
+			QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, AccumuloException,
+			AccumuloSecurityException {
+
+		final URI e1 = new URIImpl("uri:e1");
+		final URI e2 = new URIImpl("uri:e2");
+		final URI e3 = new URIImpl("uri:e3");
+		final URI f1 = new URIImpl("uri:f1");
+		final URI f2 = new URIImpl("uri:f2");
+		final URI f3 = new URIImpl("uri:f3");
+		final URI g1 = new URIImpl("uri:g1");
+		final URI g2 = new URIImpl("uri:g2");
+		final URI g3 = new URIImpl("uri:g3");
+
+		conn.add(e1, talksTo, f1);
+		conn.add(f1, talksTo, g1);
+		conn.add(g1, talksTo, e1);
+		conn.add(e2, talksTo, f2);
+		conn.add(f2, talksTo, g2);
+		conn.add(g2, talksTo, e2);
+		conn.add(e3, talksTo, f3);
+		conn.add(f3, talksTo, g3);
+		conn.add(g3, talksTo, e3);
+
+		final String queryString = ""//
+				+ "SELECT ?x ?y ?z " //
+				+ "{" //
+				+ "Filter(?x = <uri:e1>) . " //
+				+ " ?x <uri:talksTo> ?y. " //
+				+ " ?y <uri:talksTo> ?z. " //
+				+ " ?z <uri:talksTo> <uri:e1>. " //
+				+ "}";//
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?a ?b ?c ?d " //
+				+ "{" //
+				+ "Filter(?a = ?d) . " //
+				+ " ?a <uri:talksTo> ?b. " //
+				+ " ?b <uri:talksTo> ?c. " //
+				+ " ?c <uri:talksTo> ?d. " //
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "a", "b", "c", "d" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh2);
+
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateCompoundFilterInstantiate()
+			throws RepositoryException, PcjException, MalformedQueryException,
+			SailException, QueryEvaluationException,
+			TableNotFoundException,
+			TupleQueryResultHandlerException, AccumuloException, AccumuloSecurityException {
+
+		final URI e1 = new URIImpl("uri:e1");
+		final URI f1 = new URIImpl("uri:f1");
+
+		conn.add(e1, talksTo, e1);
+		conn.add(e1, talksTo, f1);
+		conn.add(f1, talksTo, e1);
+
+		final String queryString = ""//
+				+ "SELECT ?x ?y ?z " //
+				+ "{" //
+				+ "Filter(?x = <uri:e1> && ?y = <uri:e1>) . " //
+				+ " ?x <uri:talksTo> ?y. " //
+				+ " ?y <uri:talksTo> ?z. " //
+				+ " ?z <uri:talksTo> <uri:e1>. " //
+				+ "}";//
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?a ?b ?c ?d " //
+				+ "{" //
+				+ "Filter(?a = ?d && ?b = ?d) . " //
+				+ " ?a <uri:talksTo> ?b. " //
+				+ " ?b <uri:talksTo> ?c. " //
+				+ " ?c <uri:talksTo> ?d. " //
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "a", "b", "c", "d" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(2, crh1.getCount());
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	public static class CountingResultHandler implements
+			TupleQueryResultHandler {
+		private int count = 0;
+
+		public int getCount() {
+			return count;
+		}
+
+		public void resetCount() {
+			count = 0;
+		}
+
+		@Override
+		public void startQueryResult(final List<String> arg0)
+				throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleSolution(final BindingSet arg0)
+				throws TupleQueryResultHandlerException {
+			count++;
+		}
+
+		@Override
+		public void endQueryResult() throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleBoolean(final boolean arg0)
+				throws QueryResultHandlerException {
+		}
+
+		@Override
+		public void handleLinks(final List<String> arg0)
+				throws QueryResultHandlerException {
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIntegrationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIntegrationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIntegrationTest.java
deleted file mode 100644
index 0c25c14..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIntegrationTest.java
+++ /dev/null
@@ -1,414 +0,0 @@
-package org.apache.rya.indexing.external;
-
-import java.net.UnknownHostException;
-
-/*
- * 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 java.util.List;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.rya.indexing.pcj.storage.PcjException;
-import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.openrdf.model.URI;
-import org.openrdf.model.impl.LiteralImpl;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.QueryLanguage;
-import org.openrdf.query.QueryResultHandlerException;
-import org.openrdf.query.TupleQueryResultHandler;
-import org.openrdf.query.TupleQueryResultHandlerException;
-import org.openrdf.repository.RepositoryException;
-import org.openrdf.repository.sail.SailRepository;
-import org.openrdf.repository.sail.SailRepositoryConnection;
-import org.openrdf.sail.SailException;
-
-import com.google.common.base.Optional;
-
-import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
-
-public class AccumuloConstantPcjIntegrationTest {
-
-	private SailRepositoryConnection conn, pcjConn;
-	private SailRepository repo, pcjRepo;
-	private Connector accCon;
-	String prefix = "table_";
-	String tablename = "table_INDEX_";
-	URI obj, obj2, subclass, subclass2, talksTo;
-
-	@Before
-	public void init() throws RepositoryException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, RyaDAOException, InferenceEngineException,
-			NumberFormatException, UnknownHostException, SailException {
-
-		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
-		conn = repo.getConnection();
-
-		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
-		pcjConn = pcjRepo.getConnection();
-
-		final URI sub = new URIImpl("uri:entity");
-		subclass = new URIImpl("uri:class");
-		obj = new URIImpl("uri:obj");
-		talksTo = new URIImpl("uri:talksTo");
-
-		conn.add(sub, RDF.TYPE, subclass);
-		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(sub, talksTo, obj);
-
-		final URI sub2 = new URIImpl("uri:entity2");
-		subclass2 = new URIImpl("uri:class2");
-		obj2 = new URIImpl("uri:obj2");
-
-		conn.add(sub2, RDF.TYPE, subclass2);
-		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub2, talksTo, obj2);
-
-		accCon = new MockInstance("instance").getConnector("root",new PasswordToken(""));
-
-	}
-
-	@After
-	public void close() throws RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableNotFoundException {
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 2, prefix);
-
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexVarInstantiate1() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			MalformedQueryException, SailException, QueryEvaluationException,
-			TupleQueryResultHandlerException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  <uri:entity> a ?c . "//
-				+ "  <uri:entity> <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  <uri:entity> <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh2);
-
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateThreeIndexVarInstantiate() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			MalformedQueryException, SailException, QueryEvaluationException,
-			TupleQueryResultHandlerException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		final URI sub = new URIImpl("uri:entity");
-		subclass = new URIImpl("uri:class");
-		obj = new URIImpl("uri:obj");
-		talksTo = new URIImpl("uri:talksTo");
-
-		final URI howlsAt = new URIImpl("uri:howlsAt");
-		final URI subType = new URIImpl("uri:subType");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub, howlsAt, superclass);
-		conn.add(superclass, subType, obj);
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String indexSparqlString3 = ""//
-				+ "SELECT ?wolf ?sheep ?chicken  " //
-				+ "{" //
-				+ "  ?wolf <uri:howlsAt> ?sheep . "//
-				+ "  ?sheep <uri:subType> ?chicken. "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  <uri:entity> a ?c . "//
-				+ "  <uri:entity> <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  <uri:entity> <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "  <uri:entity> <uri:howlsAt> ?f. "//
-				+ "  ?f <uri:subType> <uri:obj>. "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 3,
-				indexSparqlString3,
-				new String[] { "wolf", "sheep", "chicken" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateFilterInstantiate() throws RepositoryException,
-			PcjException, MalformedQueryException, SailException,
-			QueryEvaluationException, TableNotFoundException,
-			TupleQueryResultHandlerException, AccumuloException,
-			AccumuloSecurityException {
-
-		final URI e1 = new URIImpl("uri:e1");
-		final URI e2 = new URIImpl("uri:e2");
-		final URI e3 = new URIImpl("uri:e3");
-		final URI f1 = new URIImpl("uri:f1");
-		final URI f2 = new URIImpl("uri:f2");
-		final URI f3 = new URIImpl("uri:f3");
-		final URI g1 = new URIImpl("uri:g1");
-		final URI g2 = new URIImpl("uri:g2");
-		final URI g3 = new URIImpl("uri:g3");
-
-		conn.add(e1, talksTo, f1);
-		conn.add(f1, talksTo, g1);
-		conn.add(g1, talksTo, e1);
-		conn.add(e2, talksTo, f2);
-		conn.add(f2, talksTo, g2);
-		conn.add(g2, talksTo, e2);
-		conn.add(e3, talksTo, f3);
-		conn.add(f3, talksTo, g3);
-		conn.add(g3, talksTo, e3);
-
-		final String queryString = ""//
-				+ "SELECT ?x ?y ?z " //
-				+ "{" //
-				+ "Filter(?x = <uri:e1>) . " //
-				+ " ?x <uri:talksTo> ?y. " //
-				+ " ?y <uri:talksTo> ?z. " //
-				+ " ?z <uri:talksTo> <uri:e1>. " //
-				+ "}";//
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?a ?b ?c ?d " //
-				+ "{" //
-				+ "Filter(?a = ?d) . " //
-				+ " ?a <uri:talksTo> ?b. " //
-				+ " ?b <uri:talksTo> ?c. " //
-				+ " ?c <uri:talksTo> ?d. " //
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "a", "b", "c", "d" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh2);
-
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateCompoundFilterInstantiate()
-			throws RepositoryException, PcjException, MalformedQueryException,
-			SailException, QueryEvaluationException,
-			TableNotFoundException,
-			TupleQueryResultHandlerException, AccumuloException, AccumuloSecurityException {
-
-		final URI e1 = new URIImpl("uri:e1");
-		final URI f1 = new URIImpl("uri:f1");
-
-		conn.add(e1, talksTo, e1);
-		conn.add(e1, talksTo, f1);
-		conn.add(f1, talksTo, e1);
-
-		final String queryString = ""//
-				+ "SELECT ?x ?y ?z " //
-				+ "{" //
-				+ "Filter(?x = <uri:e1> && ?y = <uri:e1>) . " //
-				+ " ?x <uri:talksTo> ?y. " //
-				+ " ?y <uri:talksTo> ?z. " //
-				+ " ?z <uri:talksTo> <uri:e1>. " //
-				+ "}";//
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?a ?b ?c ?d " //
-				+ "{" //
-				+ "Filter(?a = ?d && ?b = ?d) . " //
-				+ " ?a <uri:talksTo> ?b. " //
-				+ " ?b <uri:talksTo> ?c. " //
-				+ " ?c <uri:talksTo> ?d. " //
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "a", "b", "c", "d" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(2, crh1.getCount());
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	public static class CountingResultHandler implements
-			TupleQueryResultHandler {
-		private int count = 0;
-
-		public int getCount() {
-			return count;
-		}
-
-		public void resetCount() {
-			count = 0;
-		}
-
-		@Override
-		public void startQueryResult(final List<String> arg0)
-				throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleSolution(final BindingSet arg0)
-				throws TupleQueryResultHandlerException {
-			count++;
-		}
-
-		@Override
-		public void endQueryResult() throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleBoolean(final boolean arg0)
-				throws QueryResultHandlerException {
-		}
-
-		@Override
-		public void handleLinks(final List<String> arg0)
-				throws QueryResultHandlerException {
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
new file mode 100644
index 0000000..53a11ed
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
@@ -0,0 +1,1440 @@
+package org.apache.rya.indexing.external;
+
+import java.net.UnknownHostException;
+
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.indexing.pcj.storage.PcjException;
+import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.QueryResultHandlerException;
+import org.openrdf.query.TupleQueryResultHandler;
+import org.openrdf.query.TupleQueryResultHandlerException;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.SailException;
+
+import com.beust.jcommander.internal.Sets;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.api.persist.RyaDAOException;
+import org.apache.rya.indexing.IndexPlanValidator.IndexPlanValidator;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
+import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
+import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
+import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
+import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
+
+public class AccumuloPcjIT {
+
+	private SailRepositoryConnection conn, pcjConn;
+	private SailRepository repo, pcjRepo;
+	private Connector accCon;
+	private final Configuration conf = getConf();
+	private final String prefix = "table_";
+	private final String tablename = "table_INDEX_";
+	private URI obj, obj2, subclass, subclass2, talksTo;
+
+	@Before
+	public void init() throws RepositoryException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			TableNotFoundException, InferenceEngineException,
+			NumberFormatException, UnknownHostException, SailException {
+
+		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
+		conn = repo.getConnection();
+
+		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
+		pcjConn = pcjRepo.getConnection();
+
+		final URI sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		conn.add(sub, RDF.TYPE, subclass);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(sub, talksTo, obj);
+
+		final URI sub2 = new URIImpl("uri:entity2");
+		subclass2 = new URIImpl("uri:class2");
+		obj2 = new URIImpl("uri:obj2");
+
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub2, talksTo, obj2);
+
+		accCon = ConfigUtils.getConnector(conf);
+
+
+	}
+
+	@After
+	public void close() throws RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException {
+
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 3, prefix);
+
+	}
+
+	@Test
+	public void testEvaluateSingleIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, PcjException,
+			SailException, MutationsRejectedException, TableNotFoundException {
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexTwoVarOrder1()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, PcjException,
+			SailException, TableNotFoundException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?l " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "o", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(2, crh1.getCount());
+		Assert.assertEquals(2, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexTwoVarOrder2() throws PcjException,
+			RepositoryException, TupleQueryResultHandlerException,
+			QueryEvaluationException, MalformedQueryException,
+			AccumuloException, AccumuloSecurityException, TableExistsException,
+			SailException, TableNotFoundException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?l " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "o", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(2, crh1.getCount());
+		Assert.assertEquals(2, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexTwoVarInvalidOrder() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableExistsException, TupleQueryResultHandlerException,
+			QueryEvaluationException, MalformedQueryException, SailException,
+			TableNotFoundException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?l " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "o", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder1()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, PcjException, SailException {
+
+		final TableOperations ops = accCon.tableOperations();
+		final Set<String> tables = ops.tableIdMap().keySet();
+		final Collection<String> vals = ops.tableIdMap().values();
+		System.out.println("Tables: " + tables + "and values " + vals);
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?c ?l ?f ?o" //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "c", "l", "f", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh2);
+
+		Assert.assertEquals(2, crh1.getCount());
+		Assert.assertEquals(2, crh2.getCount());
+	}
+
+	 @Test
+	public void testEvaluateTwoIndexThreeVarsDiffLabel() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			MalformedQueryException, SailException, QueryEvaluationException,
+			TupleQueryResultHandlerException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?owl  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?owl "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?c ?l ?f ?o" //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "owl" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "c", "l", "f", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(2, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder2() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableExistsException, TableNotFoundException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, SailException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		accCon.tableOperations().create("table2");
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(2, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder3ThreeBindingSet()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException,
+			TableExistsException, PcjException, SailException {
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		final URI obj3 = new URIImpl("uri:obj3");
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+		final URI superclass3 = new URIImpl("uri:superclass3");
+
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(sub3, talksTo, obj3);
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(subclass3, RDF.TYPE, superclass3);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?l ?e ?c  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "l", "e", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+		Assert.assertEquals(3, crh1.getCount());
+		Assert.assertEquals(3, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder5ThreeBindingSet()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException,
+			TableExistsException, PcjException, SailException {
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		final URI obj3 = new URIImpl("uri:obj3");
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+		final URI superclass3 = new URIImpl("uri:superclass3");
+
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(sub3, talksTo, obj3);
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(subclass3, RDF.TYPE, superclass3);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?l ?c  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(3, crh1.getCount());
+		Assert.assertEquals(3, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder4ThreeBindingSet()
+			throws PcjException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, TupleQueryResultHandlerException,
+			QueryEvaluationException, MalformedQueryException, SailException {
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		final URI obj3 = new URIImpl("uri:obj3");
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+		final URI superclass3 = new URIImpl("uri:superclass3");
+
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(sub3, talksTo, obj3);
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(subclass3, RDF.TYPE, superclass3);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(3, crh1.getCount());
+		Assert.assertEquals(3, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder6ThreeBindingSet()
+			throws MalformedQueryException, RepositoryException,
+			AccumuloException, AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, PcjException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			SailException {
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		final URI obj3 = new URIImpl("uri:obj3");
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+		final URI superclass3 = new URIImpl("uri:superclass3");
+
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(sub3, talksTo, obj3);
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(subclass3, RDF.TYPE, superclass3);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?c ?l ?e ?o ?f " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "c", "l", "e", "o", "f" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+
+		Assert.assertEquals(3, crh1.getCount());
+		Assert.assertEquals(3, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder7ThreeBindingSet()
+			throws PcjException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, TupleQueryResultHandlerException,
+			QueryEvaluationException, MalformedQueryException, SailException {
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		final URI obj3 = new URIImpl("uri:obj3");
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+		final URI superclass3 = new URIImpl("uri:superclass3");
+
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(sub3, talksTo, obj3);
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(subclass3, RDF.TYPE, superclass3);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?l ?c ?e ?f " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "l", "c", "e", "f" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+		Assert.assertEquals(3, crh1.getCount());
+		Assert.assertEquals(3, crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarInvalidOrder1()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			TableNotFoundException, PcjException, SailException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?c ?e ?l  " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?f ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "c", "e", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "e", "o", "f", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+
+	@Test
+	public void testEvaluateOneIndex() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableExistsException, TableNotFoundException,
+			MalformedQueryException, SailException, QueryEvaluationException,
+			TupleQueryResultHandlerException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, indexSparqlString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, indexSparqlString)
+				.evaluate(crh2);
+
+		Assert.assertEquals(crh1.count, crh2.count);
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder3() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, SailException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testSupportedVarOrders1() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			MalformedQueryException, SailException, QueryEvaluationException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf,
+				tablename + 1);
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf,
+				tablename + 2);
+
+		final Set<String> ais1Set1 = Sets.newHashSet();
+		ais1Set1.add("dog");
+
+		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
+		ais1Set1.add("duck");
+
+		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
+
+		ais1Set1.add("chicken");
+
+		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
+
+		final Set<String> ais2Set1 = Sets.newHashSet();
+		ais2Set1.add("f");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
+		ais2Set1.add("e");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
+
+		ais2Set1.add("o");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
+
+		ais2Set1.add("l");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
+
+		final Set<String> ais2Set2 = Sets.newHashSet();
+		ais2Set2.add("f");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set2));
+
+		ais2Set2.add("o");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set2));
+
+		ais2Set2.add("c");
+
+		Assert.assertTrue(!ais2.supportsBindingSet(ais2Set2));
+
+		final Set<String> ais2Set3 = Sets.newHashSet();
+		ais2Set3.add("c");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
+
+		ais2Set3.add("e");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
+
+		ais2Set3.add("l");
+
+		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexThreeVarOrder() throws PcjException,
+			RepositoryException, AccumuloException, AccumuloSecurityException,
+			TableNotFoundException, TableExistsException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, SailException {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final CountingResultHandler crh1 = new CountingResultHandler();
+		final CountingResultHandler crh2 = new CountingResultHandler();
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
+				.evaluate(crh1);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh2);
+		Assert.assertEquals(crh1.getCount(), crh2.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexValidate() throws Exception {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf, tablename+1);
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf, tablename+2);
+
+		final List<ExternalTupleSet> index = new ArrayList<>();
+		index.add(ais1);
+		index.add(ais2);
+
+		ParsedQuery pq = null;
+		final SPARQLParser sp = new SPARQLParser();
+		pq = sp.parseQuery(queryString, null);
+		final List<TupleExpr> teList = Lists.newArrayList();
+		final TupleExpr te = pq.getTupleExpr();
+
+		final PCJOptimizer pcj = new PCJOptimizer(index, false);
+        pcj.optimize(te, null, null);
+		teList.add(te);
+
+		final IndexPlanValidator ipv = new IndexPlanValidator(false);
+
+		Assert.assertTrue(ipv.isValid(te));
+
+	}
+
+	@Test
+	public void testEvaluateThreeIndexValidate() throws Exception {
+
+		final URI superclass = new URIImpl("uri:superclass");
+		final URI superclass2 = new URIImpl("uri:superclass2");
+
+		final URI sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		final URI howlsAt = new URIImpl("uri:howlsAt");
+		final URI subType = new URIImpl("uri:subType");
+		final URI superSuperclass = new URIImpl("uri:super_superclass");
+
+		conn.add(subclass, RDF.TYPE, superclass);
+		conn.add(subclass2, RDF.TYPE, superclass2);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, howlsAt, superclass);
+		conn.add(superclass, subType, superSuperclass);
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?dog ?pig ?duck  " //
+				+ "{" //
+				+ "  ?pig a ?dog . "//
+				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?o ?f ?e ?c ?l  " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "}";//
+
+		final String indexSparqlString3 = ""//
+				+ "SELECT ?wolf ?sheep ?chicken  " //
+				+ "{" //
+				+ "  ?wolf <uri:howlsAt> ?sheep . "//
+				+ "  ?sheep <uri:subType> ?chicken. "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?f ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
+				+ "  ?c a ?f . " //
+				+ "  ?e <uri:howlsAt> ?f. "//
+				+ "  ?f <uri:subType> ?o. "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
+				indexSparqlString, new String[] { "dog", "pig", "duck" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf, tablename+1);
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
+				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf, tablename+2);
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+3,
+				indexSparqlString3,
+				new String[] { "wolf", "sheep", "chicken" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final AccumuloIndexSet ais3 = new AccumuloIndexSet(conf, tablename+3);
+
+		final List<ExternalTupleSet> index = new ArrayList<>();
+		index.add(ais1);
+		index.add(ais3);
+		index.add(ais2);
+
+		ParsedQuery pq = null;
+		final SPARQLParser sp = new SPARQLParser();
+		pq = sp.parseQuery(queryString, null);
+		final List<TupleExpr> teList = Lists.newArrayList();
+		final TupleExpr te = pq.getTupleExpr();
+
+		final PCJOptimizer pcj = new PCJOptimizer(index, false);
+        pcj.optimize(te, null, null);
+
+		teList.add(te);
+
+		final IndexPlanValidator ipv = new IndexPlanValidator(false);
+
+		Assert.assertTrue(ipv.isValid(te));
+
+	}
+
+	public static class CountingResultHandler implements
+			TupleQueryResultHandler {
+		private int count = 0;
+
+		public int getCount() {
+			return count;
+		}
+
+		public void resetCount() {
+			count = 0;
+		}
+
+		@Override
+		public void startQueryResult(final List<String> arg0)
+				throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleSolution(final BindingSet arg0)
+				throws TupleQueryResultHandlerException {
+			count++;
+			System.out.println(arg0);
+		}
+
+		@Override
+		public void endQueryResult() throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleBoolean(final boolean arg0)
+				throws QueryResultHandlerException {
+
+		}
+
+		@Override
+		public void handleLinks(final List<String> arg0)
+				throws QueryResultHandlerException {
+
+		}
+
+	}
+
+	private static Configuration getConf() {
+		final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+		conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, true);
+		conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
+		conf.set(ConfigUtils.CLOUDBASE_USER, "root");
+		conf.set(ConfigUtils.CLOUDBASE_PASSWORD, "");
+		conf.set(ConfigUtils.CLOUDBASE_INSTANCE, "instance");
+		conf.set(ConfigUtils.CLOUDBASE_AUTHS, "");
+		conf.set(PrecomputedJoinIndexerConfig.PCJ_STORAGE_TYPE,PrecomputedJoinStorageType.ACCUMULO.name());
+		return conf;
+	}
+
+}


[4/5] incubator-rya git commit: RYA-332 rename integration tests. Closes #212.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIntegrationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIntegrationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIntegrationTest.java
deleted file mode 100644
index e51cea1..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIntegrationTest.java
+++ /dev/null
@@ -1,1440 +0,0 @@
-package org.apache.rya.indexing.external;
-
-import java.net.UnknownHostException;
-
-/*
- * 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 java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.indexing.pcj.storage.PcjException;
-import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.openrdf.model.URI;
-import org.openrdf.model.impl.LiteralImpl;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.QueryLanguage;
-import org.openrdf.query.QueryResultHandlerException;
-import org.openrdf.query.TupleQueryResultHandler;
-import org.openrdf.query.TupleQueryResultHandlerException;
-import org.openrdf.query.algebra.TupleExpr;
-import org.openrdf.query.parser.ParsedQuery;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-import org.openrdf.repository.RepositoryException;
-import org.openrdf.repository.sail.SailRepository;
-import org.openrdf.repository.sail.SailRepositoryConnection;
-import org.openrdf.sail.SailException;
-
-import com.beust.jcommander.internal.Sets;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
-import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.IndexPlanValidator.IndexPlanValidator;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
-import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
-import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
-import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
-import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
-
-public class AccumuloPcjIntegrationTest {
-
-	private SailRepositoryConnection conn, pcjConn;
-	private SailRepository repo, pcjRepo;
-	private Connector accCon;
-	private final Configuration conf = getConf();
-	private final String prefix = "table_";
-	private final String tablename = "table_INDEX_";
-	private URI obj, obj2, subclass, subclass2, talksTo;
-
-	@Before
-	public void init() throws RepositoryException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			TableNotFoundException, InferenceEngineException,
-			NumberFormatException, UnknownHostException, SailException {
-
-		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
-		conn = repo.getConnection();
-
-		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
-		pcjConn = pcjRepo.getConnection();
-
-		final URI sub = new URIImpl("uri:entity");
-		subclass = new URIImpl("uri:class");
-		obj = new URIImpl("uri:obj");
-		talksTo = new URIImpl("uri:talksTo");
-
-		conn.add(sub, RDF.TYPE, subclass);
-		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(sub, talksTo, obj);
-
-		final URI sub2 = new URIImpl("uri:entity2");
-		subclass2 = new URIImpl("uri:class2");
-		obj2 = new URIImpl("uri:obj2");
-
-		conn.add(sub2, RDF.TYPE, subclass2);
-		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub2, talksTo, obj2);
-
-		accCon = ConfigUtils.getConnector(conf);
-
-
-	}
-
-	@After
-	public void close() throws RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableNotFoundException {
-
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 3, prefix);
-
-	}
-
-	@Test
-	public void testEvaluateSingleIndex()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, PcjException,
-			SailException, MutationsRejectedException, TableNotFoundException {
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder1()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, PcjException,
-			SailException, TableNotFoundException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "o", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(2, crh1.getCount());
-		Assert.assertEquals(2, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder2() throws PcjException,
-			RepositoryException, TupleQueryResultHandlerException,
-			QueryEvaluationException, MalformedQueryException,
-			AccumuloException, AccumuloSecurityException, TableExistsException,
-			SailException, TableNotFoundException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "o", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(2, crh1.getCount());
-		Assert.assertEquals(2, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarInvalidOrder() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableExistsException, TupleQueryResultHandlerException,
-			QueryEvaluationException, MalformedQueryException, SailException,
-			TableNotFoundException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "o", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder1()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, PcjException, SailException {
-
-		final TableOperations ops = accCon.tableOperations();
-		final Set<String> tables = ops.tableIdMap().keySet();
-		final Collection<String> vals = ops.tableIdMap().values();
-		System.out.println("Tables: " + tables + "and values " + vals);
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?c ?l ?f ?o" //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "c", "l", "f", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh2);
-
-		Assert.assertEquals(2, crh1.getCount());
-		Assert.assertEquals(2, crh2.getCount());
-	}
-
-	 @Test
-	public void testEvaluateTwoIndexThreeVarsDiffLabel() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			MalformedQueryException, SailException, QueryEvaluationException,
-			TupleQueryResultHandlerException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?owl  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?owl "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?c ?l ?f ?o" //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "owl" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "c", "l", "f", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(2, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder2() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableExistsException, TableNotFoundException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, SailException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		accCon.tableOperations().create("table2");
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(2, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder3ThreeBindingSet()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableNotFoundException,
-			TableExistsException, PcjException, SailException {
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		final URI obj3 = new URIImpl("uri:obj3");
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-		final URI superclass3 = new URIImpl("uri:superclass3");
-
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(sub3, talksTo, obj3);
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(subclass3, RDF.TYPE, superclass3);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?l ?e ?c  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "l", "e", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-		Assert.assertEquals(3, crh1.getCount());
-		Assert.assertEquals(3, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder5ThreeBindingSet()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableNotFoundException,
-			TableExistsException, PcjException, SailException {
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		final URI obj3 = new URIImpl("uri:obj3");
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-		final URI superclass3 = new URIImpl("uri:superclass3");
-
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(sub3, talksTo, obj3);
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(subclass3, RDF.TYPE, superclass3);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(3, crh1.getCount());
-		Assert.assertEquals(3, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder4ThreeBindingSet()
-			throws PcjException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, TupleQueryResultHandlerException,
-			QueryEvaluationException, MalformedQueryException, SailException {
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		final URI obj3 = new URIImpl("uri:obj3");
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-		final URI superclass3 = new URIImpl("uri:superclass3");
-
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(sub3, talksTo, obj3);
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(subclass3, RDF.TYPE, superclass3);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(3, crh1.getCount());
-		Assert.assertEquals(3, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder6ThreeBindingSet()
-			throws MalformedQueryException, RepositoryException,
-			AccumuloException, AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, PcjException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			SailException {
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		final URI obj3 = new URIImpl("uri:obj3");
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-		final URI superclass3 = new URIImpl("uri:superclass3");
-
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(sub3, talksTo, obj3);
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(subclass3, RDF.TYPE, superclass3);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?c ?l ?e ?o ?f " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "c", "l", "e", "o", "f" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-
-		Assert.assertEquals(3, crh1.getCount());
-		Assert.assertEquals(3, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder7ThreeBindingSet()
-			throws PcjException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, TupleQueryResultHandlerException,
-			QueryEvaluationException, MalformedQueryException, SailException {
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		final URI obj3 = new URIImpl("uri:obj3");
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-		final URI superclass3 = new URIImpl("uri:superclass3");
-
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(sub3, talksTo, obj3);
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(subclass3, RDF.TYPE, superclass3);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj3, RDFS.LABEL, new LiteralImpl("label3"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?l ?c ?e ?f " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "l", "c", "e", "f" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-		Assert.assertEquals(3, crh1.getCount());
-		Assert.assertEquals(3, crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarInvalidOrder1()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			TableNotFoundException, PcjException, SailException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?f ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "c", "e", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "e", "o", "f", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-
-	@Test
-	public void testEvaluateOneIndex() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableExistsException, TableNotFoundException,
-			MalformedQueryException, SailException, QueryEvaluationException,
-			TupleQueryResultHandlerException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, indexSparqlString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, indexSparqlString)
-				.evaluate(crh2);
-
-		Assert.assertEquals(crh1.count, crh2.count);
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder3() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, SailException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testSupportedVarOrders1() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			MalformedQueryException, SailException, QueryEvaluationException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf,
-				tablename + 1);
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename + 2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf,
-				tablename + 2);
-
-		final Set<String> ais1Set1 = Sets.newHashSet();
-		ais1Set1.add("dog");
-
-		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
-		ais1Set1.add("duck");
-
-		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
-
-		ais1Set1.add("chicken");
-
-		Assert.assertTrue(ais1.supportsBindingSet(ais1Set1));
-
-		final Set<String> ais2Set1 = Sets.newHashSet();
-		ais2Set1.add("f");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
-		ais2Set1.add("e");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
-
-		ais2Set1.add("o");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
-
-		ais2Set1.add("l");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set1));
-
-		final Set<String> ais2Set2 = Sets.newHashSet();
-		ais2Set2.add("f");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set2));
-
-		ais2Set2.add("o");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set2));
-
-		ais2Set2.add("c");
-
-		Assert.assertTrue(!ais2.supportsBindingSet(ais2Set2));
-
-		final Set<String> ais2Set3 = Sets.newHashSet();
-		ais2Set3.add("c");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
-
-		ais2Set3.add("e");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
-
-		ais2Set3.add("l");
-
-		Assert.assertTrue(ais2.supportsBindingSet(ais2Set3));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexThreeVarOrder() throws PcjException,
-			RepositoryException, AccumuloException, AccumuloSecurityException,
-			TableNotFoundException, TableExistsException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, SailException {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final CountingResultHandler crh1 = new CountingResultHandler();
-		final CountingResultHandler crh2 = new CountingResultHandler();
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString)
-				.evaluate(crh1);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, prefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh2);
-		Assert.assertEquals(crh1.getCount(), crh2.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexValidate() throws Exception {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf, tablename+1);
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf, tablename+2);
-
-		final List<ExternalTupleSet> index = new ArrayList<>();
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = null;
-		final SPARQLParser sp = new SPARQLParser();
-		pq = sp.parseQuery(queryString, null);
-		final List<TupleExpr> teList = Lists.newArrayList();
-		final TupleExpr te = pq.getTupleExpr();
-
-		final PCJOptimizer pcj = new PCJOptimizer(index, false);
-        pcj.optimize(te, null, null);
-		teList.add(te);
-
-		final IndexPlanValidator ipv = new IndexPlanValidator(false);
-
-		Assert.assertTrue(ipv.isValid(te));
-
-	}
-
-	@Test
-	public void testEvaluateThreeIndexValidate() throws Exception {
-
-		final URI superclass = new URIImpl("uri:superclass");
-		final URI superclass2 = new URIImpl("uri:superclass2");
-
-		final URI sub = new URIImpl("uri:entity");
-		subclass = new URIImpl("uri:class");
-		obj = new URIImpl("uri:obj");
-		talksTo = new URIImpl("uri:talksTo");
-
-		final URI howlsAt = new URIImpl("uri:howlsAt");
-		final URI subType = new URIImpl("uri:subType");
-		final URI superSuperclass = new URIImpl("uri:super_superclass");
-
-		conn.add(subclass, RDF.TYPE, superclass);
-		conn.add(subclass2, RDF.TYPE, superclass2);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub, howlsAt, superclass);
-		conn.add(superclass, subType, superSuperclass);
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?duck  " //
-				+ "{" //
-				+ "  ?pig a ?dog . "//
-				+ "  ?pig <http://www.w3.org/2000/01/rdf-schema#label> ?duck "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?o ?f ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "}";//
-
-		final String indexSparqlString3 = ""//
-				+ "SELECT ?wolf ?sheep ?chicken  " //
-				+ "{" //
-				+ "  ?wolf <uri:howlsAt> ?sheep . "//
-				+ "  ?sheep <uri:subType> ?chicken. "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?f ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l. "//
-				+ "  ?c a ?f . " //
-				+ "  ?e <uri:howlsAt> ?f. "//
-				+ "  ?f <uri:subType> ?o. "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+1,
-				indexSparqlString, new String[] { "dog", "pig", "duck" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais1 = new AccumuloIndexSet(conf, tablename+1);
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+2,
-				indexSparqlString2, new String[] { "o", "f", "e", "c", "l" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais2 = new AccumuloIndexSet(conf, tablename+2);
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablename+3,
-				indexSparqlString3,
-				new String[] { "wolf", "sheep", "chicken" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final AccumuloIndexSet ais3 = new AccumuloIndexSet(conf, tablename+3);
-
-		final List<ExternalTupleSet> index = new ArrayList<>();
-		index.add(ais1);
-		index.add(ais3);
-		index.add(ais2);
-
-		ParsedQuery pq = null;
-		final SPARQLParser sp = new SPARQLParser();
-		pq = sp.parseQuery(queryString, null);
-		final List<TupleExpr> teList = Lists.newArrayList();
-		final TupleExpr te = pq.getTupleExpr();
-
-		final PCJOptimizer pcj = new PCJOptimizer(index, false);
-        pcj.optimize(te, null, null);
-
-		teList.add(te);
-
-		final IndexPlanValidator ipv = new IndexPlanValidator(false);
-
-		Assert.assertTrue(ipv.isValid(te));
-
-	}
-
-	public static class CountingResultHandler implements
-			TupleQueryResultHandler {
-		private int count = 0;
-
-		public int getCount() {
-			return count;
-		}
-
-		public void resetCount() {
-			count = 0;
-		}
-
-		@Override
-		public void startQueryResult(final List<String> arg0)
-				throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleSolution(final BindingSet arg0)
-				throws TupleQueryResultHandlerException {
-			count++;
-			System.out.println(arg0);
-		}
-
-		@Override
-		public void endQueryResult() throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleBoolean(final boolean arg0)
-				throws QueryResultHandlerException {
-
-		}
-
-		@Override
-		public void handleLinks(final List<String> arg0)
-				throws QueryResultHandlerException {
-
-		}
-
-	}
-
-	private static Configuration getConf() {
-		final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
-		conf.setBoolean(ConfigUtils.USE_MOCK_INSTANCE, true);
-		conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
-		conf.set(ConfigUtils.CLOUDBASE_USER, "root");
-		conf.set(ConfigUtils.CLOUDBASE_PASSWORD, "");
-		conf.set(ConfigUtils.CLOUDBASE_INSTANCE, "instance");
-		conf.set(ConfigUtils.CLOUDBASE_AUTHS, "");
-		conf.set(PrecomputedJoinIndexerConfig.PCJ_STORAGE_TYPE,PrecomputedJoinStorageType.ACCUMULO.name());
-		return conf;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
index 854ef2d..af30db3 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
@@ -59,7 +59,7 @@ import com.beust.jcommander.internal.Lists;
 import com.google.common.base.Optional;
 
 import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.external.PrecompJoinOptimizerIntegrationTest.CountingResultHandler;
+import org.apache.rya.indexing.external.PrecompJoinOptimizerIT.CountingResultHandler;
 import org.apache.rya.indexing.external.PrecompJoinOptimizerTest.NodeCollector;
 import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.apache.rya.indexing.external.tupleSet.SimpleExternalTupleSet;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIT.java
new file mode 100644
index 0000000..c133a4a
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIT.java
@@ -0,0 +1,505 @@
+package org.apache.rya.indexing.external;
+
+import java.net.UnknownHostException;
+
+/*
+ * 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 java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.rya.indexing.pcj.storage.PcjException;
+import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.QueryResultHandlerException;
+import org.openrdf.query.TupleQueryResultHandler;
+import org.openrdf.query.TupleQueryResultHandlerException;
+import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.SailException;
+
+import com.google.common.base.Optional;
+
+import org.apache.rya.api.persist.RyaDAOException;
+import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
+
+public class PrecompJoinOptimizerIT {
+
+	private SailRepositoryConnection conn, pcjConn;
+	private SailRepository repo, pcjRepo;
+	private Connector accCon;
+	String tablePrefix = "table_";
+	URI sub, sub2, obj, obj2, subclass, subclass2, talksTo;
+
+	@Before
+	public void init() throws RepositoryException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			TableNotFoundException, InferenceEngineException, NumberFormatException,
+			UnknownHostException, SailException {
+
+		repo = PcjIntegrationTestingUtil.getNonPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+
+		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		pcjConn = pcjRepo.getConnection();
+
+		sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		conn.add(sub, RDF.TYPE, subclass);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(sub, talksTo, obj);
+
+		sub2 = new URIImpl("uri:entity2");
+		subclass2 = new URIImpl("uri:class2");
+		obj2 = new URIImpl("uri:obj2");
+
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub2, talksTo, obj2);
+
+		accCon = new MockInstance("instance").getConnector("root",
+				new PasswordToken(""));
+
+	}
+
+	@After
+	public void close() throws RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException {
+
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 2, tablePrefix);
+
+	}
+
+	@Test
+	public void testEvaluateSingeIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
+
+		Assert.assertEquals(2, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexTwoVarOrder1() throws AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			RepositoryException, MalformedQueryException, SailException,
+			QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?l " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(2, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?e = <uri:entity>) " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "   Filter(?e = <uri:entity>) " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterWithUnion()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?l = \"label2\") " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?o ?m ?l" //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ " { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m  }"//
+				+ " UNION { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterWithLeftJoin()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		final String indexSparqlString1 = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?l = \"label3\") " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString1, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?o ?m ?l" //
+				+ "{" //
+				+ "  Filter(?l = \"label3\") " //
+				+ "  ?e a ?c . " //
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . " //
+				+ "  OPTIONAL { ?e <uri:talksTo> ?o . ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m }"//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexUnionFilter() throws AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			RepositoryException, MalformedQueryException, SailException,
+			QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, RDF.TYPE, obj);
+		conn.add(sub2, RDF.TYPE, obj2);
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e a ?o . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e a ?c . "//
+				+ " { ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDF.TYPE, obj2);
+		final CountingResultHandler crh = new CountingResultHandler();
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(6, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexLeftJoinUnionFilter()
+			throws AccumuloException, AccumuloSecurityException,
+			TableExistsException, RepositoryException, MalformedQueryException,
+			SailException, QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException, InferenceEngineException,
+			NumberFormatException, UnknownHostException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, RDF.TYPE, obj);
+		conn.add(sub2, RDF.TYPE, obj2);
+
+		final URI livesIn = new URIImpl("uri:livesIn");
+		final URI city = new URIImpl("uri:city");
+		final URI city2 = new URIImpl("uri:city2");
+		final URI city3 = new URIImpl("uri:city3");
+		conn.add(sub, livesIn, city);
+		conn.add(sub2, livesIn, city2);
+		conn.add(sub2, livesIn, city3);
+		conn.add(sub, livesIn, city3);
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?o . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?e ?l ?o " //
+				+ "{" //
+				+ " Filter(?c = <uri:city3>) " //
+				+ " ?e <uri:livesIn> ?c . "//
+				+ " OPTIONAL{{ ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }}"//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub2, livesIn, city3);
+		conn.add(sub, livesIn, city3);
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(6, crh.getCount());
+
+	}
+
+	public static class CountingResultHandler implements
+			TupleQueryResultHandler {
+		private int count = 0;
+
+		public int getCount() {
+			return count;
+		}
+
+		public void resetCount() {
+			count = 0;
+		}
+
+		@Override
+		public void startQueryResult(final List<String> arg0)
+				throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleSolution(final BindingSet arg0)
+				throws TupleQueryResultHandlerException {
+			System.out.println(arg0);
+			count++;
+			System.out.println("Count is " + count);
+		}
+
+		@Override
+		public void endQueryResult() throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleBoolean(final boolean arg0)
+				throws QueryResultHandlerException {
+			// TODO Auto-generated method stub
+
+		}
+
+		@Override
+		public void handleLinks(final List<String> arg0)
+				throws QueryResultHandlerException {
+			// TODO Auto-generated method stub
+
+		}
+	}
+
+}
\ No newline at end of file


[3/5] incubator-rya git commit: RYA-332 rename integration tests. Closes #212.

Posted by ca...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
deleted file mode 100644
index 1896025..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
+++ /dev/null
@@ -1,505 +0,0 @@
-package org.apache.rya.indexing.external;
-
-import java.net.UnknownHostException;
-
-/*
- * 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 java.util.List;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.rya.indexing.pcj.storage.PcjException;
-import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.openrdf.model.URI;
-import org.openrdf.model.impl.LiteralImpl;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.QueryLanguage;
-import org.openrdf.query.QueryResultHandlerException;
-import org.openrdf.query.TupleQueryResultHandler;
-import org.openrdf.query.TupleQueryResultHandlerException;
-import org.openrdf.repository.RepositoryException;
-import org.openrdf.repository.sail.SailRepository;
-import org.openrdf.repository.sail.SailRepositoryConnection;
-import org.openrdf.sail.SailException;
-
-import com.google.common.base.Optional;
-
-import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
-
-public class PrecompJoinOptimizerIntegrationTest {
-
-	private SailRepositoryConnection conn, pcjConn;
-	private SailRepository repo, pcjRepo;
-	private Connector accCon;
-	String tablePrefix = "table_";
-	URI sub, sub2, obj, obj2, subclass, subclass2, talksTo;
-
-	@Before
-	public void init() throws RepositoryException,
-			TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			TableNotFoundException, InferenceEngineException, NumberFormatException,
-			UnknownHostException, SailException {
-
-		repo = PcjIntegrationTestingUtil.getNonPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-
-		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		pcjConn = pcjRepo.getConnection();
-
-		sub = new URIImpl("uri:entity");
-		subclass = new URIImpl("uri:class");
-		obj = new URIImpl("uri:obj");
-		talksTo = new URIImpl("uri:talksTo");
-
-		conn.add(sub, RDF.TYPE, subclass);
-		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(sub, talksTo, obj);
-
-		sub2 = new URIImpl("uri:entity2");
-		subclass2 = new URIImpl("uri:class2");
-		obj2 = new URIImpl("uri:obj2");
-
-		conn.add(sub2, RDF.TYPE, subclass2);
-		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub2, talksTo, obj2);
-
-		accCon = new MockInstance("instance").getConnector("root",
-				new PasswordToken(""));
-
-	}
-
-	@After
-	public void close() throws RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableNotFoundException {
-
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 2, tablePrefix);
-
-	}
-
-	@Test
-	public void testEvaluateSingeIndex()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			SailException, TableNotFoundException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "}";//
-
-		final CountingResultHandler crh = new CountingResultHandler();
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub, talksTo, obj);
-		conn.add(sub2, talksTo, obj2);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-
-		Assert.assertEquals(2, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder1() throws AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			RepositoryException, MalformedQueryException, SailException,
-			QueryEvaluationException, TableNotFoundException,
-			TupleQueryResultHandlerException, RyaDAOException, PcjException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-		final CountingResultHandler crh = new CountingResultHandler();
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(2, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateSingeFilterIndex()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			SailException, TableNotFoundException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  Filter(?e = <uri:entity>) " //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "   Filter(?e = <uri:entity>) " //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "}";//
-
-		final CountingResultHandler crh = new CountingResultHandler();
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub, talksTo, obj);
-		conn.add(sub2, talksTo, obj2);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(1, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateSingeFilterWithUnion()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			SailException, TableNotFoundException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  Filter(?l = \"label2\") " //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?o ?m ?l" //
-				+ "{" //
-				+ "   Filter(?l = \"label2\") " //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ " { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m  }"//
-				+ " UNION { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-				+ "}";//
-
-		final CountingResultHandler crh = new CountingResultHandler();
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub, talksTo, obj);
-		conn.add(sub2, talksTo, obj2);
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(1, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateSingeFilterWithLeftJoin()
-			throws TupleQueryResultHandlerException, QueryEvaluationException,
-			MalformedQueryException, RepositoryException, AccumuloException,
-			AccumuloSecurityException, TableExistsException, RyaDAOException,
-			SailException, TableNotFoundException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		final String indexSparqlString1 = ""//
-				+ "SELECT ?e ?l ?c " //
-				+ "{" //
-				+ "  Filter(?l = \"label3\") " //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final URI sub3 = new URIImpl("uri:entity3");
-		final URI subclass3 = new URIImpl("uri:class3");
-		conn.add(sub3, RDF.TYPE, subclass3);
-		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString1, new String[] { "e", "l", "c" },
-				Optional.<PcjVarOrderFactory> absent());
-		final String queryString = ""//
-				+ "SELECT ?e ?c ?o ?m ?l" //
-				+ "{" //
-				+ "  Filter(?l = \"label3\") " //
-				+ "  ?e a ?c . " //
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . " //
-				+ "  OPTIONAL { ?e <uri:talksTo> ?o . ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m }"//
-				+ "}";//
-
-		final CountingResultHandler crh = new CountingResultHandler();
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub, talksTo, obj);
-		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(1, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexUnionFilter() throws AccumuloException,
-			AccumuloSecurityException, TableExistsException,
-			RepositoryException, MalformedQueryException, SailException,
-			QueryEvaluationException, TableNotFoundException,
-			TupleQueryResultHandlerException, RyaDAOException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub, RDF.TYPE, obj);
-		conn.add(sub2, RDF.TYPE, obj2);
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "   Filter(?l = \"label2\") " //
-				+ "  ?e a ?o . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "   Filter(?l = \"label2\") " //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?c ?e ?l ?o " //
-				+ "{" //
-				+ "   Filter(?l = \"label2\") " //
-				+ "  ?e a ?c . "//
-				+ " { ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub2, RDF.TYPE, subclass2);
-		conn.add(sub2, RDF.TYPE, obj2);
-		final CountingResultHandler crh = new CountingResultHandler();
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(6, crh.getCount());
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexLeftJoinUnionFilter()
-			throws AccumuloException, AccumuloSecurityException,
-			TableExistsException, RepositoryException, MalformedQueryException,
-			SailException, QueryEvaluationException, TableNotFoundException,
-			TupleQueryResultHandlerException, RyaDAOException, PcjException, InferenceEngineException,
-			NumberFormatException, UnknownHostException {
-
-		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-		conn.add(sub, RDF.TYPE, obj);
-		conn.add(sub2, RDF.TYPE, obj2);
-
-		final URI livesIn = new URIImpl("uri:livesIn");
-		final URI city = new URIImpl("uri:city");
-		final URI city2 = new URIImpl("uri:city2");
-		final URI city3 = new URIImpl("uri:city3");
-		conn.add(sub, livesIn, city);
-		conn.add(sub2, livesIn, city2);
-		conn.add(sub2, livesIn, city3);
-		conn.add(sub, livesIn, city3);
-
-		final String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?o . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		final String queryString = ""//
-				+ "SELECT ?c ?e ?l ?o " //
-				+ "{" //
-				+ " Filter(?c = <uri:city3>) " //
-				+ " ?e <uri:livesIn> ?c . "//
-				+ " OPTIONAL{{ ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }}"//
-				+ "}";//
-
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
-				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
-				Optional.<PcjVarOrderFactory> absent());
-
-		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
-		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
-		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
-		conn = repo.getConnection();
-		conn.add(sub2, livesIn, city3);
-		conn.add(sub, livesIn, city3);
-
-		final CountingResultHandler crh = new CountingResultHandler();
-		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
-				crh);
-
-		Assert.assertEquals(6, crh.getCount());
-
-	}
-
-	public static class CountingResultHandler implements
-			TupleQueryResultHandler {
-		private int count = 0;
-
-		public int getCount() {
-			return count;
-		}
-
-		public void resetCount() {
-			count = 0;
-		}
-
-		@Override
-		public void startQueryResult(final List<String> arg0)
-				throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleSolution(final BindingSet arg0)
-				throws TupleQueryResultHandlerException {
-			System.out.println(arg0);
-			count++;
-			System.out.println("Count is " + count);
-		}
-
-		@Override
-		public void endQueryResult() throws TupleQueryResultHandlerException {
-		}
-
-		@Override
-		public void handleBoolean(final boolean arg0)
-				throws QueryResultHandlerException {
-			// TODO Auto-generated method stub
-
-		}
-
-		@Override
-		public void handleLinks(final List<String> arg0)
-				throws QueryResultHandlerException {
-			// TODO Auto-generated method stub
-
-		}
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
new file mode 100644
index 0000000..7e410e0
--- /dev/null
+++ b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIT.java
@@ -0,0 +1,573 @@
+/**
+ * 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.rya.indexing.pcj.storage.accumulo;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.accumulo.AccumuloRyaDAO;
+import org.apache.rya.accumulo.MiniAccumuloClusterInstance;
+import org.apache.rya.accumulo.MiniAccumuloSingleton;
+import org.apache.rya.accumulo.RyaTestInstanceRule;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.indexing.pcj.storage.PcjException;
+import org.apache.rya.indexing.pcj.storage.PcjMetadata;
+import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
+import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
+import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
+import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
+import org.apache.rya.rdftriplestore.RyaSailRepository;
+import org.apache.zookeeper.ClientCnxn;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.openrdf.model.Statement;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.NumericLiteralImpl;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+import org.openrdf.repository.RepositoryConnection;
+import org.openrdf.repository.RepositoryException;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Sets;
+
+/**
+ * Performs integration test using {@link MiniAccumuloCluster} to ensure the
+ * functions of {@link PcjTables} work within a cluster setting.
+ */
+public class PcjTablesIT {
+
+    private static final String USE_MOCK_INSTANCE = ".useMockInstance";
+    private static final String CLOUDBASE_INSTANCE = "sc.cloudbase.instancename";
+    private static final String CLOUDBASE_USER = "sc.cloudbase.username";
+    private static final String CLOUDBASE_PASSWORD = "sc.cloudbase.password";
+
+    private static final AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
+
+    // The MiniAccumuloCluster is re-used between tests.
+    private MiniAccumuloClusterInstance cluster = MiniAccumuloSingleton.getInstance();
+
+    // Rya data store and connections.
+    protected RyaSailRepository ryaRepo = null;
+    protected RepositoryConnection ryaConn = null;
+
+    @Rule
+    public RyaTestInstanceRule testInstance = new RyaTestInstanceRule(false);
+
+    @BeforeClass
+    public static void killLoudLogs() {
+        Logger.getLogger(ClientCnxn.class).setLevel(Level.ERROR);
+    }
+
+    @Before
+    public void resetTestEnvironmanet() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, RepositoryException, IOException, InterruptedException {
+        // Setup the Rya library to use the Mini Accumulo.
+        ryaRepo = setupRya();
+        ryaConn = ryaRepo.getConnection();
+    }
+
+    @After
+    public void shutdownMiniCluster() throws IOException, InterruptedException, RepositoryException {
+        // Stop Rya.
+        if (ryaRepo != null) {
+            ryaRepo.shutDown();
+        }
+    }
+
+    private String getRyaInstanceName() {
+        return testInstance.getRyaInstanceName();
+    }
+    
+    /**
+     * Format a Mini Accumulo to be a Rya repository.
+     *
+     * @return The Rya repository sitting on top of the Mini Accumulo.
+     */
+    private RyaSailRepository setupRya() throws AccumuloException, AccumuloSecurityException, RepositoryException {
+        // Setup the Rya Repository that will be used to create Repository Connections.
+        final RdfCloudTripleStore ryaStore = new RdfCloudTripleStore();
+        final AccumuloRyaDAO crdfdao = new AccumuloRyaDAO();
+        crdfdao.setConnector( cluster.getConnector() );
+
+        // Setup Rya configuration values.
+        final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
+        conf.setTablePrefix(getRyaInstanceName());
+        conf.setDisplayQueryPlan(true);
+
+        conf.setBoolean(USE_MOCK_INSTANCE, false);
+        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, getRyaInstanceName());
+        conf.set(CLOUDBASE_USER, cluster.getUsername());
+        conf.set(CLOUDBASE_PASSWORD, cluster.getPassword());
+        conf.set(CLOUDBASE_INSTANCE, cluster.getInstanceName());
+
+        crdfdao.setConf(conf);
+        ryaStore.setRyaDAO(crdfdao);
+
+        final RyaSailRepository ryaRepo = new RyaSailRepository(ryaStore);
+        ryaRepo.initialize();
+
+        return ryaRepo;
+    }
+
+    /**
+     * Ensure that when a new PCJ table is created, it is initialized with the
+     * correct metadata values.
+     * <p>
+     * The method being tested is {@link PcjTables#createPcjTable(Connector, String, Set, String)}
+     */
+    @Test
+    public void createPcjTable() throws PcjException, AccumuloException, AccumuloSecurityException {
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Create a PCJ table in the Mini Accumulo.
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Fetch the PcjMetadata and ensure it has the correct values.
+        final PcjMetadata pcjMetadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+
+        // Ensure the metadata matches the expected value.
+        final PcjMetadata expected = new PcjMetadata(sparql, 0L, varOrders);
+        assertEquals(expected, pcjMetadata);
+    }
+
+    /**
+     * Ensure when results have been written to the PCJ table that they are in Accumulo.
+     * <p>
+     * The method being tested is {@link PcjTables#addResults(Connector, String, java.util.Collection)}
+     */
+    @Test
+    public void addResults() throws PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Create a PCJ table in the Mini Accumulo.
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Add a few results to the PCJ table.
+        final MapBindingSet alice = new MapBindingSet();
+        alice.addBinding("name", new URIImpl("http://Alice"));
+        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
+
+        final MapBindingSet bob = new MapBindingSet();
+        bob.addBinding("name", new URIImpl("http://Bob"));
+        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
+
+        final MapBindingSet charlie = new MapBindingSet();
+        charlie.addBinding("name", new URIImpl("http://Charlie"));
+        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
+
+        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
+        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
+                new VisibilityBindingSet(alice),
+                new VisibilityBindingSet(bob),
+                new VisibilityBindingSet(charlie)));
+
+        // Make sure the cardinality was updated.
+        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+        assertEquals(3, metadata.getCardinality());
+
+        // Scan Accumulo for the stored results.
+        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
+
+        // Ensure the expected results match those that were stored.
+        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
+        expectedResults.putAll("name;age", results);
+        expectedResults.putAll("age;name", results);
+        assertEquals(expectedResults, fetchedResults);
+    }
+
+    @Test
+    public void listResults() throws Exception {
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Create a PCJ table in the Mini Accumulo.
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Add a few results to the PCJ table.
+        final MapBindingSet alice = new MapBindingSet();
+        alice.addBinding("name", new URIImpl("http://Alice"));
+        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
+
+        final MapBindingSet bob = new MapBindingSet();
+        bob.addBinding("name", new URIImpl("http://Bob"));
+        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
+
+        final MapBindingSet charlie = new MapBindingSet();
+        charlie.addBinding("name", new URIImpl("http://Charlie"));
+        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
+
+        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
+                new VisibilityBindingSet(alice),
+                new VisibilityBindingSet(bob),
+                new VisibilityBindingSet(charlie)));
+
+        // Fetch the Binding Sets that have been stored in the PCJ table.
+        final Set<BindingSet> results = new HashSet<>();
+
+        final CloseableIterator<BindingSet> resultsIt = pcjs.listResults(accumuloConn, pcjTableName, new Authorizations());
+        try {
+            while(resultsIt.hasNext()) {
+                results.add( resultsIt.next() );
+            }
+        } finally {
+            resultsIt.close();
+        }
+
+        // Verify the fetched results match the expected ones.
+        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet(alice, bob, charlie);
+        assertEquals(expected, results);
+    }
+
+    /**
+     * Ensure when results are already stored in Rya, that we are able to populate
+     * the PCJ table for a new SPARQL query using those results.
+     * <p>
+     * The method being tested is: {@link PcjTables#populatePcj(Connector, String, RepositoryConnection, String)}
+     */
+    @Test
+    public void populatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
+        // Load some Triples into Rya.
+        final Set<Statement> triples = new HashSet<>();
+        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://hasAge"), new NumericLiteralImpl(16, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://hasAge"), new NumericLiteralImpl(12, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://hasAge"), new NumericLiteralImpl(43, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+
+        for(final Statement triple : triples) {
+            ryaConn.add(triple);
+        }
+
+        // Create a PCJ table that will include those triples in its results.
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Populate the PCJ table using a Rya connection.
+        pcjs.populatePcj(accumuloConn, pcjTableName, ryaConn);
+
+        // Scan Accumulo for the stored results.
+        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
+
+        // Make sure the cardinality was updated.
+        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+        assertEquals(3, metadata.getCardinality());
+
+        // Ensure the expected results match those that were stored.
+        final MapBindingSet alice = new MapBindingSet();
+        alice.addBinding("name", new URIImpl("http://Alice"));
+        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
+
+        final MapBindingSet bob = new MapBindingSet();
+        bob.addBinding("name", new URIImpl("http://Bob"));
+        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
+
+        final MapBindingSet charlie = new MapBindingSet();
+        charlie.addBinding("name", new URIImpl("http://Charlie"));
+        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
+
+        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
+
+        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
+        expectedResults.putAll("name;age", results);
+        expectedResults.putAll("age;name", results);
+        assertEquals(expectedResults, fetchedResults);
+    }
+
+    /**
+     * Ensure the method that creates a new PCJ table, scans Rya for matches, and
+     * stores them in the PCJ table works.
+     * <p>
+     * The method being tested is: {@link PcjTables#createAndPopulatePcj(RepositoryConnection, Connector, String, String, String[], Optional)}
+     */
+    @Test
+    public void createAndPopulatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
+        // Load some Triples into Rya.
+        final Set<Statement> triples = new HashSet<>();
+        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://hasAge"), new NumericLiteralImpl(16, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://hasAge"), new NumericLiteralImpl(12, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://hasAge"), new NumericLiteralImpl(43, XMLSchema.INTEGER)) );
+        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
+
+        for(final Statement triple : triples) {
+            ryaConn.add(triple);
+        }
+
+        // Create a PCJ table that will include those triples in its results.
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+
+        // Create and populate the PCJ table.
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createAndPopulatePcj(ryaConn, accumuloConn, pcjTableName, sparql, new String[]{"name", "age"}, Optional.<PcjVarOrderFactory>absent());
+
+        // Make sure the cardinality was updated.
+        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+        assertEquals(3, metadata.getCardinality());
+
+        // Scan Accumulo for the stored results.
+        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
+
+        // Ensure the expected results match those that were stored.
+        final MapBindingSet alice = new MapBindingSet();
+        alice.addBinding("name", new URIImpl("http://Alice"));
+        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
+
+        final MapBindingSet bob = new MapBindingSet();
+        bob.addBinding("name", new URIImpl("http://Bob"));
+        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
+
+        final MapBindingSet charlie = new MapBindingSet();
+        charlie.addBinding("name", new URIImpl("http://Charlie"));
+        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
+
+        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
+
+        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
+        expectedResults.putAll("name;age", results);
+        expectedResults.putAll("age;name", results);
+
+        assertEquals(expectedResults, fetchedResults);
+    }
+
+    @Test
+    public void listPcjs() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Set up the table names that will be used.
+        final String instance1 = "instance1_";
+        final String instance2 = "instance2_";
+
+        final String instance1_table1 = new PcjTableNameFactory().makeTableName(instance1, "table1");
+        final String instance1_table2 = new PcjTableNameFactory().makeTableName(instance1, "table2");
+        final String instance1_table3 = new PcjTableNameFactory().makeTableName(instance1, "table3");
+
+        final String instance2_table1 = new PcjTableNameFactory().makeTableName(instance2, "table1");
+
+        // Create the PCJ Tables that are in instance 1 and instance 2.
+        final Set<VariableOrder> varOrders = Sets.<VariableOrder>newHashSet( new VariableOrder("x") );
+        final String sparql = "SELECT x WHERE ?x <http://isA> <http://Food>";
+
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, instance1_table1, varOrders, sparql);
+        pcjs.createPcjTable(accumuloConn, instance1_table2, varOrders, sparql);
+        pcjs.createPcjTable(accumuloConn, instance1_table3, varOrders, sparql);
+
+        pcjs.createPcjTable(accumuloConn, instance2_table1, varOrders, sparql);
+
+        // Ensure all of the names have been stored for instance 1 and 2.
+        final Set<String> expected1 = Sets.newHashSet(instance1_table1, instance1_table2, instance1_table3);
+        final Set<String> instance1Tables = Sets.newHashSet( pcjs.listPcjTables(accumuloConn, instance1) );
+        assertEquals(expected1, instance1Tables);
+
+        final Set<String> expected2 = Sets.newHashSet(instance2_table1);
+        final Set<String> instance2Tables = Sets.newHashSet( pcjs.listPcjTables(accumuloConn, instance2) );
+        assertEquals(expected2, instance2Tables);
+    }
+
+    @Test
+    public void purge() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
+        final String sparql =
+                "SELECT ?name ?age " +
+                "{" +
+                  "FILTER(?age < 30) ." +
+                  "?name <http://hasAge> ?age." +
+                  "?name <http://playsSport> \"Soccer\" " +
+                "}";
+
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Create a PCJ table in the Mini Accumulo.
+        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
+        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Add a few results to the PCJ table.
+        final MapBindingSet alice = new MapBindingSet();
+        alice.addBinding("name", new URIImpl("http://Alice"));
+        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
+
+        final MapBindingSet bob = new MapBindingSet();
+        bob.addBinding("name", new URIImpl("http://Bob"));
+        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
+
+        final MapBindingSet charlie = new MapBindingSet();
+        charlie.addBinding("name", new URIImpl("http://Charlie"));
+        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
+
+        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
+                new VisibilityBindingSet(alice),
+                new VisibilityBindingSet(bob),
+                new VisibilityBindingSet(charlie)));
+
+        // Make sure the cardinality was updated.
+        PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+        assertEquals(3, metadata.getCardinality());
+
+        // Purge the data.
+        pcjs.purgePcjTable(accumuloConn, pcjTableName);
+
+        // Make sure the cardinality was updated to 0.
+        metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+        assertEquals(0, metadata.getCardinality());
+    }
+
+    @Test
+    public void dropPcj() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
+        final Connector accumuloConn = cluster.getConnector();
+
+        // Create a PCJ index.
+        final String tableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "thePcj");
+        final Set<VariableOrder> varOrders = Sets.<VariableOrder>newHashSet( new VariableOrder("x") );
+        final String sparql = "SELECT x WHERE ?x <http://isA> <http://Food>";
+
+        final PcjTables pcjs = new PcjTables();
+        pcjs.createPcjTable(accumuloConn, tableName, varOrders, sparql);
+
+        // Fetch its metadata to show that it has actually been created.
+        final PcjMetadata expectedMetadata = new PcjMetadata(sparql, 0L, varOrders);
+        PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, tableName);
+        assertEquals(expectedMetadata, metadata);
+
+        // Drop it.
+        pcjs.dropPcjTable(accumuloConn, tableName);
+
+        // Show the metadata is no longer present.
+        PCJStorageException tableDoesNotExistException = null;
+        try {
+            metadata = pcjs.getPcjMetadata(accumuloConn, tableName);
+        } catch(final PCJStorageException e) {
+            tableDoesNotExistException = e;
+        }
+        assertNotNull(tableDoesNotExistException);
+    }
+
+    /**
+     * Scan accumulo for the results that are stored in a PCJ table. The
+     * multimap stores a set of deserialized binding sets that were in the PCJ
+     * table for every variable order that is found in the PCJ metadata.
+     */
+    private static Multimap<String, BindingSet> loadPcjResults(final Connector accumuloConn, final String pcjTableName) throws PcjException, TableNotFoundException, BindingSetConversionException {
+        final Multimap<String, BindingSet> fetchedResults = HashMultimap.create();
+
+        // Get the variable orders the data was written to.
+        final PcjTables pcjs = new PcjTables();
+        final PcjMetadata pcjMetadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
+
+        // Scan Accumulo for the stored results.
+        for(final VariableOrder varOrder : pcjMetadata.getVarOrders()) {
+            final Scanner scanner = accumuloConn.createScanner(pcjTableName, new Authorizations());
+            scanner.fetchColumnFamily( new Text(varOrder.toString()) );
+
+            for(final Entry<Key, Value> entry : scanner) {
+                final byte[] serializedResult = entry.getKey().getRow().getBytes();
+                final BindingSet result = converter.convert(serializedResult, varOrder);
+                fetchedResults.put(varOrder.toString(), result);
+            }
+        }
+
+        return fetchedResults;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIntegrationTest.java b/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIntegrationTest.java
deleted file mode 100644
index e43ab83..0000000
--- a/extras/rya.indexing.pcj/src/test/java/org/apache/rya/indexing/pcj/storage/accumulo/PcjTablesIntegrationTest.java
+++ /dev/null
@@ -1,573 +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.rya.indexing.pcj.storage.accumulo;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-import org.apache.rya.accumulo.AccumuloRyaDAO;
-import org.apache.rya.accumulo.MiniAccumuloClusterInstance;
-import org.apache.rya.accumulo.MiniAccumuloSingleton;
-import org.apache.rya.accumulo.RyaTestInstanceRule;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
-import org.apache.rya.indexing.pcj.storage.PcjException;
-import org.apache.rya.indexing.pcj.storage.PcjMetadata;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.CloseableIterator;
-import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage.PCJStorageException;
-import org.apache.rya.indexing.pcj.storage.accumulo.BindingSetConverter.BindingSetConversionException;
-import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
-import org.apache.rya.rdftriplestore.RyaSailRepository;
-import org.apache.zookeeper.ClientCnxn;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.openrdf.model.Statement;
-import org.openrdf.model.impl.LiteralImpl;
-import org.openrdf.model.impl.NumericLiteralImpl;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.vocabulary.XMLSchema;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.impl.MapBindingSet;
-import org.openrdf.repository.RepositoryConnection;
-import org.openrdf.repository.RepositoryException;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
-
-/**
- * Performs integration test using {@link MiniAccumuloCluster} to ensure the
- * functions of {@link PcjTables} work within a cluster setting.
- */
-public class PcjTablesIntegrationTest {
-
-    private static final String USE_MOCK_INSTANCE = ".useMockInstance";
-    private static final String CLOUDBASE_INSTANCE = "sc.cloudbase.instancename";
-    private static final String CLOUDBASE_USER = "sc.cloudbase.username";
-    private static final String CLOUDBASE_PASSWORD = "sc.cloudbase.password";
-
-    private static final AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
-
-    // The MiniAccumuloCluster is re-used between tests.
-    private MiniAccumuloClusterInstance cluster = MiniAccumuloSingleton.getInstance();
-
-    // Rya data store and connections.
-    protected RyaSailRepository ryaRepo = null;
-    protected RepositoryConnection ryaConn = null;
-
-    @Rule
-    public RyaTestInstanceRule testInstance = new RyaTestInstanceRule(false);
-
-    @BeforeClass
-    public static void killLoudLogs() {
-        Logger.getLogger(ClientCnxn.class).setLevel(Level.ERROR);
-    }
-
-    @Before
-    public void resetTestEnvironmanet() throws AccumuloException, AccumuloSecurityException, TableNotFoundException, RepositoryException, IOException, InterruptedException {
-        // Setup the Rya library to use the Mini Accumulo.
-        ryaRepo = setupRya();
-        ryaConn = ryaRepo.getConnection();
-    }
-
-    @After
-    public void shutdownMiniCluster() throws IOException, InterruptedException, RepositoryException {
-        // Stop Rya.
-        if (ryaRepo != null) {
-            ryaRepo.shutDown();
-        }
-    }
-
-    private String getRyaInstanceName() {
-        return testInstance.getRyaInstanceName();
-    }
-    
-    /**
-     * Format a Mini Accumulo to be a Rya repository.
-     *
-     * @return The Rya repository sitting on top of the Mini Accumulo.
-     */
-    private RyaSailRepository setupRya() throws AccumuloException, AccumuloSecurityException, RepositoryException {
-        // Setup the Rya Repository that will be used to create Repository Connections.
-        final RdfCloudTripleStore ryaStore = new RdfCloudTripleStore();
-        final AccumuloRyaDAO crdfdao = new AccumuloRyaDAO();
-        crdfdao.setConnector( cluster.getConnector() );
-
-        // Setup Rya configuration values.
-        final AccumuloRdfConfiguration conf = new AccumuloRdfConfiguration();
-        conf.setTablePrefix(getRyaInstanceName());
-        conf.setDisplayQueryPlan(true);
-
-        conf.setBoolean(USE_MOCK_INSTANCE, false);
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, getRyaInstanceName());
-        conf.set(CLOUDBASE_USER, cluster.getUsername());
-        conf.set(CLOUDBASE_PASSWORD, cluster.getPassword());
-        conf.set(CLOUDBASE_INSTANCE, cluster.getInstanceName());
-
-        crdfdao.setConf(conf);
-        ryaStore.setRyaDAO(crdfdao);
-
-        final RyaSailRepository ryaRepo = new RyaSailRepository(ryaStore);
-        ryaRepo.initialize();
-
-        return ryaRepo;
-    }
-
-    /**
-     * Ensure that when a new PCJ table is created, it is initialized with the
-     * correct metadata values.
-     * <p>
-     * The method being tested is {@link PcjTables#createPcjTable(Connector, String, Set, String)}
-     */
-    @Test
-    public void createPcjTable() throws PcjException, AccumuloException, AccumuloSecurityException {
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Create a PCJ table in the Mini Accumulo.
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
-
-        // Fetch the PcjMetadata and ensure it has the correct values.
-        final PcjMetadata pcjMetadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-
-        // Ensure the metadata matches the expected value.
-        final PcjMetadata expected = new PcjMetadata(sparql, 0L, varOrders);
-        assertEquals(expected, pcjMetadata);
-    }
-
-    /**
-     * Ensure when results have been written to the PCJ table that they are in Accumulo.
-     * <p>
-     * The method being tested is {@link PcjTables#addResults(Connector, String, java.util.Collection)}
-     */
-    @Test
-    public void addResults() throws PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Create a PCJ table in the Mini Accumulo.
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
-
-        // Add a few results to the PCJ table.
-        final MapBindingSet alice = new MapBindingSet();
-        alice.addBinding("name", new URIImpl("http://Alice"));
-        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
-
-        final MapBindingSet bob = new MapBindingSet();
-        bob.addBinding("name", new URIImpl("http://Bob"));
-        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
-
-        final MapBindingSet charlie = new MapBindingSet();
-        charlie.addBinding("name", new URIImpl("http://Charlie"));
-        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
-
-        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
-        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
-                new VisibilityBindingSet(alice),
-                new VisibilityBindingSet(bob),
-                new VisibilityBindingSet(charlie)));
-
-        // Make sure the cardinality was updated.
-        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-        assertEquals(3, metadata.getCardinality());
-
-        // Scan Accumulo for the stored results.
-        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
-
-        // Ensure the expected results match those that were stored.
-        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
-        expectedResults.putAll("name;age", results);
-        expectedResults.putAll("age;name", results);
-        assertEquals(expectedResults, fetchedResults);
-    }
-
-    @Test
-    public void listResults() throws Exception {
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Create a PCJ table in the Mini Accumulo.
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
-
-        // Add a few results to the PCJ table.
-        final MapBindingSet alice = new MapBindingSet();
-        alice.addBinding("name", new URIImpl("http://Alice"));
-        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
-
-        final MapBindingSet bob = new MapBindingSet();
-        bob.addBinding("name", new URIImpl("http://Bob"));
-        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
-
-        final MapBindingSet charlie = new MapBindingSet();
-        charlie.addBinding("name", new URIImpl("http://Charlie"));
-        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
-
-        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
-                new VisibilityBindingSet(alice),
-                new VisibilityBindingSet(bob),
-                new VisibilityBindingSet(charlie)));
-
-        // Fetch the Binding Sets that have been stored in the PCJ table.
-        final Set<BindingSet> results = new HashSet<>();
-
-        final CloseableIterator<BindingSet> resultsIt = pcjs.listResults(accumuloConn, pcjTableName, new Authorizations());
-        try {
-            while(resultsIt.hasNext()) {
-                results.add( resultsIt.next() );
-            }
-        } finally {
-            resultsIt.close();
-        }
-
-        // Verify the fetched results match the expected ones.
-        final Set<BindingSet> expected = Sets.<BindingSet>newHashSet(alice, bob, charlie);
-        assertEquals(expected, results);
-    }
-
-    /**
-     * Ensure when results are already stored in Rya, that we are able to populate
-     * the PCJ table for a new SPARQL query using those results.
-     * <p>
-     * The method being tested is: {@link PcjTables#populatePcj(Connector, String, RepositoryConnection, String)}
-     */
-    @Test
-    public void populatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
-        // Load some Triples into Rya.
-        final Set<Statement> triples = new HashSet<>();
-        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://hasAge"), new NumericLiteralImpl(16, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://hasAge"), new NumericLiteralImpl(12, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://hasAge"), new NumericLiteralImpl(43, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-
-        for(final Statement triple : triples) {
-            ryaConn.add(triple);
-        }
-
-        // Create a PCJ table that will include those triples in its results.
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
-
-        // Populate the PCJ table using a Rya connection.
-        pcjs.populatePcj(accumuloConn, pcjTableName, ryaConn);
-
-        // Scan Accumulo for the stored results.
-        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
-
-        // Make sure the cardinality was updated.
-        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-        assertEquals(3, metadata.getCardinality());
-
-        // Ensure the expected results match those that were stored.
-        final MapBindingSet alice = new MapBindingSet();
-        alice.addBinding("name", new URIImpl("http://Alice"));
-        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
-
-        final MapBindingSet bob = new MapBindingSet();
-        bob.addBinding("name", new URIImpl("http://Bob"));
-        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
-
-        final MapBindingSet charlie = new MapBindingSet();
-        charlie.addBinding("name", new URIImpl("http://Charlie"));
-        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
-
-        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
-
-        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
-        expectedResults.putAll("name;age", results);
-        expectedResults.putAll("age;name", results);
-        assertEquals(expectedResults, fetchedResults);
-    }
-
-    /**
-     * Ensure the method that creates a new PCJ table, scans Rya for matches, and
-     * stores them in the PCJ table works.
-     * <p>
-     * The method being tested is: {@link PcjTables#createAndPopulatePcj(RepositoryConnection, Connector, String, String, String[], Optional)}
-     */
-    @Test
-    public void createAndPopulatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException, AccumuloException, AccumuloSecurityException {
-        // Load some Triples into Rya.
-        final Set<Statement> triples = new HashSet<>();
-        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://hasAge"), new NumericLiteralImpl(16, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Bob"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://hasAge"), new NumericLiteralImpl(12, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Charlie"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://hasAge"), new NumericLiteralImpl(43, XMLSchema.INTEGER)) );
-        triples.add( new StatementImpl(new URIImpl("http://Eve"), new URIImpl("http://playsSport"), new LiteralImpl("Soccer")) );
-
-        for(final Statement triple : triples) {
-            ryaConn.add(triple);
-        }
-
-        // Create a PCJ table that will include those triples in its results.
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-
-        // Create and populate the PCJ table.
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createAndPopulatePcj(ryaConn, accumuloConn, pcjTableName, sparql, new String[]{"name", "age"}, Optional.<PcjVarOrderFactory>absent());
-
-        // Make sure the cardinality was updated.
-        final PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-        assertEquals(3, metadata.getCardinality());
-
-        // Scan Accumulo for the stored results.
-        final Multimap<String, BindingSet> fetchedResults = loadPcjResults(accumuloConn, pcjTableName);
-
-        // Ensure the expected results match those that were stored.
-        final MapBindingSet alice = new MapBindingSet();
-        alice.addBinding("name", new URIImpl("http://Alice"));
-        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
-
-        final MapBindingSet bob = new MapBindingSet();
-        bob.addBinding("name", new URIImpl("http://Bob"));
-        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
-
-        final MapBindingSet charlie = new MapBindingSet();
-        charlie.addBinding("name", new URIImpl("http://Charlie"));
-        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
-
-        final Set<BindingSet> results = Sets.<BindingSet>newHashSet(alice, bob, charlie);
-
-        final Multimap<String, BindingSet> expectedResults = HashMultimap.create();
-        expectedResults.putAll("name;age", results);
-        expectedResults.putAll("age;name", results);
-
-        assertEquals(expectedResults, fetchedResults);
-    }
-
-    @Test
-    public void listPcjs() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Set up the table names that will be used.
-        final String instance1 = "instance1_";
-        final String instance2 = "instance2_";
-
-        final String instance1_table1 = new PcjTableNameFactory().makeTableName(instance1, "table1");
-        final String instance1_table2 = new PcjTableNameFactory().makeTableName(instance1, "table2");
-        final String instance1_table3 = new PcjTableNameFactory().makeTableName(instance1, "table3");
-
-        final String instance2_table1 = new PcjTableNameFactory().makeTableName(instance2, "table1");
-
-        // Create the PCJ Tables that are in instance 1 and instance 2.
-        final Set<VariableOrder> varOrders = Sets.<VariableOrder>newHashSet( new VariableOrder("x") );
-        final String sparql = "SELECT x WHERE ?x <http://isA> <http://Food>";
-
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, instance1_table1, varOrders, sparql);
-        pcjs.createPcjTable(accumuloConn, instance1_table2, varOrders, sparql);
-        pcjs.createPcjTable(accumuloConn, instance1_table3, varOrders, sparql);
-
-        pcjs.createPcjTable(accumuloConn, instance2_table1, varOrders, sparql);
-
-        // Ensure all of the names have been stored for instance 1 and 2.
-        final Set<String> expected1 = Sets.newHashSet(instance1_table1, instance1_table2, instance1_table3);
-        final Set<String> instance1Tables = Sets.newHashSet( pcjs.listPcjTables(accumuloConn, instance1) );
-        assertEquals(expected1, instance1Tables);
-
-        final Set<String> expected2 = Sets.newHashSet(instance2_table1);
-        final Set<String> instance2Tables = Sets.newHashSet( pcjs.listPcjTables(accumuloConn, instance2) );
-        assertEquals(expected2, instance2Tables);
-    }
-
-    @Test
-    public void purge() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
-        final String sparql =
-                "SELECT ?name ?age " +
-                "{" +
-                  "FILTER(?age < 30) ." +
-                  "?name <http://hasAge> ?age." +
-                  "?name <http://playsSport> \"Soccer\" " +
-                "}";
-
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Create a PCJ table in the Mini Accumulo.
-        final String pcjTableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "testPcj");
-        final Set<VariableOrder> varOrders = new ShiftVarOrderFactory().makeVarOrders(new VariableOrder("name;age"));
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
-
-        // Add a few results to the PCJ table.
-        final MapBindingSet alice = new MapBindingSet();
-        alice.addBinding("name", new URIImpl("http://Alice"));
-        alice.addBinding("age", new NumericLiteralImpl(14, XMLSchema.INTEGER));
-
-        final MapBindingSet bob = new MapBindingSet();
-        bob.addBinding("name", new URIImpl("http://Bob"));
-        bob.addBinding("age", new NumericLiteralImpl(16, XMLSchema.INTEGER));
-
-        final MapBindingSet charlie = new MapBindingSet();
-        charlie.addBinding("name", new URIImpl("http://Charlie"));
-        charlie.addBinding("age", new NumericLiteralImpl(12, XMLSchema.INTEGER));
-
-        pcjs.addResults(accumuloConn, pcjTableName, Sets.<VisibilityBindingSet>newHashSet(
-                new VisibilityBindingSet(alice),
-                new VisibilityBindingSet(bob),
-                new VisibilityBindingSet(charlie)));
-
-        // Make sure the cardinality was updated.
-        PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-        assertEquals(3, metadata.getCardinality());
-
-        // Purge the data.
-        pcjs.purgePcjTable(accumuloConn, pcjTableName);
-
-        // Make sure the cardinality was updated to 0.
-        metadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-        assertEquals(0, metadata.getCardinality());
-    }
-
-    @Test
-    public void dropPcj() throws PCJStorageException, AccumuloException, AccumuloSecurityException {
-        final Connector accumuloConn = cluster.getConnector();
-
-        // Create a PCJ index.
-        final String tableName = new PcjTableNameFactory().makeTableName(getRyaInstanceName(), "thePcj");
-        final Set<VariableOrder> varOrders = Sets.<VariableOrder>newHashSet( new VariableOrder("x") );
-        final String sparql = "SELECT x WHERE ?x <http://isA> <http://Food>";
-
-        final PcjTables pcjs = new PcjTables();
-        pcjs.createPcjTable(accumuloConn, tableName, varOrders, sparql);
-
-        // Fetch its metadata to show that it has actually been created.
-        final PcjMetadata expectedMetadata = new PcjMetadata(sparql, 0L, varOrders);
-        PcjMetadata metadata = pcjs.getPcjMetadata(accumuloConn, tableName);
-        assertEquals(expectedMetadata, metadata);
-
-        // Drop it.
-        pcjs.dropPcjTable(accumuloConn, tableName);
-
-        // Show the metadata is no longer present.
-        PCJStorageException tableDoesNotExistException = null;
-        try {
-            metadata = pcjs.getPcjMetadata(accumuloConn, tableName);
-        } catch(final PCJStorageException e) {
-            tableDoesNotExistException = e;
-        }
-        assertNotNull(tableDoesNotExistException);
-    }
-
-    /**
-     * Scan accumulo for the results that are stored in a PCJ table. The
-     * multimap stores a set of deserialized binding sets that were in the PCJ
-     * table for every variable order that is found in the PCJ metadata.
-     */
-    private static Multimap<String, BindingSet> loadPcjResults(final Connector accumuloConn, final String pcjTableName) throws PcjException, TableNotFoundException, BindingSetConversionException {
-        final Multimap<String, BindingSet> fetchedResults = HashMultimap.create();
-
-        // Get the variable orders the data was written to.
-        final PcjTables pcjs = new PcjTables();
-        final PcjMetadata pcjMetadata = pcjs.getPcjMetadata(accumuloConn, pcjTableName);
-
-        // Scan Accumulo for the stored results.
-        for(final VariableOrder varOrder : pcjMetadata.getVarOrders()) {
-            final Scanner scanner = accumuloConn.createScanner(pcjTableName, new Authorizations());
-            scanner.fetchColumnFamily( new Text(varOrder.toString()) );
-
-            for(final Entry<Key, Value> entry : scanner) {
-                final byte[] serializedResult = entry.getKey().getRow().getBytes();
-                final BindingSet result = converter.convert(serializedResult, varOrder);
-                fetchedResults.put(varOrder.toString(), result);
-            }
-        }
-
-        return fetchedResults;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/9c12630b/osgi/camel.rya/src/test/java/org/apache/rya/camel/cbsail/CbSailIntegrationTest.java
----------------------------------------------------------------------
diff --git a/osgi/camel.rya/src/test/java/org/apache/rya/camel/cbsail/CbSailIntegrationTest.java b/osgi/camel.rya/src/test/java/org/apache/rya/camel/cbsail/CbSailIntegrationTest.java
deleted file mode 100644
index 4928886..0000000
--- a/osgi/camel.rya/src/test/java/org/apache/rya/camel/cbsail/CbSailIntegrationTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-package org.apache.rya.camel.cbsail;
-
-/*
- * 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.apache.rya.camel.cbsail.CbSailComponent;
-import org.apache.camel.EndpointInject;
-import org.apache.camel.Exchange;
-import org.apache.camel.Processor;
-import org.apache.camel.ProducerTemplate;
-import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.test.CamelTestSupport;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import java.util.HashMap;
-
-public class CbSailIntegrationTest extends CamelTestSupport {
-
-    @EndpointInject(uri = "cbsail:tquery?server=stratus13&port=2181&user=root&pwd=password&instanceName=stratus")
-    ProducerTemplate producer;
-
-    public void testCbSail() throws Exception {
-        String underGradInfo = "PREFIX rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#>" +
-                        " PREFIX ub: <urn:test:onto:univ#>" +
-                        " SELECT * WHERE" +
-                        " {" +
-                        "       <http://www.Department0.University0.edu/UndergraduateStudent600> ?pred ?obj ." +
-                        " }";
-        HashMap map = new HashMap();
-        map.put(CbSailComponent.SPARQL_QUERY_PROP, underGradInfo);
-        map.put(CbSailComponent.START_TIME_QUERY_PROP, 0l);
-        map.put(CbSailComponent.TTL_QUERY_PROP, 86400000l);
-        Object o = producer.requestBodyAndHeaders(null, map);
-        System.out.println(o);
-        Thread.sleep(100000);
-    }
-
-    @Override
-    protected RouteBuilder createRouteBuilder() {
-        return new RouteBuilder() {
-
-            @Override
-            public void configure() throws Exception {
-                ValueFactory vf = new ValueFactoryImpl();
-                String underGradInfo = "PREFIX rdf: <http://www.w3.org/1999/02/22-rdf-syntax-ns#>" +
-                        " PREFIX ub: <urn:test:onto:univ#>" +
-                        " SELECT * WHERE" +
-                        " {" +
-                        "       <http://www.Department0.University0.edu/UndergraduateStudent60> ?pred ?obj ." +
-                        " }";
-                String rawEvents = "PREFIX nh: <http://org.apache.com/2011/02/nh#>\n" +
-                        " SELECT * WHERE\n" +
-                        " {\n" +
-                        "     ?uuid nh:timestamp ?timestamp.\n" +
-                        "     ?uuid nh:site ?site;\n" +
-                        "          nh:system ?system;\n" +
-                        "          nh:dataSupplier ?dataSupplier;\n" +
-                        "          nh:dataType ?dataType;\n" +
-                        "          <http://org.apache.com/2011/02/nh#count> ?data.\n" +
-                        " } LIMIT 100";
-                String latestModels = "PREFIX nh: <http://org.apache.com/rdf/2011/02/model#>" +
-                        " PREFIX xsd: <http://www.w3.org/2001/XMLSchema#>" +
-                        " SELECT * WHERE" +
-                        " {" +
-                        "     ?modelUuid nh:dayOfWeek \"5\";" +
-                        "          nh:hourOfDay \"3\";" +
-                        "          nh:timestamp ?timestamp;" +
-//                        "     FILTER (xsd:integer(?timestamp) > 1297652964633)." +
-                        "          nh:dataProperty \"count\";" +
-                        "          nh:modelType \"org.apache.learning.tpami.SimpleGaussianMMModel\";" +
-                        "          nh:site ?site;" +
-                        "          nh:dataSupplier ?dataSupplier;" +
-                        "          nh:system ?system;" +
-                        "          nh:dataType ?dataType;" +
-                        "          nh:model ?model;" +
-                        "          nh:key ?key." +
-                        " }";
-
-                from("timer://foo?fixedRate=true&period=60000").
-                        setHeader(CbSailComponent.SPARQL_QUERY_PROP, constant(underGradInfo)).
-//        setBody(constant(new StatementImpl(vf.createURI("http://www.Department0.University0.edu/UndergraduateStudent610"), vf.createURI("urn:test:onto:univ#testPred"), vf.createLiteral("test")))).
-                        to("cbsail:tquery?server=stratus13&port=2181&user=root&pwd=password&instanceName=stratus&queryOutput=XML" +
-//        "&ttl=259200000"
-//        + "&sparql=" + latestModels" +
-                        "").process(new Processor() {
-
-                    @Override
-                    public void process(Exchange exchange) throws Exception {
-                        System.out.println(exchange.getIn().getBody());
-//                        if (body != null)
-//                            System.out.println(body.size());
-                    }
-                }).end();
-            }
-        };
-    }
-
-}