You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by dl...@apache.org on 2018/01/04 22:07:23 UTC

[01/17] incubator-rya git commit: RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.

Repository: incubator-rya
Updated Branches:
  refs/heads/master 62de7c5d1 -> 4576f556a


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 2f3214d..52dd064 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
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.indexing.statement.metadata;
 
+import static org.junit.Assert.assertEquals;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -29,11 +31,11 @@ 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.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
 import org.apache.rya.mongodb.MongoTestBase;
 import org.apache.rya.sail.config.RyaSailFactory;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.impl.URIImpl;
@@ -50,10 +52,11 @@ import org.openrdf.sail.Sail;
 
 public class MongoStatementMetadataIT extends MongoTestBase {
 
-    private Sail sail;
-    private SailRepository repo;
-    private SailRepositoryConnection conn;
-    private MongoDBRyaDAO dao;
+//    private Sail sail;
+//    private SailRepository repo;
+//    private SailRepositoryConnection conn;
+//    private MongoDBRyaDAO dao;
+
     private final String query1 = "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 ?a ?b ?c where {_:blankNode1 rdf:type owl:Annotation; ano:Source ?a; "
@@ -61,46 +64,66 @@ public class MongoStatementMetadataIT extends MongoTestBase {
             + "_:blankNode2 rdf:type owl:Annotation; ano:Source ?a; "
             + "ano:Property <http://talksTo>; ano:Target ?b; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date }";
 
-    @Before
-    public void init() throws Exception {
-        final Set<RyaURI> propertySet = new HashSet<RyaURI>(
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        final Set<RyaURI> propertySet = new HashSet<>(
                 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, super.getMongoClient());
-        dao.init();
     }
 
+//    @Before
+//    public void init() throws Exception {
+//        final Set<RyaURI> propertySet = new HashSet<>(
+//                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();
+//        dao.setConf(conf);
+//        dao.init();
+//    }
+
     @Test
     public void simpleQueryWithoutBindingSet() throws Exception {
-        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"));
-
-        final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement);
-
-        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
-
-        final QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-        bs.addBinding("y", new LiteralImpl("Joe"));
-
-        final List<BindingSet> bsList = new ArrayList<>();
-        while (result.hasNext()) {
-            bsList.add(result.next());
+        Sail sail = RyaSailFactory.getInstance(conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            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"));
+
+            final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
+
+            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+
+            final QueryBindingSet bs = new QueryBindingSet();
+            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bs.addBinding("y", new LiteralImpl("Joe"));
+
+            final List<BindingSet> bsList = new ArrayList<>();
+            while (result.hasNext()) {
+                bsList.add(result.next());
+            }
+
+            System.out.println(bsList);
+            assertEquals(1, bsList.size());
+            assertEquals(bs, bsList.get(0));
+            dao.delete(statement, conf);
+        } finally {
+//            dao.destroy();
+//            sail.shutDown();
         }
-
-        System.out.println(bsList);
-        Assert.assertEquals(1, bsList.size());
-        Assert.assertEquals(bs, bsList.get(0));
-        dao.delete(statement, conf);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataExternalSetProviderTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataExternalSetProviderTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataExternalSetProviderTest.java
index 5e28a5b..f3d0c1e 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataExternalSetProviderTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataExternalSetProviderTest.java
@@ -7,9 +7,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
@@ -73,7 +73,7 @@ public class StatementMetadataExternalSetProviderTest {
 
         patterns.addAll(StatementPatternCollector.process(pq.getTupleExpr()));
         JoinSegment<StatementMetadataNode<?>> segment = new JoinSegment<>(
-                new HashSet<QueryModelNode>(patterns), patterns, new HashMap<ValueExpr, Filter>());
+                new HashSet<>(patterns), patterns, new HashMap<ValueExpr, Filter>());
         List<StatementMetadataNode<?>> extSets = metaProvider.getExternalSets(segment);
 
         expected.add(new StatementMetadataNode<>(sp, conf));
@@ -81,7 +81,7 @@ public class StatementMetadataExternalSetProviderTest {
         Assert.assertEquals(expected, extSets);
 
     }
-    
+
     @Test
     public void createSingleMongoMetadataNode() throws MalformedQueryException {
 
@@ -99,7 +99,7 @@ public class StatementMetadataExternalSetProviderTest {
 
         patterns.addAll(StatementPatternCollector.process(pq.getTupleExpr()));
         JoinSegment<StatementMetadataNode<?>> segment = new JoinSegment<>(
-                new HashSet<QueryModelNode>(patterns), patterns, new HashMap<ValueExpr, Filter>());
+                new HashSet<>(patterns), patterns, new HashMap<ValueExpr, Filter>());
         List<StatementMetadataNode<?>> extSets = metaProvider.getExternalSets(segment);
 
         expected.add(new StatementMetadataNode<>(sp,conf));
@@ -107,8 +107,8 @@ public class StatementMetadataExternalSetProviderTest {
         Assert.assertEquals(expected, extSets);
 
     }
-    
-    
+
+
     @Test
     public void createMultipleMetadataNode() throws MalformedQueryException {
 
@@ -133,7 +133,7 @@ public class StatementMetadataExternalSetProviderTest {
 
         patterns.addAll(StatementPatternCollector.process(pq2.getTupleExpr()));
         JoinSegment<StatementMetadataNode<?>> segment = new JoinSegment<>(
-                new HashSet<QueryModelNode>(patterns), patterns, new HashMap<ValueExpr, Filter>());
+                new HashSet<>(patterns), patterns, new HashMap<ValueExpr, Filter>());
         List<StatementMetadataNode<?>> extSets = metaProvider.getExternalSets(segment);
 
         expected.add(new StatementMetadataNode<>(sp1,conf));
@@ -147,7 +147,7 @@ public class StatementMetadataExternalSetProviderTest {
         if (useMongo) {
             MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration();
             conf.setBoolean("sc.useMongo", true);
-            conf.setMongoInstance("localhost");
+            conf.setMongoHostname("localhost");
             conf.setMongoPort("27017");
             conf.setMongoDBName("rya_");
             return conf;
@@ -164,7 +164,7 @@ public class StatementMetadataExternalSetProviderTest {
             return conf;
         }
     }
-    
+
     private void removePatternWithGivenSubject(String subject, Set<StatementPattern> patterns) {
         Iterator<StatementPattern> spIter = patterns.iterator();
         while(spIter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataOptimizerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataOptimizerTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataOptimizerTest.java
index d4c9436..4a65d99 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataOptimizerTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/StatementMetadataOptimizerTest.java
@@ -7,9 +7,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
@@ -90,8 +90,8 @@ public class StatementMetadataOptimizerTest {
 
     @Before
     public void init() {
-        RdfCloudTripleStoreConfiguration mongoConf = (RdfCloudTripleStoreConfiguration) getConf(true);
-        RdfCloudTripleStoreConfiguration accumuloConf = (RdfCloudTripleStoreConfiguration) getConf(false);
+        RdfCloudTripleStoreConfiguration mongoConf = getConf(true);
+        RdfCloudTripleStoreConfiguration accumuloConf = getConf(false);
         mongoOptimizer = new StatementMetadataOptimizer(mongoConf);
         accumuloOptimizer = new StatementMetadataOptimizer(accumuloConf);
     }
@@ -127,12 +127,12 @@ public class StatementMetadataOptimizerTest {
     private static RdfCloudTripleStoreConfiguration getConf(boolean useMongo) {
 
         RdfCloudTripleStoreConfiguration conf;
-        Set<RyaURI> propertySet = new HashSet<RyaURI>(
+        Set<RyaURI> propertySet = new HashSet<>(
                 Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
         if (useMongo) {
             MongoDBRdfConfiguration mConf = new MongoDBRdfConfiguration();
             mConf.setBoolean("sc.useMongo", true);
-            mConf.setMongoInstance("localhost");
+            mConf.setMongoHostname("localhost");
             mConf.setMongoPort("27017");
             mConf.setMongoDBName("rya_");
             conf = mConf;
@@ -152,9 +152,9 @@ public class StatementMetadataOptimizerTest {
     private static Set<StatementMetadataNode<?>> getExpected(String query) throws MalformedQueryException {
         ParsedQuery pq = parser.parseQuery(query, null);
         StatementMetadataExternalSetProvider provider = new StatementMetadataExternalSetProvider(
-                (RdfCloudTripleStoreConfiguration) getConf(false));
+                getConf(false));
         List<StatementPattern> patterns = StatementPatternCollector.process(pq.getTupleExpr());
-        JoinSegment<StatementMetadataNode<?>> segment = new JoinSegment<StatementMetadataNode<?>>(
+        JoinSegment<StatementMetadataNode<?>> segment = new JoinSegment<>(
                 new HashSet<QueryModelNode>(patterns), new ArrayList<QueryModelNode>(patterns),
                 new HashMap<ValueExpr, Filter>());
         return new HashSet<>(provider.getExternalSets(segment));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexingExample/src/main/java/InferenceExamples.java
----------------------------------------------------------------------
diff --git a/extras/indexingExample/src/main/java/InferenceExamples.java b/extras/indexingExample/src/main/java/InferenceExamples.java
index d1d9dc4..5b5a937 100644
--- a/extras/indexingExample/src/main/java/InferenceExamples.java
+++ b/extras/indexingExample/src/main/java/InferenceExamples.java
@@ -31,16 +31,11 @@ import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration.MongoDBIndexingConfigBuilder;
 import org.apache.rya.mongodb.EmbeddedMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
 import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
 import org.apache.rya.sail.config.RyaSailFactory;
 import org.apache.zookeeper.ClientCnxn;
-import org.openrdf.model.Namespace;
-import org.openrdf.model.URI;
-import org.openrdf.model.ValueFactory;
 import org.openrdf.model.vocabulary.OWL;
-import org.openrdf.model.vocabulary.RDF;
 import org.openrdf.model.vocabulary.RDFS;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.MalformedQueryException;
@@ -53,7 +48,6 @@ import org.openrdf.query.TupleQueryResultHandlerException;
 import org.openrdf.query.Update;
 import org.openrdf.query.UpdateExecutionException;
 import org.openrdf.repository.RepositoryException;
-import org.openrdf.repository.RepositoryResult;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
@@ -106,7 +100,7 @@ public class InferenceExamples {
 	            repository = new SailRepository(sail);
 	            conn = repository.getConnection();
 
-	            
+
 	            final long start = System.currentTimeMillis();
 
 	                testInfer(conn, sail);
@@ -122,7 +116,6 @@ public class InferenceExamples {
 	            log.info("Shutting down");
 	            closeQuietly(conn);
 	            closeQuietly(repository);
-	            MongoConnectorFactory.closeMongoClient();
 	        }
 	    }
 
@@ -152,7 +145,7 @@ public class InferenceExamples {
 	       //     .setUseMockMongo(USE_MOCK).setUseInference(USE_INFER).setAuths("U");
 	        MongoDBIndexingConfigBuilder builder = MongoIndexingConfiguration.builder()
 		            .setUseMockMongo(USE_EMBEDDED_MONGO).setUseInference(true).setAuths("U");
-	        
+
 	        if (USE_EMBEDDED_MONGO) {
 	            final MongoClient c = EmbeddedMongoFactory.newFactory().newMongoClient();
 	            final ServerAddress address = c.getAddress();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 f8927d4..477e579 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -31,7 +31,6 @@ import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration.MongoDBIndexingConfigBuilder;
 import org.apache.rya.mongodb.EmbeddedMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
 import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
 import org.apache.rya.sail.config.RyaSailFactory;
@@ -132,7 +131,6 @@ public class MongoRyaDirectExample {
             log.info("Shutting down");
             closeQuietly(conn);
             closeQuietly(repository);
-            MongoConnectorFactory.closeMongoClient();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/ITBase.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/ITBase.java b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/ITBase.java
index e7f020d..60fff56 100644
--- a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/ITBase.java
+++ b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/ITBase.java
@@ -265,7 +265,7 @@ public abstract class ITBase {
         conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
 
         conf.setMongoPort(""+port);
-        conf.setMongoInstance(hostname);
+        conf.setMongoHostname(hostname);
         conf.setMongoDBName(ryaInstanceName);
         return conf;
     }


[04/17] incubator-rya git commit: RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.

Posted by dl...@apache.org.
RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.


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

Branch: refs/heads/master
Commit: 767349dac9822cd13e92f9b117d1b5d2dad13e3d
Parents: 62de7c5
Author: kchilton2 <ke...@gmail.com>
Authored: Fri Dec 22 20:52:27 2017 -0500
Committer: kchilton2 <ke...@gmail.com>
Committed: Wed Dec 27 14:08:11 2017 -0500

----------------------------------------------------------------------
 .../AbstractMongoDBRdfConfigurationBuilder.java |  39 +-
 .../rya/mongodb/EmbeddedMongoFactory.java       |  25 +-
 .../rya/mongodb/MongoConnectorFactory.java      | 134 ----
 .../apache/rya/mongodb/MongoDBQueryEngine.java  |  48 +-
 .../rya/mongodb/MongoDBRdfConfiguration.java    | 275 ++++---
 .../org/apache/rya/mongodb/MongoDBRyaDAO.java   |  71 +-
 .../StatefulMongoDBRdfConfiguration.java        |  79 ++
 .../mongodb/dao/MongoDBNamespaceManager.java    |  20 +-
 .../dao/SimpleMongoDBNamespaceManager.java      |  12 +-
 .../rya/mongodb/EmbeddedMongoSingleton.java     |  55 +-
 .../rya/mongodb/MongoDBQueryEngineTest.java     | 152 ++--
 .../mongodb/MongoDBRdfConfigurationTest.java    |  49 +-
 .../rya/mongodb/MongoDBRyaBatchWriterIT.java    |  34 +-
 .../org/apache/rya/mongodb/MongoDBRyaDAOIT.java | 654 ++++++++--------
 .../apache/rya/mongodb/MongoDBRyaDAOTest.java   | 147 ++--
 .../apache/rya/mongodb/MongoRyaTestBase.java    |  73 --
 .../org/apache/rya/mongodb/MongoTestBase.java   |  36 +-
 .../src/test/resources/log4j.properties         |  27 +
 .../indexing/entity/EntityIndexOptimizer.java   |   5 +
 .../entity/update/mongo/MongoEntityIndexer.java |  31 +-
 .../indexing/mongodb/AbstractMongoIndexer.java  |  36 +-
 .../rya/indexing/mongodb/MongoDbSmartUri.java   |  19 +-
 .../matching/RyaQueryEngineFactory.java         |  29 +-
 .../apache/rya/sail/config/RyaSailFactory.java  | 167 ++++-
 .../rya/indexing/mongo/MongoEntityIndexIT.java  | 183 +++--
 .../mongo/MongoFreeTextIndexerTest.java         |  58 +-
 .../mongo/MongoIndexingConfigurationTest.java   |   8 +-
 .../mongo/MongoTemporalIndexerTest.java         | 751 ++++++++++---------
 .../metadata/MongoStatementMetadataIT.java      |  97 ++-
 ...tatementMetadataExternalSetProviderTest.java |  20 +-
 .../StatementMetadataOptimizerTest.java         |  16 +-
 .../src/main/java/InferenceExamples.java        |  11 +-
 .../src/main/java/MongoRyaDirectExample.java    |   2 -
 .../org/apache/rya/indexing/export/ITBase.java  |   2 +-
 34 files changed, 1769 insertions(+), 1596 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
index f50bf9a..a2bd03f 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
@@ -59,11 +59,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
 
     /**
      * Sets Mongo user.
-     * 
+     *
      * @param user - user name used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoUser(String user) {
+    public B setMongoUser(final String user) {
         this.user = user;
         return confBuilder();
     }
@@ -71,11 +71,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * Sets password for Mongo user specified by
      * {@link AbstractMongoDBRdfConfigurationBuilder#setMongoUser(String)}.
-     * 
+     *
      * @param password - password used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoPassword(String password) {
+    public B setMongoPassword(final String password) {
         this.pass = password;
         return confBuilder();
     }
@@ -83,11 +83,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * Sets Mongo port. This parameter must be set to connect to an instance of
      * MongoDB and will default to "27017" if no value is specified.
-     * 
+     *
      * @param port - port used to connect Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoPort(String port) {
+    public B setMongoPort(final String port) {
         this.port = port;
         return confBuilder();
     }
@@ -95,11 +95,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * Sets Mongo host. This parameter must be set to connect to an instance of
      * MongoDB and will default to "localhost" if no value is specified.
-     * 
+     *
      * @param host - host used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoHost(String host) {
+    public B setMongoHost(final String host) {
         this.host = host;
         return confBuilder();
     }
@@ -107,11 +107,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * Sets MongoDB name. This parameter must be set to connect to an instance
      * of MongoDB and will default to "rya_triples" is no value is specified.
-     * 
+     *
      * @param name - name of MongoDB to connect to
      * @return specified builder for chaining method invocations
      */
-    public B setMongoDBName(String name) {
+    public B setMongoDBName(final String name) {
         this.mongoDBName = name;
         return confBuilder();
     }
@@ -120,11 +120,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * Sets MongoDB Collection prefix. This parameter must be set to connect to
      * an instance of MongoDB and will default to "rya_" is no value is
      * specified.
-     * 
+     *
      * @param name - name of Collection to connect to
      * @return specified builder for chaining method invocations
      */
-    public B setMongoCollectionPrefix(String prefix) {
+    public B setMongoCollectionPrefix(final String prefix) {
         this.mongoCollectionPrefix = prefix;
         return confBuilder();
     }
@@ -132,11 +132,11 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * Set whether to use instance of embedded Mongo as backend for Rya
      * instance.
-     * 
+     *
      * @param useMock - indicates whether to use embedded Mongo as Rya backing
      * @return specified builder for chaining method invocations
      */
-    public B setUseMockMongo(boolean useMock) {
+    public B setUseMockMongo(final boolean useMock) {
         this.useMock = useMock;
         return confBuilder();
     }
@@ -144,21 +144,22 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
     /**
      * @return extension of {@link MongoDBRdfConfiguration} with specified parameters set
      */
+    @Override
     public C build() {
         return getConf(super.build());
     }
 
     /**
      * Assigns builder values to appropriate parameters within the {@link Configuration} object.
-     * 
+     *
      * @param conf - Configuration object
      * @return - Configuration object with parameters set
      */
-    private C getConf(C conf) {
+    private C getConf(final C conf) {
 
         conf.setUseMock(useMock);
         conf.set("sc.useMongo", "true");
-        
+
         if (user != null) {
             conf.setMongoUser(user);
         }
@@ -166,9 +167,9 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
             conf.setMongoPassword(pass);
         }
         conf.setMongoDBName(mongoDBName);
-        conf.setCollectionName(mongoCollectionPrefix);
+        conf.setRyaInstance(mongoCollectionPrefix);
         conf.setTablePrefix(mongoCollectionPrefix);
-        conf.setMongoInstance(host);
+        conf.setMongoHostname(host);
         conf.setMongoPort(port);
 
         return conf;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
index f023739..d695ffa 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
@@ -6,9 +6,9 @@
  * 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
@@ -46,7 +46,7 @@ public class EmbeddedMongoFactory {
     public static EmbeddedMongoFactory newFactory() throws IOException {
         return EmbeddedMongoFactory.with(Version.Main.PRODUCTION);
     }
-    
+
     public static EmbeddedMongoFactory with(final IFeatureAwareVersion version) throws IOException {
         return new EmbeddedMongoFactory(version);
     }
@@ -56,9 +56,9 @@ public class EmbeddedMongoFactory {
 
     /**
      * Create the testing utility using the specified version of MongoDB.
-     * 
+     *
      * @param version
-     *            version of MongoDB.
+     *            - version of MongoDB.
      */
     private EmbeddedMongoFactory(final IFeatureAwareVersion version) throws IOException {
         final MongodStarter runtime = MongodStarter.getInstance(new RuntimeConfigBuilder().defaultsWithLogger(Command.MongoD, logger).build());
@@ -67,7 +67,7 @@ public class EmbeddedMongoFactory {
     }
 
     private IMongodConfig newMongodConfig(final IFeatureAwareVersion version) throws UnknownHostException, IOException {
-        Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
+        final Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
         return new MongodConfigBuilder().version(version).net(net).build();
     }
 
@@ -79,7 +79,7 @@ public class EmbeddedMongoFactory {
 
     /**
      * Creates a new Mongo connection.
-     * 
+     *
      * @throws MongoException
      * @throws UnknownHostException
      */
@@ -88,10 +88,19 @@ public class EmbeddedMongoFactory {
     }
 
     /**
+     * Gives access to the process configuration.
+     *
+     */
+    public IMongodConfig getMongoServerDetails() {
+        return mongodProcess.getConfig();
+    }
+
+
+    /**
      * Cleans up the resources created by the utility.
      */
     public void shutdown() {
         mongodProcess.stop();
         mongodExecutable.stop();
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoConnectorFactory.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoConnectorFactory.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoConnectorFactory.java
deleted file mode 100644
index 50c3a52..0000000
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoConnectorFactory.java
+++ /dev/null
@@ -1,134 +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.mongodb;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import org.apache.commons.configuration.ConfigurationRuntimeException;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.http.annotation.ThreadSafe;
-
-import com.mongodb.MongoClient;
-import com.mongodb.MongoCredential;
-import com.mongodb.MongoException;
-import com.mongodb.ServerAddress;
-
-import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import edu.umd.cs.findbugs.annotations.NonNull;
-/**
- * Mongo convention generally allows for a single instance of a {@link MongoClient}
- * throughout the life cycle of an application.  This MongoConnectorFactory lazy
- * loads a Mongo Client and uses the same one whenever {@link MongoConnectorFactory#getMongoClient(Configuration)}
- * is invoked.
- */
-@ThreadSafe
-@DefaultAnnotation(NonNull.class)
-public class MongoConnectorFactory {
-    private static MongoClient mongoClient;
-
-    private final static String MSG_INTRO = "Failed to connect to MongoDB: ";
-
-    /**
-     * @param conf The {@link Configuration} defining how to construct the MongoClient.
-     * @return A {@link MongoClient}.  This client is lazy loaded and the same one
-     * is used throughout the lifecycle of the application.
-     * @throws IOException - if MongodForTestsFactory constructor has an io exception.
-     * @throws ConfigurationRuntimeException - Thrown if the configured server, port, user, or others are missing.
-     * @throws MongoException  if can't connect despite conf parameters are given
-     */
-    public static synchronized MongoClient getMongoClient(final Configuration conf)
-            throws ConfigurationRuntimeException, MongoException {
-        if (mongoClient == null) {
-            if(conf instanceof MongoDBRdfConfiguration && ((MongoDBRdfConfiguration) conf).getMongoClient() != null) {
-                mongoClient = ((MongoDBRdfConfiguration) conf).getMongoClient();
-            } else {
-                createMongoClientForServer(conf);
-            }
-        }
-        return mongoClient;
-    }
-
-    /**
-     * Silently closes the underlying Mongo client.
-     */
-    public static synchronized void closeMongoClient() {
-        IOUtils.closeQuietly(mongoClient);
-        mongoClient = null;
-    }
-
-    /**
-     * Create a MongoDB client object and assign it to this class's static mongoClient
-     * @param conf configuration containing connection parameters
-     * @throws ConfigurationRuntimeException - Thrown if the configured server, port, user, or others are missing.
-     * @throws MongoException  if can't connect despite conf parameters are given
-     */
-    private static void createMongoClientForServer(final Configuration conf)
-            throws ConfigurationRuntimeException, MongoException {
-        // Connect to a running Mongo server
-        final String host = requireNonNull(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE), MSG_INTRO+"host name is required");
-        final int port = requireNonNullInt(conf.get(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT), MSG_INTRO+"Port number is required.");
-        final ServerAddress server = new ServerAddress(host, port);
-        // check for authentication credentials
-        if (conf.get(MongoDBRdfConfiguration.MONGO_USER) != null) {
-            final String username = conf.get(MongoDBRdfConfiguration.MONGO_USER);
-            final String dbName = requireNonNull(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME),
-                    MSG_INTRO + MongoDBRdfConfiguration.MONGO_DB_NAME + " is null but required configuration if "
-                            + MongoDBRdfConfiguration.MONGO_USER + " is configured.");
-            final char[] pswd = requireNonNull(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD),
-                    MSG_INTRO + MongoDBRdfConfiguration.MONGO_USER_PASSWORD + " is null but required configuration if "
-                            + MongoDBRdfConfiguration.MONGO_USER + " is configured.").toCharArray();
-            final MongoCredential cred = MongoCredential.createCredential(username, dbName, pswd);
-            mongoClient = new MongoClient(server, Arrays.asList(cred));
-        } else {
-            // No user was configured:
-            mongoClient = new MongoClient(server);
-        }
-    }
-
-    /**
-     * Throw exception for un-configured required values.
-     *
-     * @param required  String to check
-     * @param message  throw configuration exception with this description
-     * @return unaltered required string
-     * @throws ConfigurationRuntimeException  if required is null
-     */
-    private static String requireNonNull(final String required, final String message) throws ConfigurationRuntimeException {
-        if (required == null) {
-            throw new ConfigurationRuntimeException(message);
-        }
-        return required;
-    }
-
-    /*
-     * Same as above, check that it is a integer and return the parsed integer.
-     */
-    private static int requireNonNullInt(final String required, final String message) throws ConfigurationRuntimeException {
-        if (required == null) {
-            throw new ConfigurationRuntimeException(message);
-        }
-        try {
-            return Integer.parseInt(required);
-        } catch (final NumberFormatException e) {
-            throw new ConfigurationRuntimeException(message);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
index f1115b1..d107d43 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
@@ -1,5 +1,4 @@
-package org.apache.rya.mongodb;
-/*
+/**
  * 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
@@ -17,8 +16,7 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
-
-import static com.google.common.base.Preconditions.checkNotNull;
+package org.apache.rya.mongodb;
 
 import java.io.IOException;
 import java.util.AbstractMap;
@@ -47,7 +45,6 @@ import org.openrdf.query.impl.MapBindingSet;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
-import com.mongodb.MongoClient;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 
@@ -57,48 +54,41 @@ import info.aduna.iteration.CloseableIteration;
  * Date: 7/17/12
  * Time: 9:28 AM
  */
-public class MongoDBQueryEngine implements RyaQueryEngine<MongoDBRdfConfiguration> {
-
-    private MongoDBRdfConfiguration configuration;
-    private final MongoClient mongoClient;
-    private final MongoDBStorageStrategy<RyaStatement> strategy;
-
-    public MongoDBQueryEngine(final MongoDBRdfConfiguration conf, final MongoClient mongoClient) {
-        this.mongoClient = checkNotNull(mongoClient);
-        strategy = new SimpleMongoDBStorageStrategy();
-    }
+public class MongoDBQueryEngine implements RyaQueryEngine<StatefulMongoDBRdfConfiguration> {
 
+    private StatefulMongoDBRdfConfiguration configuration;
+    private final MongoDBStorageStrategy<RyaStatement> strategy = new SimpleMongoDBStorageStrategy();
 
     @Override
-    public void setConf(final MongoDBRdfConfiguration conf) {
+    public void setConf(final StatefulMongoDBRdfConfiguration conf) {
         configuration = conf;
     }
 
     @Override
-    public MongoDBRdfConfiguration getConf() {
+    public StatefulMongoDBRdfConfiguration getConf() {
         return configuration;
     }
 
     @Override
     public CloseableIteration<RyaStatement, RyaDAOException> query(
-            final RyaStatement stmt, MongoDBRdfConfiguration conf)
+            final RyaStatement stmt, final StatefulMongoDBRdfConfiguration conf)
             throws RyaDAOException {
         Preconditions.checkNotNull(stmt);
         Preconditions.checkNotNull(conf);
-        
-        Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
-        Collection<Entry<RyaStatement, BindingSet>> collection = Collections.singleton(entry);
-        
+
+        final Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
+        final Collection<Entry<RyaStatement, BindingSet>> collection = Collections.singleton(entry);
+
         return new RyaStatementCursorIterator(queryWithBindingSet(collection, conf));
     }
 
     @Override
     public CloseableIteration<? extends Entry<RyaStatement, BindingSet>, RyaDAOException> queryWithBindingSet(
             final Collection<Entry<RyaStatement, BindingSet>> stmts,
-            MongoDBRdfConfiguration conf) throws RyaDAOException {
+            final StatefulMongoDBRdfConfiguration conf) throws RyaDAOException {
         Preconditions.checkNotNull(stmts);
         Preconditions.checkNotNull(conf);
-        
+
         final Multimap<RyaStatement, BindingSet> rangeMap = HashMultimap.create();
 
         //TODO: cannot span multiple tables here
@@ -121,7 +111,7 @@ public class MongoDBQueryEngine implements RyaQueryEngine<MongoDBRdfConfiguratio
     }
     @Override
     public CloseableIteration<RyaStatement, RyaDAOException> batchQuery(
-            final Collection<RyaStatement> stmts, MongoDBRdfConfiguration conf)
+            final Collection<RyaStatement> stmts, final StatefulMongoDBRdfConfiguration conf)
             throws RyaDAOException {
         final Map<RyaStatement, BindingSet> queries = new HashMap<>();
 
@@ -131,7 +121,7 @@ public class MongoDBQueryEngine implements RyaQueryEngine<MongoDBRdfConfiguratio
 
         return new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), conf));
     }
-    
+
     @Override
     public CloseableIterable<RyaStatement> query(final RyaQuery ryaQuery)
             throws RyaDAOException {
@@ -151,12 +141,12 @@ public class MongoDBQueryEngine implements RyaQueryEngine<MongoDBRdfConfiguratio
             queries.put(stmt, new MapBindingSet());
         }
 
-        Iterator<RyaStatement> iterator = new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), getConf()));
+        final Iterator<RyaStatement> iterator = new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), getConf()));
         return CloseableIterables.wrap((Iterable<RyaStatement>) () -> iterator);
     }
 
-    private MongoCollection<Document> getCollection(final MongoDBRdfConfiguration conf) {
-        final MongoDatabase db = mongoClient.getDatabase(conf.getMongoDBName());
+    private MongoCollection<Document> getCollection(final StatefulMongoDBRdfConfiguration conf) {
+        final MongoDatabase db = conf.getMongoClient().getDatabase(conf.getMongoDBName());
         return db.getCollection(conf.getTriplesCollectionName());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
index 418a155..4d06ea1 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
@@ -1,6 +1,4 @@
-package org.apache.rya.mongodb;
-
-/*
+/**
  * 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
@@ -18,72 +16,75 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.mongodb;
+
+import static java.util.Objects.requireNonNull;
 
-import java.util.List;
 import java.util.Properties;
 
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.mongodb.MongoClient;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
+/**
+ * A {@link RdfCloudTripleStoreConfiguration} that configures how Rya connects to a MongoDB Rya triple store.
+ */
 public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
-    public static final String MONGO_INSTANCE = "mongo.db.instance";
-    public static final String MONGO_INSTANCE_PORT = "mongo.db.port";
-    public static final String MONGO_GEO_MAXDISTANCE = "mongo.geo.maxdist";
+
+    // MongoDB Server connection values.
+    public static final String MONGO_HOSTNAME = "mongo.db.instance";
+    public static final String MONGO_PORT = "mongo.db.port";
+
+    // MongoDB Database values.
     public static final String MONGO_DB_NAME = "mongo.db.name";
-    public static final String MONGO_COLLECTION_PREFIX = "mongo.db.collectionprefix";
     public static final String MONGO_USER = "mongo.db.user";
     public static final String MONGO_USER_PASSWORD = "mongo.db.userpassword";
-    public static final String CONF_ADDITIONAL_INDEXERS = "ac.additional.indexers";
+
+    // Rya Instance values.
+    public static final String MONGO_COLLECTION_PREFIX = "mongo.db.collectionprefix";
+
+    // Rya Sail configuration values.
     public static final String USE_MOCK_MONGO = ".useMockInstance";
     public static final String CONF_FLUSH_EACH_UPDATE = "rya.mongodb.dao.flusheachupdate";
+    public static final String CONF_ADDITIONAL_INDEXERS = "ac.additional.indexers";
+    public static final String MONGO_GEO_MAXDISTANCE = "mongo.geo.maxdist";
 
-    private MongoClient mongoClient;
-
+    /**
+     * Constructs an empty instance of {@link MongoDBRdfConfiguration}.
+     */
     public MongoDBRdfConfiguration() {
         super();
     }
 
+    /**
+     * Constructs an instance of {@link MongoDBRdfConfiguration} pre-loaded with values.
+     *
+     * @param other - The values that will be cloned into the constructed object. (not null)
+     */
     public MongoDBRdfConfiguration(final Configuration other) {
-        super(other);
+        super( requireNonNull(other) );
     }
 
     /**
-     * Creates a MongoRdfConfiguration object from a Properties file. This
-     * method assumes that all values in the Properties file are Strings and
-     * that the Properties file uses the keys below.
+     * Reads a {@link Properties} object into a {@link MongoDBRdfConfiguration}.
+     * See {@link MongoDBRdfConfigurationBuilder#fromProperties(Properties)} for which keys
+     * are to be used within the properties object. This method will replace that object's keys
+     * with the configuration object's keys since they are not the same.
      *
-     * <br>
-     * <ul>
-     * <li>"mongo.auths" - String of Mongo authorizations.  Empty auths used by default.
-     * <li>"mongo.visibilities" - String of Mongo visibilities assigned to ingested triples.
-     * <li>"mongo.user" - Mongo user.  Empty by default.
-     * <li>"mongo.password" - Mongo password.  Empty by default.
-     * <li>"mongo.host" - Mongo host.  Default host is "localhost"
-     * <li>"mongo.port" - Mongo port.  Default port is "27017".
-     * <li>"mongo.db.name" - Name of MongoDB.  Default name is "rya_triples".
-     * <li>"mongo.collection.prefix" - Mongo collection prefix. Default is "rya_".
-     * <li>"mongo.rya.prefix" - Prefix for Mongo Rya instance.  Same as value of "mongo.collection.prefix".
-     * <li>"use.mock" - Use a Embedded Mongo instance as back-end for Rya instance. False by default.
-     * <li>"use.display.plan" - Display query plan during evaluation. Useful for debugging.  True by default.
-     * <li>"use.inference" - Use backward chaining inference during query.  False by default.
-     * </ul>
-     * <br>
-     *
-     * @param props
-     *            - Properties file containing Mongo specific configuration
-     *            parameters
-     * @return MongoRdfConfiguration with properties set
+     * @param props - The properties containing Mongo specific configuration parameters. (not null)
+     * @return A {@link } loaded with the values that were in {@code props}.
      */
     public static MongoDBRdfConfiguration fromProperties(final Properties props) {
+        requireNonNull(props);
         return MongoDBRdfConfigurationBuilder.fromProperties(props);
     }
 
-    public MongoDBRdfConfigurationBuilder getBuilder() {
+    /**
+     * @return A new instance of {@link MongoDBRdfConfigurationBuilder}.
+     */
+    public static MongoDBRdfConfigurationBuilder getBuilder() {
         return new MongoDBRdfConfigurationBuilder();
     }
 
@@ -92,184 +93,162 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
         return new MongoDBRdfConfiguration(this);
     }
 
-    public Authorizations getAuthorizations() {
-        final String[] auths = getAuths();
-        if (auths == null || auths.length == 0) {
-            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
-        }
-        return new Authorizations(auths);
-    }
-
     /**
-     * @return {@code true} if each statement added to the batch writer should
-     * be flushed and written right away to the datastore. {@code false} if the
-     * statements should be queued and written to the datastore when the queue
-     * is full or after enough time has passed without a write.<p>
-     * Defaults to {@code true} if nothing is specified.
+     * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
+     * or if it should connect to a MongoDB Server that is running somewhere.
+     *
+     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
      */
-    public boolean flushEachUpdate(){
-        return getBoolean(CONF_FLUSH_EACH_UPDATE, true);
+    public void setUseMock(final boolean useMock) {
+        this.setBoolean(USE_MOCK_MONGO, useMock);
     }
 
     /**
-     * Sets the {@link #CONF_FLUSH_EACH_UPDATE} property of the configuration.
-     * @param flush {@code true} if each statement added to the batch writer
-     * should be flushed and written right away to the datastore. {@code false}
-     * if the statements should be queued and written to the datastore when the
-     * queue is full or after enough time has passed without a write.
+     * Indicates whether the Rya client should spin up an embedded MongoDB instance and connect to that
+     * or if it should connect to a MongoDB Server that is running somewhere.
+     *
+     * @return {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
      */
-    public void setFlush(final boolean flush){
-        setBoolean(CONF_FLUSH_EACH_UPDATE, flush);
+    public boolean getUseMock() {
+        return getBoolean(USE_MOCK_MONGO, false);
     }
 
     /**
-     * @return name of Mongo Collection containing Rya triples
+     * @return The hostname of the MongoDB Server to connect to. (default: localhost)
      */
-    public String getTriplesCollectionName() {
-        return this.get(MONGO_COLLECTION_PREFIX, "rya") + "_triples";
+    public String getMongoHostname() {
+        return get(MONGO_HOSTNAME, "localhost");
     }
 
     /**
-     * @return name of Mongo Collection
+     * @param hostname - The hostname of the MongoDB Server to connect to.
      */
-    public String getCollectionName() {
-        return this.get(MONGO_COLLECTION_PREFIX, "rya");
+    public void setMongoHostname(final String hostname) {
+        requireNonNull(hostname);
+        set(MONGO_HOSTNAME, hostname);
     }
 
     /**
-     * Sets Mongo Collection name
-     * @param name - name of Mongo Collection to connect to
+     * @return The port of the MongoDB Server to connect to. (default: 27017)
      */
-    public void setCollectionName(final String name) {
-        Preconditions.checkNotNull(name);
-        this.set(MONGO_COLLECTION_PREFIX, name);
+    public String getMongoPort() {
+        return get(MONGO_PORT, AbstractMongoDBRdfConfigurationBuilder.DEFAULT_MONGO_PORT);
     }
 
     /**
-     * @return name of Mongo Host
+     * @param port - The port of the MongoDB Server to connect to.
      */
-    public String getMongoInstance() {
-        return this.get(MONGO_INSTANCE, "localhost");
+    public void setMongoPort(final String port) {
+        requireNonNull(port);
+        set(MONGO_PORT, port);
     }
 
     /**
-     * Sets name of Mongo Host
-     * @param name - name of Mongo Host to connect to
+     * @return The name of the MongoDB Database to connect to. (default: rya)
      */
-    public void setMongoInstance(final String name) {
-        Preconditions.checkNotNull(name);
-        this.set(MONGO_INSTANCE, name);
+    public String getMongoDBName() {
+        return get(MONGO_DB_NAME, "rya");
     }
 
     /**
-     * @return port that Mongo is running on
+     * @param database - The name of the MongoDb Database to connect to.
      */
-    public String getMongoPort() {
-        return this.get(MONGO_INSTANCE_PORT, AbstractMongoDBRdfConfigurationBuilder.DEFAULT_MONGO_PORT);
+    public void setMongoDBName(final String database) {
+        requireNonNull(database);
+        set(MONGO_DB_NAME, database);
     }
 
     /**
-     * Sets port that Mongo will run on
-     * @param name - Mongo port to connect to
+     * @param user - The user used to connect to the MongoDB Database that hosts the Rya Instance. (not null)
      */
-    public void setMongoPort(final String name) {
-        Preconditions.checkNotNull(name);
-        this.set(MONGO_INSTANCE_PORT, name);
+    public void setMongoUser(final String user) {
+        requireNonNull(user);
+        set(MONGO_USER, user);
     }
 
     /**
-     * @return name of MongoDB
+     * @return The user used to connect to the MongoDB Database that hosts the Rya Instance.
      */
-    public String getMongoDBName() {
-        return this.get(MONGO_DB_NAME, "rya");
+    public @Nullable String getMongoUser() {
+        return get(MONGO_USER);
     }
 
     /**
-     * Sets name of MongoDB
-     * @param name - name of MongoDB to connect to
+     * @param password - The password used to connect to the MongoDB Database that hosts the Rya Instance.
      */
-    public void setMongoDBName(final String name) {
-        Preconditions.checkNotNull(name);
-        this.set(MONGO_DB_NAME, name);
+    public void setMongoPassword(final String password) {
+        requireNonNull(password);
+        set(MONGO_USER_PASSWORD, password);
     }
 
     /**
-     * Tells Rya to use an embedded Mongo instance as its backing
-     * if set to true.  By default this is set to false.
-     * @param useMock
+     * @return The password used to connect to the MongoDB Database that hosts the Rya Instance.
      */
-    public void setUseMock(final boolean useMock) {
-        this.setBoolean(USE_MOCK_MONGO, useMock);
+    public @Nullable String getMongoPassword() {
+        return get(MONGO_USER_PASSWORD);
     }
 
     /**
-     * Get whether an embedded Mongo is being used as the backing
-     * for Rya.
-     * @return true if embedded Mongo is being used, and false otherwise
+     * @return The name of the Rya instance to connect to. (default: rya)
      */
-    public boolean getUseMock() {
-        return getBoolean(USE_MOCK_MONGO, false);
+    public String getRyaInstance() {
+        return get(MONGO_COLLECTION_PREFIX, "rya");
     }
 
     /**
-     * @return name of NameSpace Mongo Collection
+     * @param name - The name of the Rya instance to connect to.
      */
-    public String getNameSpacesCollectionName() {
-        return this.get(MONGO_COLLECTION_PREFIX, "rya") + "_ns";
+    public void setRyaInstance(final String name) {
+        requireNonNull(name);
+        set(MONGO_COLLECTION_PREFIX, name);
     }
 
     /**
-     * Sets name of Mongo User
-     * @param user - name of Mongo user to connect to
+     * @return The name of the MongoDB Collection that contains Rya statements. (default: rya_triples)
      */
-    public void setMongoUser(final String user) {
-        Preconditions.checkNotNull(user);
-        set(MONGO_USER, user);
+    public String getTriplesCollectionName() {
+        return getRyaInstance() + "_triples";
     }
 
     /**
-     * @return name of Mongo user
+     * @return The name of the MongoDB Collection that contains the Rya namespace. (default: rya_ns)
      */
-    public String getMongoUser() {
-        return get(MONGO_USER);
+    public String getNameSpacesCollectionName() {
+        return getRyaInstance() + "_ns";
     }
 
     /**
-     * Sets Mongo password
-     * @param password - password to connect to Mongo
+     * @return The authorizations that will be used when accessing data. (default: empty)
      */
-    public void setMongoPassword(final String password) {
-        Preconditions.checkNotNull(password);
-        set(MONGO_USER_PASSWORD, password);
+    public Authorizations getAuthorizations() {
+        final String[] auths = getAuths();
+        if (auths == null || auths.length == 0) {
+            return MongoDbRdfConstants.ALL_AUTHORIZATIONS;
+        }
+        return new Authorizations(auths);
     }
 
     /**
-     * @return Mongo password
+     * Indicates whether each statement added to the batch writer should be flushed and written
+     * right away to the datastore or not. If this is turned off, then the statements will be
+     * queued and written to the datastore when the queue is full or after enough time has
+     * passed without a write.
+     *
+     * @return {@code true} if flushing after each updated is enabled; otherwise {@code false}. (default: true)
      */
-    public String getMongoPassword() {
-        return get(MONGO_USER_PASSWORD);
-    }
-
-    public void setAdditionalIndexers(final Class<? extends MongoSecondaryIndex>... indexers) {
-        final List<String> strs = Lists.newArrayList();
-        for (final Class<?> ai : indexers){
-            strs.add(ai.getName());
-        }
-
-        setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
-    }
-
-    public List<MongoSecondaryIndex> getAdditionalIndexers() {
-        return getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
-    }
-
-    public void setMongoClient(final MongoClient client) {
-        Preconditions.checkNotNull(client);
-        this.mongoClient = client;
+    public boolean flushEachUpdate(){
+        return getBoolean(CONF_FLUSH_EACH_UPDATE, true);
     }
 
-    public MongoClient getMongoClient() {
-        return mongoClient;
+    /**
+     * Set whether each statement added to the batch writer should be flushed and written
+     * right away to the datastore or not. If this is turned off, then the statements will be
+     * queued and written to the datastore when the queue is full or after enough time has
+     * passed without a write.
+     *
+     * @param flush - {@code true} if flushing after each updated is enabled; otherwise {@code false}.
+     */
+    public void setFlush(final boolean flush){
+        setBoolean(CONF_FLUSH_EACH_UPDATE, flush);
     }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 d263b9c..01bbee7 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
@@ -1,6 +1,4 @@
-package org.apache.rya.mongodb;
-
-/*
+/**
  * 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
@@ -18,10 +16,11 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.mongodb;
 
+import static java.util.Objects.requireNonNull;
 
 import java.io.IOException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -57,13 +56,13 @@ import com.mongodb.MongoClient;
 /**
  * Default DAO for mongo backed RYA allowing for CRUD operations.
  */
-public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
+public final class MongoDBRyaDAO implements RyaDAO<StatefulMongoDBRdfConfiguration>{
     private static final Logger log = Logger.getLogger(MongoDBRyaDAO.class);
 
     private boolean isInitialized = false;
     private boolean flushEachUpdate = true;
-    private MongoDBRdfConfiguration conf;
-    private final MongoClient mongoClient;
+    private StatefulMongoDBRdfConfiguration conf;
+    private MongoClient mongoClient;
     private DB db;
     private DBCollection coll;
     private MongoDBQueryEngine queryEngine;
@@ -75,51 +74,24 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
 
     private MongoDbBatchWriter<DBObject> mongoDbBatchWriter;
 
-    /**
-     * Creates a new instance of {@link MongoDBRyaDAO}.
-     * @param conf the {@link MongoDBRdfConfiguration}.
-     * @throws RyaDAOException
-     */
-    public MongoDBRyaDAO(final MongoDBRdfConfiguration conf) throws RyaDAOException, NumberFormatException, UnknownHostException {
-        this(conf, MongoConnectorFactory.getMongoClient(conf));
-    }
-
-    /**
-     * Creates a new instance of {@link MongoDBRyaDAO}.
-     * @param conf the {@link MongoDBRdfConfiguration}.
-     * @param mongoClient the {@link MongoClient}.
-     * @throws RyaDAOException
-     */
-    public MongoDBRyaDAO(final MongoDBRdfConfiguration conf, final MongoClient mongoClient) throws RyaDAOException {
-        this.conf = conf;
-        this.mongoClient = mongoClient;
-        conf.setMongoClient(mongoClient);
-        auths = conf.getAuthorizations();
-        flushEachUpdate = conf.flushEachUpdate();
-        init();
-    }
-
     @Override
-    public void setConf(final MongoDBRdfConfiguration conf) {
-        this.conf = conf;
+    public void setConf(final StatefulMongoDBRdfConfiguration conf) {
+        this.conf = requireNonNull(conf);
+        mongoClient = this.conf.getMongoClient();
         auths = conf.getAuthorizations();
-    }
-
-    public MongoClient getMongoClient(){
-        return mongoClient;
+        flushEachUpdate = conf.flushEachUpdate();
     }
 
     public void setDB(final DB db) {
         this.db = db;
     }
 
-
     public void setDBCollection(final DBCollection coll) {
         this.coll = coll;
     }
 
     @Override
-    public MongoDBRdfConfiguration getConf() {
+    public StatefulMongoDBRdfConfiguration getConf() {
         return conf;
     }
 
@@ -131,13 +103,14 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
         secondaryIndexers = conf.getAdditionalIndexers();
         for(final MongoSecondaryIndex index: secondaryIndexers) {
             index.setConf(conf);
-            index.setClient(mongoClient);
+            index.setClient(conf.getMongoClient());
         }
 
         db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
         coll = db.getCollection(conf.getTriplesCollectionName());
         nameSpaceManager = new SimpleMongoDBNamespaceManager(db.getCollection(conf.getNameSpacesCollectionName()));
-        queryEngine = new MongoDBQueryEngine(conf, mongoClient);
+        queryEngine = new MongoDBQueryEngine();
+        queryEngine.setConf(conf);
         storageStrategy = new SimpleMongoDBStorageStrategy();
         storageStrategy.createIndices(coll);
         for(final MongoSecondaryIndex index: secondaryIndexers) {
@@ -145,7 +118,7 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
         }
 
         final MongoDbBatchWriterConfig mongoDbBatchWriterConfig = MongoDbBatchWriterUtils.getMongoDbBatchWriterConfig(conf);
-        mongoDbBatchWriter = new MongoDbBatchWriter<DBObject>(new DbCollectionType(coll), mongoDbBatchWriterConfig);
+        mongoDbBatchWriter = new MongoDbBatchWriter<>(new DbCollectionType(coll), mongoDbBatchWriterConfig);
         try {
             mongoDbBatchWriter.start();
         } catch (final MongoDbBatchWriterException e) {
@@ -217,7 +190,7 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
 
     @Override
     public void add(final Iterator<RyaStatement> statementIter) throws RyaDAOException {
-        final List<DBObject> dbInserts = new ArrayList<DBObject>();
+        final List<DBObject> dbInserts = new ArrayList<>();
         while (statementIter.hasNext()){
             final RyaStatement ryaStatement = statementIter.next();
             final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, ryaStatement.getColumnVisibility());
@@ -247,7 +220,7 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
     }
 
     @Override
-    public void delete(final RyaStatement statement, final MongoDBRdfConfiguration conf)
+    public void delete(final RyaStatement statement, final StatefulMongoDBRdfConfiguration conf)
             throws RyaDAOException {
         final boolean canDelete = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
         if (canDelete) {
@@ -266,14 +239,14 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
     }
 
     @Override
-    public void dropGraph(final MongoDBRdfConfiguration conf, final RyaURI... graphs)
+    public void dropGraph(final StatefulMongoDBRdfConfiguration conf, final RyaURI... graphs)
             throws RyaDAOException {
 
     }
 
     @Override
     public void delete(final Iterator<RyaStatement> statements,
-            final MongoDBRdfConfiguration conf) throws RyaDAOException {
+            final StatefulMongoDBRdfConfiguration conf) throws RyaDAOException {
         while (statements.hasNext()){
             final RyaStatement ryaStatement = statements.next();
             final boolean canDelete = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, ryaStatement.getColumnVisibility());
@@ -298,12 +271,12 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
     }
 
     @Override
-    public RyaQueryEngine<MongoDBRdfConfiguration> getQueryEngine() {
+    public RyaQueryEngine<StatefulMongoDBRdfConfiguration> getQueryEngine() {
         return queryEngine;
     }
 
     @Override
-    public RyaNamespaceManager<MongoDBRdfConfiguration> getNamespaceManager() {
+    public RyaNamespaceManager<StatefulMongoDBRdfConfiguration> getNamespaceManager() {
         return nameSpaceManager;
     }
 
@@ -336,4 +309,4 @@ public final class MongoDBRyaDAO implements RyaDAO<MongoDBRdfConfiguration>{
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
new file mode 100644
index 0000000..1263166
--- /dev/null
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
@@ -0,0 +1,79 @@
+/**
+ * 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 static java.util.Objects.requireNonNull;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.mongodb.MongoClient;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
+
+/**
+ * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
+ * {@link Configuration} objects.
+ * </p>
+ * HACK:
+ * This class is part of a hack to get around how Rya uses reflection to initialize indexers, optimizers, etc.
+ * Those classes have empty constructors, so they are not able to receive Mongo specific components at construction
+ * time. However, they all receive a {@link Configuration} prior to initialization. If an object of this class
+ * is that configuration object, then shared objects may be passed into the constructed components.
+ */
+@DefaultAnnotation(NonNull.class)
+public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
+
+    private final MongoClient mongoClient;
+    private final List<MongoSecondaryIndex> indexers;
+    // TODO optimizers? They're causing problems because they aren't getting this configuration object
+    //      like the indexers do.
+
+    /**
+     * Constructs an instance of {@link StatefulMongoDBRdfConfiguration} pre-loaded with values.
+     *
+     * @param other - The values that will be cloned into the constructed object. (not null)
+     * @param mongoClient - The {@link MongoClient} that Rya will use. (not null)
+     * @param indexers - The {@link MongoSecondaryIndex}s that Rya will use. (not null)
+     */
+    public StatefulMongoDBRdfConfiguration(
+            final Configuration other,
+            final MongoClient mongoClient,
+            final List<MongoSecondaryIndex> indexers) {
+        super(other);
+        this.mongoClient = requireNonNull(mongoClient);
+        this.indexers = requireNonNull(indexers);
+    }
+
+    /**
+     * @return The {@link MongoClient} that Rya will use.
+     */
+    public MongoClient getMongoClient() {
+        return mongoClient;
+    }
+
+    /**
+     * @return The {@link MongoSecondaryIndex}s that Rya will use.
+     */
+    public List<MongoSecondaryIndex> getAdditionalIndexers() {
+        return indexers;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/MongoDBNamespaceManager.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/MongoDBNamespaceManager.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/MongoDBNamespaceManager.java
index 1d86ce4..24d47ee 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/MongoDBNamespaceManager.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/MongoDBNamespaceManager.java
@@ -1,6 +1,4 @@
-package org.apache.rya.mongodb.dao;
-
-/*
+/**
  * 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
@@ -8,9 +6,9 @@ package org.apache.rya.mongodb.dao;
  * 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,18 +16,14 @@ package org.apache.rya.mongodb.dao;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.mongodb.dao;
 
-
-import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.persist.RyaNamespaceManager;
-import org.apache.rya.api.persist.query.RyaQuery;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
 import com.mongodb.DBCollection;
-import com.mongodb.DBObject;
 
-public interface MongoDBNamespaceManager extends RyaNamespaceManager<MongoDBRdfConfiguration>{
+public interface MongoDBNamespaceManager extends RyaNamespaceManager<StatefulMongoDBRdfConfiguration>{
 
 	public void createIndices(DBCollection coll);
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
index 6956f49..88fa2b5 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
@@ -25,7 +25,7 @@ import java.util.Map;
 
 import org.apache.commons.codec.binary.Hex;
 import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.model.Namespace;
 
 import com.mongodb.BasicDBObject;
@@ -109,7 +109,7 @@ public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
 	private static final String ID = "_id";
 	private static final String PREFIX = "prefix";
 	private static final String NAMESPACE = "namespace";
-	private MongoDBRdfConfiguration conf;
+	private StatefulMongoDBRdfConfiguration conf;
 	private final DBCollection nsColl;
 
 
@@ -123,15 +123,13 @@ public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
 		coll.createIndex(NAMESPACE);
 	}
 
-
 	@Override
-	public void setConf(final MongoDBRdfConfiguration paramC) {
-		this.conf = paramC;
+	public void setConf(final StatefulMongoDBRdfConfiguration conf) {
+		this.conf = conf;
 	}
 
 	@Override
-	public MongoDBRdfConfiguration getConf() {
-		// TODO Auto-generated method stub
+	public StatefulMongoDBRdfConfiguration getConf() {
 		return conf;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
index e068405..a481ccf 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
@@ -19,11 +19,15 @@
 package org.apache.rya.mongodb;
 
 import java.io.IOException;
+import java.net.UnknownHostException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.mongodb.MongoClient;
+import com.mongodb.MongoException;
+
+import de.flapdoodle.embed.mongo.config.IMongodConfig;
 
 /**
  * To be used for tests. Creates a singleton {@link MongoClient} to be used
@@ -31,8 +35,27 @@ import com.mongodb.MongoClient;
  * embedded mongo factory ends up orphaning processes, consuming resources.
  */
 public class EmbeddedMongoSingleton {
-    public static MongoClient getInstance() {
-        return InstanceHolder.SINGLETON.instance;
+
+    public static MongoClient getNewMongoClient() throws UnknownHostException, MongoException {
+    	final MongoClient client = InstanceHolder.SINGLETON.factory.newMongoClient();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                try {
+                    client.close();
+                } catch (final Throwable t) {
+                    // logging frameworks will likely be shut down
+                    t.printStackTrace(System.err);
+                }
+            }
+        });
+
+        return client;
+    }
+
+    public static IMongodConfig getMongodConfig() {
+        return InstanceHolder.SINGLETON.mongodConfig;
     }
 
     private EmbeddedMongoSingleton() {
@@ -44,32 +67,14 @@ public class EmbeddedMongoSingleton {
         SINGLETON;
 
         private final Logger log;
-        private MongoClient instance;
+        private IMongodConfig mongodConfig;
+        private EmbeddedMongoFactory factory;
 
         InstanceHolder() {
             log = LoggerFactory.getLogger(EmbeddedMongoSingleton.class);
-            instance = null;
             try {
-                instance = EmbeddedMongoFactory.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);
-                        }
-                    }
-                });
-
+            	factory = EmbeddedMongoFactory.newFactory();
+                mongodConfig = factory.getMongoServerDetails();
             } catch (final IOException e) {
                 log.error("Unexpected error while starting mongo client", e);
             } catch (final Throwable e) {
@@ -79,4 +84,4 @@ public class EmbeddedMongoSingleton {
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 a8f2d88..f7235a6 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
@@ -1,5 +1,25 @@
+/*
+* * 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 static org.junit.Assert.assertEquals;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -27,38 +47,18 @@ import org.apache.rya.api.RdfCloudTripleStoreUtils;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
 import org.apache.rya.api.domain.RyaURI;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.impl.URIImpl;
 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;
-
+/**
+ * Integration tests the methods of {@link MongoDBQueryEngine}.
+ */
 public class MongoDBQueryEngineTest extends MongoTestBase {
-    private MongoClient client;
-    private MongoDBRyaDAO dao;
-
-    private MongoDBQueryEngine engine;
-
-    private static final String DB_NAME = "testInstance";
-
-    @Before
-    public void setUp() throws Exception {
-        client = super.getMongoClient();
-        conf.setAuths("A", "B", "C");
-
-        engine = new MongoDBQueryEngine(conf, client);
-
-        // Add Data
-        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"));
-    }
 
     private RyaStatement getStatement(final String s, final String p, final String o) {
         final RyaStatementBuilder builder = new RyaStatementBuilder();
@@ -85,59 +85,101 @@ public class MongoDBQueryEngineTest extends MongoTestBase {
 
     @Test
     public void statementQuery() throws Exception {
-        final RyaStatement s = getStatement("u:a", null, null);
-        Assert.assertEquals(2, size(engine.query(s, conf)));
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
+
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
+
+            final RyaStatement s = getStatement("u:a", null, null);
+            assertEquals(2, size(engine.query(s, conf)));
+        } finally {
+            dao.destroy();
+        }
     }
 
     @SuppressWarnings("unchecked")
     @Test
     public void batchbindingSetsQuery() throws Exception {
-        final RyaStatement s1 = getStatement(null, null, "u:b");
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
 
-        final MapBindingSet bs1 = new MapBindingSet();
-        bs1.addBinding("foo", new URIImpl("u:x"));
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
 
-        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, conf)));
+            // Run the test.
+            final RyaStatement s1 = getStatement(null, null, "u:b");
 
+            final MapBindingSet bs1 = new MapBindingSet();
+            bs1.addBinding("foo", new URIImpl("u:x"));
 
-        final MapBindingSet bs2 = new MapBindingSet();
-        bs2.addBinding("foo", new URIImpl("u:y"));
+            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs1);
+            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
+            assertEquals(1, size(engine.queryWithBindingSet(stmts1, conf)));
 
-        final RyaStatement s2 = getStatement(null, null, "u:c");
 
-        final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(s2, bs2);
+            final MapBindingSet bs2 = new MapBindingSet();
+            bs2.addBinding("foo", new URIImpl("u:y"));
 
-        final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
-        Assert.assertEquals(2, size(engine.queryWithBindingSet(stmts2, conf)));
+            final RyaStatement s2 = getStatement(null, null, "u:c");
 
+            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs2);
 
-        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>> stmts2 = Lists.newArrayList(e1, e2);
+            assertEquals(2, size(engine.queryWithBindingSet(stmts2, conf)));
+
+
+            final Map.Entry<RyaStatement, BindingSet> e3 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs1);
+            final Map.Entry<RyaStatement, BindingSet> e4 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs2);
+
+            final Collection<Entry<RyaStatement, BindingSet>> stmts3 = Lists.newArrayList(e1, e2, e3, e4);
+            assertEquals(4, size(engine.queryWithBindingSet(stmts3, conf)));
+        } finally {
+            dao.destroy();
+        }
+    }
 
-        final Collection<Entry<RyaStatement, BindingSet>> stmts3 = Lists.newArrayList(e1, e2, e3, e4);
-        Assert.assertEquals(4, size(engine.queryWithBindingSet(stmts3, conf)));
-}
     @SuppressWarnings("unchecked")
     @Test
     public void bindingSetsQuery() throws Exception {
-        final RyaStatement s = getStatement("u:a", null, null);
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
 
-        final MapBindingSet bs1 = new MapBindingSet();
-        bs1.addBinding("foo", new URIImpl("u:x"));
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
 
-        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, conf)));
+            // Run the test.
+            final RyaStatement s = getStatement("u:a", null, null);
 
+            final MapBindingSet bs1 = new MapBindingSet();
+            bs1.addBinding("foo", new URIImpl("u:x"));
 
-        final MapBindingSet bs2 = new MapBindingSet();
-        bs2.addBinding("foo", new URIImpl("u:y"));
+            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs1);
+            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
+            assertEquals(2, size(engine.queryWithBindingSet(stmts1, conf)));
 
-        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, conf)));
-}
-}
+            final MapBindingSet bs2 = new MapBindingSet();
+            bs2.addBinding("foo", new URIImpl("u:y"));
+
+            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs2);
+
+            final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
+            assertEquals(4, size(engine.queryWithBindingSet(stmts2, conf)));
+        } finally {
+            dao.destroy();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
index c047fde..2d28bb5 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
@@ -33,16 +33,16 @@ public class MongoDBRdfConfigurationTest {
 
     @Test
     public void testBuilder() {
-        String prefix = "prefix_";
-        String auth = "U,V,W";
-        String visibility = "U,W";
-        String user = "user";
-        String password = "password";
-        boolean useMock = true;
-        boolean useInference = true;
-        boolean displayPlan = false;
+        final String prefix = "prefix_";
+        final String auth = "U,V,W";
+        final String visibility = "U,W";
+        final String user = "user";
+        final String password = "password";
+        final boolean useMock = true;
+        final boolean useInference = true;
+        final boolean displayPlan = false;
 
-        MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration().getBuilder()
+        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration().getBuilder()
                 .setVisibilities(visibility)
                 .setUseInference(useInference)
                 .setDisplayQueryPlan(displayPlan)
@@ -60,11 +60,11 @@ public class MongoDBRdfConfigurationTest {
         assertEquals(conf.getCv(), visibility);
         assertEquals(conf.isInfer(), useInference);
         assertEquals(conf.isDisplayQueryPlan(), displayPlan);
-        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getMongoHostname(), "host");
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.getRyaInstance(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
 
@@ -72,32 +72,31 @@ public class MongoDBRdfConfigurationTest {
 
     @Test
     public void testBuilderFromProperties() throws FileNotFoundException, IOException {
-        String prefix = "prefix_";
-        String auth = "U";
-        String visibility = "U";
-        String user = "user";
-        String password = "password";
-        boolean useMock = true;
-        boolean useInference = true;
-        boolean displayPlan = false;
+        final String prefix = "prefix_";
+        final String auth = "U";
+        final String visibility = "U";
+        final String user = "user";
+        final String password = "password";
+        final boolean useMock = true;
+        final boolean useInference = true;
+        final boolean displayPlan = false;
 
-        Properties props = new Properties();
+        final Properties props = new Properties();
         props.load(new FileInputStream("src/test/resources/rya.properties"));
 
-        MongoDBRdfConfiguration conf = MongoDBRdfConfiguration.fromProperties(props);
+        final MongoDBRdfConfiguration conf = MongoDBRdfConfiguration.fromProperties(props);
 
         assertEquals(conf.getTablePrefix(), prefix);
         assertTrue(Arrays.equals(conf.getAuths(), new String[] { auth }));
         assertEquals(conf.getCv(), visibility);
         assertEquals(conf.isInfer(), useInference);
         assertEquals(conf.isDisplayQueryPlan(), displayPlan);
-        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getMongoHostname(), "host");
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.getRyaInstance(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
     }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
index 68bbc27..afd7308 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.mongodb;
 
+import static org.junit.Assert.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -33,20 +35,16 @@ import org.apache.rya.mongodb.batch.collection.MongoCollectionType;
 import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
 import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
 import org.bson.Document;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
 import com.mongodb.DBObject;
-import com.mongodb.MongoClient;
 
 /**
  * Integration tests for the {@link MongoDbBatchWriter}.
  */
 public class MongoDBRyaBatchWriterIT extends MongoTestBase {
-    private MongoDBRyaDAO dao;
 
     private static void setupLogging() {
         BasicConfigurator.configure();
@@ -57,14 +55,11 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
         setupLogging();
     }
 
-    @Before
-    public void setUp() throws Exception {
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.setBoolean("rya.mongodb.dao.flusheachupdate", false);
         conf.setInt("rya.mongodb.dao.batchwriter.size", 50_000);
         conf.setLong("rya.mongodb.dao.batchwriter.flushtime", 100L);
-
-        final MongoClient client = super.getMongoClient();
-        dao = new MongoDBRyaDAO(conf, client);
     }
 
     @Test
@@ -81,11 +76,18 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
         statements.add(statement(1));
         statements.add(statement(6));
 
-        dao.add(statements.iterator());
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
 
-        dao.flush();
+            dao.add(statements.iterator());
+            dao.flush();
 
-        Assert.assertEquals(6, getRyaCollection().count());
+            assertEquals(6, getRyaCollection().count());
+        } finally {
+            dao.destroy();
+        }
     }
 
     @Test
@@ -104,7 +106,7 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
 
         final DbCollectionType collectionType = new DbCollectionType(getRyaDbCollection());
         final MongoDbBatchWriterConfig mongoDbBatchWriterConfig = MongoDbBatchWriterUtils.getMongoDbBatchWriterConfig(conf);
-        final MongoDbBatchWriter<DBObject> mongoDbBatchWriter = new MongoDbBatchWriter<DBObject>(collectionType, mongoDbBatchWriterConfig);
+        final MongoDbBatchWriter<DBObject> mongoDbBatchWriter = new MongoDbBatchWriter<>(collectionType, mongoDbBatchWriterConfig);
 
         mongoDbBatchWriter.start();
         mongoDbBatchWriter.addObjectsToQueue(objects);
@@ -114,7 +116,7 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
         mongoDbBatchWriter.flush();
         Thread.sleep(1_000);
         mongoDbBatchWriter.shutdown();
-        Assert.assertEquals(4, getRyaDbCollection().count());
+        assertEquals(4, getRyaDbCollection().count());
     }
 
     @Test
@@ -133,7 +135,7 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
 
         final MongoCollectionType mongoCollectionType = new MongoCollectionType(getRyaCollection());
         final MongoDbBatchWriterConfig mongoDbBatchWriterConfig = MongoDbBatchWriterUtils.getMongoDbBatchWriterConfig(conf);
-        final MongoDbBatchWriter<Document> mongoDbBatchWriter = new MongoDbBatchWriter<Document>(mongoCollectionType, mongoDbBatchWriterConfig);
+        final MongoDbBatchWriter<Document> mongoDbBatchWriter = new MongoDbBatchWriter<>(mongoCollectionType, mongoDbBatchWriterConfig);
 
         mongoDbBatchWriter.start();
         mongoDbBatchWriter.addObjectsToQueue(documents);
@@ -143,7 +145,7 @@ public class MongoDBRyaBatchWriterIT extends MongoTestBase {
         mongoDbBatchWriter.flush();
         Thread.sleep(1_000);
         mongoDbBatchWriter.shutdown();
-        Assert.assertEquals(4, getRyaCollection().count());
+        assertEquals(4, getRyaCollection().count());
     }
 
     private static Document toDocument(final DBObject dbObject) {


[13/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index c7e73ed..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorTest.java
+++ /dev/null
@@ -1,2053 +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.smarturi.duplication;
-
-import static java.util.Objects.requireNonNull;
-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.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang.builder.ReflectionToStringBuilder;
-import org.apache.rya.api.domain.RyaSchema;
-import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaTypeUtils;
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.api.resolver.RdfToRyaConversions;
-import org.apache.rya.indexing.entity.model.Entity;
-import org.apache.rya.indexing.entity.model.Entity.Builder;
-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.EntityStorage.EntityStorageException;
-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.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;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-
-/**
- * Tests the methods of {@link DuplicateDataDetector}.
- */
-public class DuplicateDataDetectorTest extends MongoTestBase {
-    private static final String RYA_INSTANCE_NAME = "testInstance";
-
-    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");
-
-    private static final Date NOW = new Date();
-
-    /**
-     * 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(NOW.getTime()).minusYears(40))))
-            .setProperty(PERSON_TYPE_URI, new Property(HAS_EXPIRATION_DATE, dateRyaType(NOW)))
-            .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 createErrorMessage(final Object originalValue, final Object testValue, final boolean expected, final boolean actual, final Tolerance tolerance) {
-        final String message = "The test value \"" + testValue + "\" was " + (expected ? "" : "NOT ") + "supposed to be almost equals to \"" + originalValue + "\" when the tolerance was \"" + tolerance.toString() + "\" but " + (actual ? "was" : "wasn't") + ".";
-        return message;
-    }
-
-    @Test
-    public void testCompareEntities() throws SmartUriException, ConfigurationException {
-        final Entity entity1 = createBobEntity();
-        final Entity entity2 = new Builder(entity1).build();
-
-        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
-        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
-        assertTrue(areDuplicates);
-    }
-
-    @Test
-    public void testCompareSmartUris() throws SmartUriException, ConfigurationException {
-        final Entity entity1 = createBobEntity();
-        final Entity entity2 = new Builder(entity1).build();
-
-        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
-        final boolean areDuplicates = duplicateDataDetector.compareSmartUris(entity1.getSmartUri(), entity2.getSmartUri());
-        assertTrue(areDuplicates);
-    }
-
-    @Test
-    public void testEntitySubjectsDifferent() throws SmartUriException, ConfigurationException {
-        final Entity entity1 = createBobEntity();
-        final Builder builder = new Builder(entity1);
-        builder.setSubject(createRyaUri("Susan"));
-        final Entity entity2 = builder.build();
-
-        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
-        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
-        assertTrue(areDuplicates);
-    }
-
-    @Test
-    public void testEntityMissingType() throws SmartUriException, ConfigurationException {
-        final Entity entity1 = createBobEntity();
-        final Builder builder = new Builder(entity1);
-        builder.setExplicitType(new RyaURI("urn:example/manager"));
-        final Entity entity2 = builder.build();
-
-        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
-        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
-        assertFalse(areDuplicates);
-    }
-
-    @Test
-    public void testEntityMissingProperty() throws SmartUriException, ConfigurationException {
-        final Entity entity1 = createBobEntity();
-        final Builder builder = new Builder(entity1);
-        builder.unsetProperty(PERSON_TYPE_URI, HAS_SSN);
-        final Entity entity2 = builder.build();
-
-        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
-        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
-        assertFalse(areDuplicates);
-    }
-
-    @Test
-    public void testReadConfigFile() throws SmartUriException, ConfigurationException {
-        final DuplicateDataConfig duplicateDataConfig = new DuplicateDataConfig();
-
-        assertNotNull(duplicateDataConfig.getBooleanTolerance());
-        assertNotNull(duplicateDataConfig.getByteTolerance());
-        assertNotNull(duplicateDataConfig.getDateTolerance());
-        assertNotNull(duplicateDataConfig.getDoubleTolerance());
-        assertNotNull(duplicateDataConfig.getFloatTolerance());
-        assertNotNull(duplicateDataConfig.getIntegerTolerance());
-        assertNotNull(duplicateDataConfig.getLongTolerance());
-        assertNotNull(duplicateDataConfig.getShortTolerance());
-        assertNotNull(duplicateDataConfig.getStringTolerance());
-        assertNotNull(duplicateDataConfig.getUriTolerance());
-
-        assertNotNull(duplicateDataConfig.getEquivalentTermsMap());
-        assertNotNull(duplicateDataConfig.isDetectionEnabled());
-    }
-
-    @Test
-    public void testBooleanProperty() throws SmartUriException {
-        System.out.println("Boolean Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(false, tolerance, false));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(false, tolerance, true));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(false, tolerance, true));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(false, tolerance, false));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(false, tolerance, true));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(false, tolerance, true));
-        builder.add(new TestInput(true, tolerance, true)); // Equals value
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_GLASSES);
-    }
-
-    @Test
-    public void testByteProperty() throws SmartUriException {
-        System.out.println("Byte Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((byte) 0xff, tolerance, false));
-        builder.add(new TestInput((byte) 0x00, tolerance, false));
-        builder.add(new TestInput((byte) 0x01, tolerance, false));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, false));
-        builder.add(new TestInput((byte) 0x04, tolerance, false));
-        builder.add(new TestInput((byte) 0x05, tolerance, false));
-        builder.add(new TestInput((byte) 0x10, tolerance, false));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((byte) 0xff, tolerance, false));
-        builder.add(new TestInput((byte) 0x00, tolerance, false));
-        builder.add(new TestInput((byte) 0x01, tolerance, true));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, true));
-        builder.add(new TestInput((byte) 0x04, tolerance, false));
-        builder.add(new TestInput((byte) 0x05, tolerance, false));
-        builder.add(new TestInput((byte) 0x10, tolerance, false));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((byte) 0xff, tolerance, false));
-        builder.add(new TestInput((byte) 0x00, tolerance, true));
-        builder.add(new TestInput((byte) 0x01, tolerance, true));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, true));
-        builder.add(new TestInput((byte) 0x04, tolerance, true));
-        builder.add(new TestInput((byte) 0x05, tolerance, false));
-        builder.add(new TestInput((byte) 0x10, tolerance, false));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((byte) 0xff, tolerance, false));
-        builder.add(new TestInput((byte) 0x00, tolerance, false));
-        builder.add(new TestInput((byte) 0x01, tolerance, false));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, false));
-        builder.add(new TestInput((byte) 0x04, tolerance, false));
-        builder.add(new TestInput((byte) 0x05, tolerance, false));
-        builder.add(new TestInput((byte) 0x10, tolerance, false));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
-        // Tolerance 50.0%
-        tolerance = new Tolerance(0.50, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((byte) 0xff, tolerance, false));
-        builder.add(new TestInput((byte) 0x00, tolerance, false));
-        builder.add(new TestInput((byte) 0x01, tolerance, true));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, true));
-        builder.add(new TestInput((byte) 0x04, tolerance, false));
-        builder.add(new TestInput((byte) 0x05, tolerance, false));
-        builder.add(new TestInput((byte) 0x10, tolerance, false));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, true));
-        builder.add(new TestInput((byte) 0xff, tolerance, true));
-        builder.add(new TestInput((byte) 0x00, tolerance, true));
-        builder.add(new TestInput((byte) 0x01, tolerance, true));
-        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
-        builder.add(new TestInput((byte) 0x03, tolerance, true));
-        builder.add(new TestInput((byte) 0x04, tolerance, true));
-        builder.add(new TestInput((byte) 0x05, tolerance, true));
-        builder.add(new TestInput((byte) 0x10, tolerance, true));
-        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_NUMBER_OF_CHILDREN);
-    }
-
-    @Test
-    public void testDateProperty() throws SmartUriException {
-        System.out.println("Date Property Test");
-        final long ONE_YEAR_IN_MILLIS = 1000L * 60L * 60L * 24L * 365L;
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new Date(0L), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, false));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new Date(0L), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new Date(0L), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new Date(0L), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, false));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new Date(0L), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.985)), tolerance, false));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.989)), tolerance, false));
-        // It's tricky near the exact threshold since it might create a fraction
-        // of a time which is rounded. Check if it's fraction and test it with
-        // the floor and ceiling values if it is. Otherwise, use the exact value
-        // if it is not a fraction.
-        final double lowerThresholdFloor = Math.floor(NOW.getTime() * 0.99);
-        final double lowerThresholdCeiling = Math.ceil(NOW.getTime() * 0.99);
-        // If the floor equals the ceiling then it's not a fraction.
-        if (lowerThresholdFloor != lowerThresholdCeiling) {
-           builder.add(new TestInput(new Date((long) lowerThresholdFloor), tolerance, false));
-        }
-        builder.add(new TestInput(new Date((long) lowerThresholdCeiling), tolerance, true));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.991)), tolerance, true));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.995)), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, true));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.005)), tolerance, true));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.009)), tolerance, true));
-        // It's tricky near the exact threshold since it might create a fraction
-        // of a time which is rounded. Check if it's fraction and test it with
-        // the floor and ceiling values if it is. Otherwise, use the exact value
-        // if it is not a fraction.
-        final double upperThresholdFloor = Math.floor(NOW.getTime() * 1.01);
-        final double upperThresholdCeiling = Math.ceil(NOW.getTime() * 1.01);
-        builder.add(new TestInput(new Date((long) upperThresholdFloor), tolerance, true));
-        // If the floor equals the ceiling then it's not a fraction.
-        if (upperThresholdFloor != upperThresholdCeiling) {
-           builder.add(new TestInput(new Date((long) upperThresholdCeiling), tolerance, false));
-        }
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.011)), tolerance, false));
-        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.015)), tolerance, false));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new Date(0L), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
-        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
-        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, true));
-        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, true));
-        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_EXPIRATION_DATE);
-    }
-
-    @Test
-    public void testDateTimeProperty() throws SmartUriException {
-        System.out.println("DateTime Property Test");
-        final DateTime dob = new DateTime(NOW).minusYears(40);
-        final long ONE_YEAR_IN_MILLIS = 1000L * 60L * 60L * 24L * 365L;
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, false));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
-        builder.add(new TestInput(dob.minus(10000), tolerance, false));
-        builder.add(new TestInput(dob.minus(2000), tolerance, false));
-        builder.add(new TestInput(dob.minus(1001), tolerance, false));
-        builder.add(new TestInput(dob.minus(1000), tolerance, false));
-        builder.add(new TestInput(dob.minus(999), tolerance, false));
-        builder.add(new TestInput(dob.minus(3), tolerance, false));
-        builder.add(new TestInput(dob.minus(2), tolerance, false));
-        builder.add(new TestInput(dob.minus(1), tolerance, false));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, false));
-        builder.add(new TestInput(dob.plus(2), tolerance, false));
-        builder.add(new TestInput(dob.plus(3), tolerance, false));
-        builder.add(new TestInput(dob.plus(999), tolerance, false));
-        builder.add(new TestInput(dob.plus(1000), tolerance, false));
-        builder.add(new TestInput(dob.plus(1001), tolerance, false));
-        builder.add(new TestInput(dob.plus(2000), tolerance, false));
-        builder.add(new TestInput(dob.plus(10000), tolerance, false));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, false));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
-        builder.add(new TestInput(dob.minus(10000), tolerance, false));
-        builder.add(new TestInput(dob.minus(2000), tolerance, false));
-        builder.add(new TestInput(dob.minus(1001), tolerance, false));
-        builder.add(new TestInput(dob.minus(1000), tolerance, false));
-        builder.add(new TestInput(dob.minus(999), tolerance, false));
-        builder.add(new TestInput(dob.minus(3), tolerance, false));
-        builder.add(new TestInput(dob.minus(2), tolerance, false));
-        builder.add(new TestInput(dob.minus(1), tolerance, true));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, true));
-        builder.add(new TestInput(dob.plus(2), tolerance, false));
-        builder.add(new TestInput(dob.plus(3), tolerance, false));
-        builder.add(new TestInput(dob.plus(999), tolerance, false));
-        builder.add(new TestInput(dob.plus(1000), tolerance, false));
-        builder.add(new TestInput(dob.plus(1001), tolerance, false));
-        builder.add(new TestInput(dob.plus(2000), tolerance, false));
-        builder.add(new TestInput(dob.plus(10000), tolerance, false));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, false));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
-        builder.add(new TestInput(dob.minus(10000), tolerance, false));
-        builder.add(new TestInput(dob.minus(2000), tolerance, false));
-        builder.add(new TestInput(dob.minus(1001), tolerance, false));
-        builder.add(new TestInput(dob.minus(1000), tolerance, false));
-        builder.add(new TestInput(dob.minus(999), tolerance, false));
-        builder.add(new TestInput(dob.minus(3), tolerance, false));
-        builder.add(new TestInput(dob.minus(2), tolerance, true));
-        builder.add(new TestInput(dob.minus(1), tolerance, true));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, true));
-        builder.add(new TestInput(dob.plus(2), tolerance, true));
-        builder.add(new TestInput(dob.plus(3), tolerance, false));
-        builder.add(new TestInput(dob.plus(999), tolerance, false));
-        builder.add(new TestInput(dob.plus(1000), tolerance, false));
-        builder.add(new TestInput(dob.plus(1001), tolerance, false));
-        builder.add(new TestInput(dob.plus(2000), tolerance, false));
-        builder.add(new TestInput(dob.plus(10000), tolerance, false));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, false));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
-        builder.add(new TestInput(dob.minus(10000), tolerance, false));
-        builder.add(new TestInput(dob.minus(2000), tolerance, false));
-        builder.add(new TestInput(dob.minus(1001), tolerance, false));
-        builder.add(new TestInput(dob.minus(1000), tolerance, false));
-        builder.add(new TestInput(dob.minus(999), tolerance, false));
-        builder.add(new TestInput(dob.minus(3), tolerance, false));
-        builder.add(new TestInput(dob.minus(2), tolerance, false));
-        builder.add(new TestInput(dob.minus(1), tolerance, false));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, false));
-        builder.add(new TestInput(dob.plus(2), tolerance, false));
-        builder.add(new TestInput(dob.plus(3), tolerance, false));
-        builder.add(new TestInput(dob.plus(999), tolerance, false));
-        builder.add(new TestInput(dob.plus(1000), tolerance, false));
-        builder.add(new TestInput(dob.plus(1001), tolerance, false));
-        builder.add(new TestInput(dob.plus(2000), tolerance, false));
-        builder.add(new TestInput(dob.plus(10000), tolerance, false));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, false));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.985)), tolerance, false));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.989)), tolerance, false));
-        // It's tricky near the exact threshold since it might create a fraction
-        // of a time which is rounded. Check if it's fraction and test it with
-        // the floor and ceiling values if it is. Otherwise, use the exact value
-        // if it is not a fraction.
-        final double lowerThresholdFloor = Math.floor(dob.getMillis() * 0.99);
-        final double lowerThresholdCeiling = Math.ceil(dob.getMillis() * 0.99);
-        // If the floor equals the ceiling then it's not a fraction.
-        if (lowerThresholdFloor != lowerThresholdCeiling) {
-           builder.add(new TestInput(new DateTime((long) lowerThresholdFloor), tolerance, false));
-        }
-        builder.add(new TestInput(new DateTime((long) lowerThresholdCeiling), tolerance, true));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.991)), tolerance, true));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.995)), tolerance, true));
-        builder.add(new TestInput(dob.minus(10000), tolerance, true));
-        builder.add(new TestInput(dob.minus(2000), tolerance, true));
-        builder.add(new TestInput(dob.minus(1001), tolerance, true));
-        builder.add(new TestInput(dob.minus(1000), tolerance, true));
-        builder.add(new TestInput(dob.minus(999), tolerance, true));
-        builder.add(new TestInput(dob.minus(3), tolerance, true));
-        builder.add(new TestInput(dob.minus(2), tolerance, true));
-        builder.add(new TestInput(dob.minus(1), tolerance, true));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, true));
-        builder.add(new TestInput(dob.plus(2), tolerance, true));
-        builder.add(new TestInput(dob.plus(3), tolerance, true));
-        builder.add(new TestInput(dob.plus(999), tolerance, true));
-        builder.add(new TestInput(dob.plus(1000), tolerance, true));
-        builder.add(new TestInput(dob.plus(1001), tolerance, true));
-        builder.add(new TestInput(dob.plus(2000), tolerance, true));
-        builder.add(new TestInput(dob.plus(10000), tolerance, true));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.005)), tolerance, true));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.009)), tolerance, true));
-        // It's tricky near the exact threshold since it might create a fraction
-        // of a time which is rounded. Check if it's fraction and test it with
-        // the floor and ceiling values if it is. Otherwise, use the exact value
-        // if it is not a fraction.
-        final double upperThresholdFloor = Math.floor(dob.getMillis() * 1.01);
-        final double upperThresholdCeiling = Math.ceil(dob.getMillis() * 1.01);
-        builder.add(new TestInput(new DateTime((long) upperThresholdFloor), tolerance, true));
-        // If the floor equals the ceiling then it's not a fraction.
-        if (upperThresholdFloor != upperThresholdCeiling) {
-           builder.add(new TestInput(new DateTime((long) upperThresholdCeiling), tolerance, false));
-        }
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.011)), tolerance, false));
-        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.015)), tolerance, false));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(new DateTime(0L), tolerance, true));
-        builder.add(new TestInput(dob.minusYears(1), tolerance, true));
-        builder.add(new TestInput(dob.minus(10000), tolerance, true));
-        builder.add(new TestInput(dob.minus(2000), tolerance, true));
-        builder.add(new TestInput(dob.minus(1001), tolerance, true));
-        builder.add(new TestInput(dob.minus(1000), tolerance, true));
-        builder.add(new TestInput(dob.minus(999), tolerance, true));
-        builder.add(new TestInput(dob.minus(3), tolerance, true));
-        builder.add(new TestInput(dob.minus(2), tolerance, true));
-        builder.add(new TestInput(dob.minus(1), tolerance, true));
-        builder.add(new TestInput(dob, tolerance, true)); // Equals value
-        builder.add(new TestInput(dob.plus(1), tolerance, true));
-        builder.add(new TestInput(dob.plus(2), tolerance, true));
-        builder.add(new TestInput(dob.plus(3), tolerance, true));
-        builder.add(new TestInput(dob.plus(999), tolerance, true));
-        builder.add(new TestInput(dob.plus(1000), tolerance, true));
-        builder.add(new TestInput(dob.plus(1001), tolerance, true));
-        builder.add(new TestInput(dob.plus(2000), tolerance, true));
-        builder.add(new TestInput(dob.plus(10000), tolerance, true));
-        builder.add(new TestInput(dob.plusYears(1), tolerance, true));
-        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_DATE_OF_BIRTH);
-    }
-
-    @Test
-    public void testDoubleProperty() throws SmartUriException {
-        System.out.println("Double Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0, tolerance, false));
-        builder.add(new TestInput(0.0, tolerance, false));
-        builder.add(new TestInput(0.01, tolerance, false));
-        builder.add(new TestInput(0.02, tolerance, false));
-        builder.add(new TestInput(0.1, tolerance, false));
-        builder.add(new TestInput(0.2, tolerance, false));
-        builder.add(new TestInput(1.0, tolerance, false));
-        builder.add(new TestInput(71, tolerance, false));
-        builder.add(new TestInput(72, tolerance, false));
-        builder.add(new TestInput(72.4, tolerance, false));
-        builder.add(new TestInput(72.47, tolerance, false));
-        builder.add(new TestInput(72.48, tolerance, false));
-        builder.add(new TestInput(72.49, tolerance, false));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, false));
-        builder.add(new TestInput(72.52, tolerance, false));
-        builder.add(new TestInput(72.53, tolerance, false));
-        builder.add(new TestInput(72.6, tolerance, false));
-        builder.add(new TestInput(73, tolerance, false));
-        builder.add(new TestInput(74, tolerance, false));
-        builder.add(new TestInput(100, tolerance, false));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
-        // Tolerance 0.01
-        tolerance = new Tolerance(0.01, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0, tolerance, false));
-        builder.add(new TestInput(0.0, tolerance, false));
-        builder.add(new TestInput(0.01, tolerance, false));
-        builder.add(new TestInput(0.02, tolerance, false));
-        builder.add(new TestInput(0.1, tolerance, false));
-        builder.add(new TestInput(0.2, tolerance, false));
-        builder.add(new TestInput(1.0, tolerance, false));
-        builder.add(new TestInput(71, tolerance, false));
-        builder.add(new TestInput(72, tolerance, false));
-        builder.add(new TestInput(72.4, tolerance, false));
-        builder.add(new TestInput(72.47, tolerance, false));
-        builder.add(new TestInput(72.48, tolerance, false));
-        builder.add(new TestInput(72.49, tolerance, true));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, true));
-        builder.add(new TestInput(72.52, tolerance, false));
-        builder.add(new TestInput(72.53, tolerance, false));
-        builder.add(new TestInput(72.6, tolerance, false));
-        builder.add(new TestInput(73, tolerance, false));
-        builder.add(new TestInput(74, tolerance, false));
-        builder.add(new TestInput(100, tolerance, false));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
-        // Tolerance 0.02
-        tolerance = new Tolerance(0.02, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0, tolerance, false));
-        builder.add(new TestInput(0.0, tolerance, false));
-        builder.add(new TestInput(0.01, tolerance, false));
-        builder.add(new TestInput(0.02, tolerance, false));
-        builder.add(new TestInput(0.1, tolerance, false));
-        builder.add(new TestInput(0.2, tolerance, false));
-        builder.add(new TestInput(1.0, tolerance, false));
-        builder.add(new TestInput(71, tolerance, false));
-        builder.add(new TestInput(72, tolerance, false));
-        builder.add(new TestInput(72.4, tolerance, false));
-        builder.add(new TestInput(72.47, tolerance, false));
-        builder.add(new TestInput(72.48, tolerance, true));
-        builder.add(new TestInput(72.49, tolerance, true));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, true));
-        builder.add(new TestInput(72.52, tolerance, true));
-        builder.add(new TestInput(72.53, tolerance, false));
-        builder.add(new TestInput(72.6, tolerance, false));
-        builder.add(new TestInput(73, tolerance, false));
-        builder.add(new TestInput(74, tolerance, false));
-        builder.add(new TestInput(100, tolerance, false));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0%
-        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0, tolerance, false));
-        builder.add(new TestInput(0.0, tolerance, false));
-        builder.add(new TestInput(0.01, tolerance, false));
-        builder.add(new TestInput(0.02, tolerance, false));
-        builder.add(new TestInput(0.1, tolerance, false));
-        builder.add(new TestInput(0.2, tolerance, false));
-        builder.add(new TestInput(1.0, tolerance, false));
-        builder.add(new TestInput(71, tolerance, false));
-        builder.add(new TestInput(71.774, tolerance, false));
-        builder.add(new TestInput(71.775, tolerance, false));
-        builder.add(new TestInput(71.776, tolerance, false));
-        builder.add(new TestInput(72, tolerance, false));
-        builder.add(new TestInput(72.4, tolerance, false));
-        builder.add(new TestInput(72.47, tolerance, false));
-        builder.add(new TestInput(72.48, tolerance, false));
-        builder.add(new TestInput(72.49, tolerance, false));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, false));
-        builder.add(new TestInput(72.52, tolerance, false));
-        builder.add(new TestInput(72.53, tolerance, false));
-        builder.add(new TestInput(72.6, tolerance, false));
-        builder.add(new TestInput(73, tolerance, false));
-        builder.add(new TestInput(73.224, tolerance, false));
-        builder.add(new TestInput(73.225, tolerance, false));
-        builder.add(new TestInput(73.226, tolerance, false));
-        builder.add(new TestInput(73, tolerance, false));
-        builder.add(new TestInput(74, tolerance, false));
-        builder.add(new TestInput(100, tolerance, false));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
-        // Tolerance 1%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0, tolerance, false));
-        builder.add(new TestInput(0.0, tolerance, false));
-        builder.add(new TestInput(0.01, tolerance, false));
-        builder.add(new TestInput(0.02, tolerance, false));
-        builder.add(new TestInput(0.1, tolerance, false));
-        builder.add(new TestInput(0.2, tolerance, false));
-        builder.add(new TestInput(1.0, tolerance, false));
-        builder.add(new TestInput(71, tolerance, false));
-        builder.add(new TestInput(71.774, tolerance, false));
-        builder.add(new TestInput(71.775, tolerance, true));
-        builder.add(new TestInput(71.776, tolerance, true));
-        builder.add(new TestInput(72, tolerance, true));
-        builder.add(new TestInput(72.4, tolerance, true));
-        builder.add(new TestInput(72.47, tolerance, true));
-        builder.add(new TestInput(72.48, tolerance, true));
-        builder.add(new TestInput(72.49, tolerance, true));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, true));
-        builder.add(new TestInput(72.52, tolerance, true));
-        builder.add(new TestInput(72.53, tolerance, true));
-        builder.add(new TestInput(72.6, tolerance, true));
-        builder.add(new TestInput(73, tolerance, true));
-        builder.add(new TestInput(73.224, tolerance, true));
-        builder.add(new TestInput(73.225, tolerance, true));
-        builder.add(new TestInput(73.226, tolerance, false));
-        builder.add(new TestInput(74, tolerance, false));
-        builder.add(new TestInput(100, tolerance, false));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
-        // Tolerance 100%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Double.MIN_VALUE, tolerance, true));
-        builder.add(new TestInput(-1.0, tolerance, true));
-        builder.add(new TestInput(0.0, tolerance, true));
-        builder.add(new TestInput(0.01, tolerance, true));
-        builder.add(new TestInput(0.02, tolerance, true));
-        builder.add(new TestInput(0.1, tolerance, true));
-        builder.add(new TestInput(0.2, tolerance, true));
-        builder.add(new TestInput(1.0, tolerance, true));
-        builder.add(new TestInput(71, tolerance, true));
-        builder.add(new TestInput(71.774, tolerance, true));
-        builder.add(new TestInput(71.775, tolerance, true));
-        builder.add(new TestInput(71.776, tolerance, true));
-        builder.add(new TestInput(72, tolerance, true));
-        builder.add(new TestInput(72.4, tolerance, true));
-        builder.add(new TestInput(72.47, tolerance, true));
-        builder.add(new TestInput(72.48, tolerance, true));
-        builder.add(new TestInput(72.49, tolerance, true));
-        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
-        builder.add(new TestInput(72.51, tolerance, true));
-        builder.add(new TestInput(72.52, tolerance, true));
-        builder.add(new TestInput(72.53, tolerance, true));
-        builder.add(new TestInput(72.6, tolerance, true));
-        builder.add(new TestInput(73, tolerance, true));
-        builder.add(new TestInput(73.224, tolerance, true));
-        builder.add(new TestInput(73.225, tolerance, true));
-        builder.add(new TestInput(73.226, tolerance, true));
-        builder.add(new TestInput(74, tolerance, true));
-        builder.add(new TestInput(100, tolerance, true));
-        builder.add(new TestInput(Double.MAX_VALUE, tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_HEIGHT);
-    }
-
-    @Test
-    public void testFloatProperty() throws SmartUriException {
-        System.out.println("Float Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0f, tolerance, false));
-        builder.add(new TestInput(0.0f, tolerance, false));
-        builder.add(new TestInput(0.01f, tolerance, false));
-        builder.add(new TestInput(0.02f, tolerance, false));
-        builder.add(new TestInput(0.1f, tolerance, false));
-        builder.add(new TestInput(0.2f, tolerance, false));
-        builder.add(new TestInput(1.0f, tolerance, false));
-        builder.add(new TestInput(250f, tolerance, false));
-        builder.add(new TestInput(250.7f, tolerance, false));
-        builder.add(new TestInput(250.72f, tolerance, false));
-        builder.add(new TestInput(250.73f, tolerance, false));
-        builder.add(new TestInput(250.74f, tolerance, false));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, false));
-        builder.add(new TestInput(250.77f, tolerance, false));
-        builder.add(new TestInput(250.78f, tolerance, false));
-        builder.add(new TestInput(250.8f, tolerance, false));
-        builder.add(new TestInput(251f, tolerance, false));
-        builder.add(new TestInput(300.0f, tolerance, false));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
-        // Tolerance 0.01
-        tolerance = new Tolerance(0.01, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0f, tolerance, false));
-        builder.add(new TestInput(0.0f, tolerance, false));
-        builder.add(new TestInput(0.01f, tolerance, false));
-        builder.add(new TestInput(0.02f, tolerance, false));
-        builder.add(new TestInput(0.1f, tolerance, false));
-        builder.add(new TestInput(0.2f, tolerance, false));
-        builder.add(new TestInput(1.0f, tolerance, false));
-        builder.add(new TestInput(250f, tolerance, false));
-        builder.add(new TestInput(250.7f, tolerance, false));
-        builder.add(new TestInput(250.72f, tolerance, false));
-        builder.add(new TestInput(250.73f, tolerance, false));
-        builder.add(new TestInput(250.74f, tolerance, true));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, true));
-        builder.add(new TestInput(250.77f, tolerance, false));
-        builder.add(new TestInput(250.78f, tolerance, false));
-        builder.add(new TestInput(250.8f, tolerance, false));
-        builder.add(new TestInput(251f, tolerance, false));
-        builder.add(new TestInput(300.0f, tolerance, false));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
-        // Tolerance 0.02
-        tolerance = new Tolerance(0.02, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0f, tolerance, false));
-        builder.add(new TestInput(0.0f, tolerance, false));
-        builder.add(new TestInput(0.01f, tolerance, false));
-        builder.add(new TestInput(0.02f, tolerance, false));
-        builder.add(new TestInput(0.1f, tolerance, false));
-        builder.add(new TestInput(0.2f, tolerance, false));
-        builder.add(new TestInput(1.0f, tolerance, false));
-        builder.add(new TestInput(250f, tolerance, false));
-        builder.add(new TestInput(250.7f, tolerance, false));
-        builder.add(new TestInput(250.72f, tolerance, false));
-        builder.add(new TestInput(250.73f, tolerance, true));
-        builder.add(new TestInput(250.74f, tolerance, true));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, true));
-        builder.add(new TestInput(250.77f, tolerance, true));
-        builder.add(new TestInput(250.78f, tolerance, false));
-        builder.add(new TestInput(250.8f, tolerance, false));
-        builder.add(new TestInput(251f, tolerance, false));
-        builder.add(new TestInput(300.0f, tolerance, false));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0f, tolerance, false));
-        builder.add(new TestInput(0.0f, tolerance, false));
-        builder.add(new TestInput(0.01f, tolerance, false));
-        builder.add(new TestInput(0.02f, tolerance, false));
-        builder.add(new TestInput(0.1f, tolerance, false));
-        builder.add(new TestInput(0.2f, tolerance, false));
-        builder.add(new TestInput(1.0f, tolerance, false));
-        builder.add(new TestInput(248.2424f, tolerance, false));
-        builder.add(new TestInput(248.2425f, tolerance, false));
-        builder.add(new TestInput(248.2426f, tolerance, false));
-        builder.add(new TestInput(250f, tolerance, false));
-        builder.add(new TestInput(250.7f, tolerance, false));
-        builder.add(new TestInput(250.72f, tolerance, false));
-        builder.add(new TestInput(250.73f, tolerance, false));
-        builder.add(new TestInput(250.74f, tolerance, false));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, false));
-        builder.add(new TestInput(250.77f, tolerance, false));
-        builder.add(new TestInput(250.78f, tolerance, false));
-        builder.add(new TestInput(250.8f, tolerance, false));
-        builder.add(new TestInput(251f, tolerance, false));
-        builder.add(new TestInput(253.2574f, tolerance, false));
-        builder.add(new TestInput(253.2575f, tolerance, false));
-        builder.add(new TestInput(253.2576f, tolerance, false));
-        builder.add(new TestInput(300.0f, tolerance, false));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1.0f, tolerance, false));
-        builder.add(new TestInput(0.0f, tolerance, false));
-        builder.add(new TestInput(0.01f, tolerance, false));
-        builder.add(new TestInput(0.02f, tolerance, false));
-        builder.add(new TestInput(0.1f, tolerance, false));
-        builder.add(new TestInput(0.2f, tolerance, false));
-        builder.add(new TestInput(1.0f, tolerance, false));
-        builder.add(new TestInput(248.2424f, tolerance, false));
-        builder.add(new TestInput(248.2425f, tolerance, true));
-        builder.add(new TestInput(248.2426f, tolerance, true));
-        builder.add(new TestInput(250f, tolerance, true));
-        builder.add(new TestInput(250.7f, tolerance, true));
-        builder.add(new TestInput(250.72f, tolerance, true));
-        builder.add(new TestInput(250.73f, tolerance, true));
-        builder.add(new TestInput(250.74f, tolerance, true));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, true));
-        builder.add(new TestInput(250.77f, tolerance, true));
-        builder.add(new TestInput(250.78f, tolerance, true));
-        builder.add(new TestInput(250.8f, tolerance, true));
-        builder.add(new TestInput(251f, tolerance, true));
-        builder.add(new TestInput(253.2574f, tolerance, true));
-        builder.add(new TestInput(253.2575f, tolerance, true));
-        builder.add(new TestInput(253.2576f, tolerance, false));
-        builder.add(new TestInput(300.0f, tolerance, false));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Float.MIN_VALUE, tolerance, true));
-        builder.add(new TestInput(-1.0f, tolerance, true));
-        builder.add(new TestInput(0.0f, tolerance, true));
-        builder.add(new TestInput(0.01f, tolerance, true));
-        builder.add(new TestInput(0.02f, tolerance, true));
-        builder.add(new TestInput(0.1f, tolerance, true));
-        builder.add(new TestInput(0.2f, tolerance, true));
-        builder.add(new TestInput(1.0f, tolerance, true));
-        builder.add(new TestInput(248.2424f, tolerance, true));
-        builder.add(new TestInput(248.2425f, tolerance, true));
-        builder.add(new TestInput(248.2426f, tolerance, true));
-        builder.add(new TestInput(250f, tolerance, true));
-        builder.add(new TestInput(250.7f, tolerance, true));
-        builder.add(new TestInput(250.72f, tolerance, true));
-        builder.add(new TestInput(250.73f, tolerance, true));
-        builder.add(new TestInput(250.74f, tolerance, true));
-        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
-        builder.add(new TestInput(250.76f, tolerance, true));
-        builder.add(new TestInput(250.77f, tolerance, true));
-        builder.add(new TestInput(250.78f, tolerance, true));
-        builder.add(new TestInput(250.8f, tolerance, true));
-        builder.add(new TestInput(251f, tolerance, true));
-        builder.add(new TestInput(253.2574f, tolerance, true));
-        builder.add(new TestInput(253.2575f, tolerance, true));
-        builder.add(new TestInput(253.2576f, tolerance, true));
-        builder.add(new TestInput(300.0f, tolerance, true));
-        builder.add(new TestInput(Float.MAX_VALUE, tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_WEIGHT);
-    }
-
-    @Test
-    public void testIntegerProperty() throws SmartUriException {
-        System.out.println("Integer Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1, tolerance, false));
-        builder.add(new TestInput(0, tolerance, false));
-        builder.add(new TestInput(1, tolerance, false));
-        builder.add(new TestInput(49997, tolerance, false));
-        builder.add(new TestInput(49998, tolerance, false));
-        builder.add(new TestInput(49999, tolerance, false));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, false));
-        builder.add(new TestInput(50002, tolerance, false));
-        builder.add(new TestInput(50003, tolerance, false));
-        builder.add(new TestInput(60000, tolerance, false));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1, tolerance, false));
-        builder.add(new TestInput(0, tolerance, false));
-        builder.add(new TestInput(1, tolerance, false));
-        builder.add(new TestInput(49997, tolerance, false));
-        builder.add(new TestInput(49998, tolerance, false));
-        builder.add(new TestInput(49999, tolerance, true));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, true));
-        builder.add(new TestInput(50002, tolerance, false));
-        builder.add(new TestInput(50003, tolerance, false));
-        builder.add(new TestInput(60000, tolerance, false));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1, tolerance, false));
-        builder.add(new TestInput(0, tolerance, false));
-        builder.add(new TestInput(1, tolerance, false));
-        builder.add(new TestInput(49997, tolerance, false));
-        builder.add(new TestInput(49998, tolerance, true));
-        builder.add(new TestInput(49999, tolerance, true));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, true));
-        builder.add(new TestInput(50002, tolerance, true));
-        builder.add(new TestInput(50003, tolerance, false));
-        builder.add(new TestInput(60000, tolerance, false));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1, tolerance, false));
-        builder.add(new TestInput(0, tolerance, false));
-        builder.add(new TestInput(1, tolerance, false));
-        builder.add(new TestInput(48999, tolerance, false));
-        builder.add(new TestInput(49000, tolerance, false));
-        builder.add(new TestInput(49001, tolerance, false));
-        builder.add(new TestInput(49499, tolerance, false));
-        builder.add(new TestInput(49500, tolerance, false));
-        builder.add(new TestInput(49501, tolerance, false));
-        builder.add(new TestInput(49997, tolerance, false));
-        builder.add(new TestInput(49998, tolerance, false));
-        builder.add(new TestInput(49999, tolerance, false));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, false));
-        builder.add(new TestInput(50002, tolerance, false));
-        builder.add(new TestInput(50003, tolerance, false));
-        builder.add(new TestInput(50499, tolerance, false));
-        builder.add(new TestInput(50500, tolerance, false));
-        builder.add(new TestInput(50501, tolerance, false));
-        builder.add(new TestInput(50999, tolerance, false));
-        builder.add(new TestInput(51000, tolerance, false));
-        builder.add(new TestInput(51001, tolerance, false));
-        builder.add(new TestInput(60000, tolerance, false));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1, tolerance, false));
-        builder.add(new TestInput(0, tolerance, false));
-        builder.add(new TestInput(1, tolerance, false));
-        builder.add(new TestInput(48999, tolerance, false));
-        builder.add(new TestInput(49000, tolerance, false));
-        builder.add(new TestInput(49001, tolerance, false));
-        builder.add(new TestInput(49499, tolerance, false));
-        builder.add(new TestInput(49500, tolerance, true));
-        builder.add(new TestInput(49501, tolerance, true));
-        builder.add(new TestInput(49997, tolerance, true));
-        builder.add(new TestInput(49998, tolerance, true));
-        builder.add(new TestInput(49999, tolerance, true));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, true));
-        builder.add(new TestInput(50002, tolerance, true));
-        builder.add(new TestInput(50003, tolerance, true));
-        builder.add(new TestInput(50499, tolerance, true));
-        builder.add(new TestInput(50500, tolerance, true));
-        builder.add(new TestInput(50501, tolerance, false));
-        builder.add(new TestInput(50999, tolerance, false));
-        builder.add(new TestInput(51000, tolerance, false));
-        builder.add(new TestInput(51001, tolerance, false));
-        builder.add(new TestInput(60000, tolerance, false));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, true));
-        builder.add(new TestInput(-1, tolerance, true));
-        builder.add(new TestInput(0, tolerance, true));
-        builder.add(new TestInput(1, tolerance, true));
-        builder.add(new TestInput(48999, tolerance, true));
-        builder.add(new TestInput(49000, tolerance, true));
-        builder.add(new TestInput(49001, tolerance, true));
-        builder.add(new TestInput(49499, tolerance, true));
-        builder.add(new TestInput(49500, tolerance, true));
-        builder.add(new TestInput(49501, tolerance, true));
-        builder.add(new TestInput(49997, tolerance, true));
-        builder.add(new TestInput(49998, tolerance, true));
-        builder.add(new TestInput(49999, tolerance, true));
-        builder.add(new TestInput(50000, tolerance, true)); // Equals value
-        builder.add(new TestInput(50001, tolerance, true));
-        builder.add(new TestInput(50002, tolerance, true));
-        builder.add(new TestInput(50003, tolerance, true));
-        builder.add(new TestInput(50499, tolerance, true));
-        builder.add(new TestInput(50500, tolerance, true));
-        builder.add(new TestInput(50501, tolerance, true));
-        builder.add(new TestInput(50999, tolerance, true));
-        builder.add(new TestInput(51000, tolerance, true));
-        builder.add(new TestInput(51001, tolerance, true));
-        builder.add(new TestInput(60000, tolerance, true));
-        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_INCOME);
-    }
-
-    @Test
-    public void testLongProperty() throws SmartUriException {
-        System.out.println("Long Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1L, tolerance, false));
-        builder.add(new TestInput(0L, tolerance, false));
-        builder.add(new TestInput(1L, tolerance, false));
-        builder.add(new TestInput(123456789009L, tolerance, false));
-        builder.add(new TestInput(123456789010L, tolerance, false));
-        builder.add(new TestInput(123456789011L, tolerance, false));
-        builder.add(new TestInput(123456789012L, tolerance, true)); // Equals value
-        builder.add(new TestInput(123456789013L, tolerance, false));
-        builder.add(new TestInput(123456789014L, tolerance, false));
-        builder.add(new TestInput(123456789015L, tolerance, false));
-        builder.add(new TestInput(223456789012L, tolerance, false));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1L, tolerance, false));
-        builder.add(new TestInput(0L, tolerance, false));
-        builder.add(new TestInput(1L, tolerance, false));
-        builder.add(new TestInput(123456789009L, tolerance, false));
-        builder.add(new TestInput(123456789010L, tolerance, false));
-        builder.add(new TestInput(123456789011L, tolerance, true));
-        builder.add(new TestInput(123456789012L, tolerance, true)); // Equals value
-        builder.add(new TestInput(123456789013L, tolerance, true));
-        builder.add(new TestInput(123456789014L, tolerance, false));
-        builder.add(new TestInput(123456789015L, tolerance, false));
-        builder.add(new TestInput(223456789012L, tolerance, false));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1L, tolerance, false));
-        builder.add(new TestInput(0L, tolerance, false));
-        builder.add(new TestInput(1L, tolerance, false));
-        builder.add(new TestInput(123456789009L, tolerance, false));
-        builder.add(new TestInput(123456789010L, tolerance, true));
-        builder.add(new TestInput(123456789011L, tolerance, true));
-        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
-        builder.add(new TestInput(123456789013L, tolerance, true));
-        builder.add(new TestInput(123456789014L, tolerance, true));
-        builder.add(new TestInput(123456789015L, tolerance, false));
-        builder.add(new TestInput(223456789012L, tolerance, false));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1L, tolerance, false));
-        builder.add(new TestInput(0L, tolerance, false));
-        builder.add(new TestInput(1L, tolerance, false));
-        builder.add(new TestInput(122222221121L, tolerance, false));
-        builder.add(new TestInput(122222221122L, tolerance, false));
-        builder.add(new TestInput(122222221123L, tolerance, false));
-        builder.add(new TestInput(123456789009L, tolerance, false));
-        builder.add(new TestInput(123456789010L, tolerance, false));
-        builder.add(new TestInput(123456789011L, tolerance, false));
-        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
-        builder.add(new TestInput(123456789013L, tolerance, false));
-        builder.add(new TestInput(123456789014L, tolerance, false));
-        builder.add(new TestInput(123456789015L, tolerance, false));
-        builder.add(new TestInput(124691356901L, tolerance, false));
-        builder.add(new TestInput(124691356902L, tolerance, false));
-        builder.add(new TestInput(124691356903L, tolerance, false));
-        builder.add(new TestInput(223456789012L, tolerance, false));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0%
-        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput(-1L, tolerance, false));
-        builder.add(new TestInput(0L, tolerance, false));
-        builder.add(new TestInput(1L, tolerance, false));
-        builder.add(new TestInput(122222221121L, tolerance, false));
-        builder.add(new TestInput(122222221122L, tolerance, true));
-        builder.add(new TestInput(122222221123L, tolerance, true));
-        builder.add(new TestInput(123456789009L, tolerance, true));
-        builder.add(new TestInput(123456789010L, tolerance, true));
-        builder.add(new TestInput(123456789011L, tolerance, true));
-        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
-        builder.add(new TestInput(123456789013L, tolerance, true));
-        builder.add(new TestInput(123456789014L, tolerance, true));
-        builder.add(new TestInput(123456789015L, tolerance, true));
-        builder.add(new TestInput(124691356901L, tolerance, true));
-        builder.add(new TestInput(124691356902L, tolerance, true));
-        builder.add(new TestInput(124691356903L, tolerance, false));
-        builder.add(new TestInput(223456789012L, tolerance, false));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
-        // Tolerance 100.0%
-        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Long.MIN_VALUE, tolerance, true));
-        builder.add(new TestInput(-1L, tolerance, true));
-        builder.add(new TestInput(0L, tolerance, true));
-        builder.add(new TestInput(1L, tolerance, true));
-        builder.add(new TestInput(122222221121L, tolerance, true));
-        builder.add(new TestInput(122222221122L, tolerance, true));
-        builder.add(new TestInput(122222221123L, tolerance, true));
-        builder.add(new TestInput(123456789009L, tolerance, true));
-        builder.add(new TestInput(123456789010L, tolerance, true));
-        builder.add(new TestInput(123456789011L, tolerance, true));
-        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
-        builder.add(new TestInput(123456789013L, tolerance, true));
-        builder.add(new TestInput(123456789014L, tolerance, true));
-        builder.add(new TestInput(123456789015L, tolerance, true));
-        builder.add(new TestInput(124691356901L, tolerance, true));
-        builder.add(new TestInput(124691356902L, tolerance, true));
-        builder.add(new TestInput(124691356903L, tolerance, true));
-        builder.add(new TestInput(223456789012L, tolerance, true));
-        builder.add(new TestInput(Long.MAX_VALUE, tolerance, true));
-
-        final ImmutableList<TestInput> testInputs = builder.build();
-
-        testProperty(testInputs, PERSON_TYPE_URI, HAS_LICENSE_NUMBER);
-    }
-
-    @Test
-    public void testShortProperty() throws SmartUriException {
-        System.out.println("Short Property Test");
-        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
-        // Tolerance 0.0
-        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((short) -1, tolerance, false));
-        builder.add(new TestInput((short) 0, tolerance, false));
-        builder.add(new TestInput((short) 1, tolerance, false));
-        builder.add(new TestInput((short) 37, tolerance, false));
-        builder.add(new TestInput((short) 38, tolerance, false));
-        builder.add(new TestInput((short) 39, tolerance, false));
-        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
-        builder.add(new TestInput((short) 41, tolerance, false));
-        builder.add(new TestInput((short) 42, tolerance, false));
-        builder.add(new TestInput((short) 43, tolerance, false));
-        builder.add(new TestInput((short) 100, tolerance, false));
-        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
-        // Tolerance 1.0
-        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((short) -1, tolerance, false));
-        builder.add(new TestInput((short) 0, tolerance, false));
-        builder.add(new TestInput((short) 1, tolerance, false));
-        builder.add(new TestInput((short) 37, tolerance, false));
-        builder.add(new TestInput((short) 38, tolerance, false));
-        builder.add(new TestInput((short) 39, tolerance, true));
-        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
-        builder.add(new TestInput((short) 41, tolerance, true));
-        builder.add(new TestInput((short) 42, tolerance, false));
-        builder.add(new TestInput((short) 43, tolerance, false));
-        builder.add(new TestInput((short) 100, tolerance, false));
-        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
-        // Tolerance 2.0
-        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
-        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((short) -1, tolerance, false));
-        builder.add(new TestInput((short) 0, tolerance, false));
-        builder.add(new TestInput((short) 1, tolerance, false));
-        builder.add(new TestInput((short) 37, tolerance, false));
-        builder.add(new TestInput((short) 38, tolerance, true));
-        builder.add(new TestInput((short) 39, tolerance, true));
-        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
-        builder.add(new TestInput((short) 41, tolerance, true));
-        builder.add(new TestInput((short) 42, tolerance, true));
-        builder.add(new TestInput((short) 43, tolerance, false));
-        builder.add(new TestInput((short) 100, tolerance, false));
-        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
-
-        // Tolerance 0.0%
-        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
-        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
-        builder.add(new TestInput((short) -1, tolerance, false));
-        builder.add(new TestInput((short) 0, tolerance, false));
-        builder.add(new TestInput((short) 1, tolerance, false));
-        builder.add(new TestInput((short) 37, tolerance, false));
-        build

<TRUNCATED>


[07/17] incubator-rya git commit: RYA-414 Removed mongo connection factory

Posted by dl...@apache.org.
RYA-414 Removed mongo connection factory

addressed the indexers that used the factory
addressed the geo project


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

Branch: refs/heads/master
Commit: 17cebae3328916bd80fbe5447da5ccb660539556
Parents: 767349d
Author: Andrew Smith <sm...@gmail.com>
Authored: Tue Dec 26 14:30:32 2017 -0500
Committer: kchilton2 <ke...@gmail.com>
Committed: Wed Dec 27 14:08:54 2017 -0500

----------------------------------------------------------------------
 .../org/apache/rya/mongodb/MongoDBRyaDAO.java   |   1 -
 .../apache/rya/mongodb/MongoSecondaryIndex.java |  20 +-
 .../StatefulMongoDBRdfConfiguration.java        |  25 +-
 .../rya/indexing/FilterFunctionOptimizer.java   |  29 +-
 .../indexing/entity/EntityIndexOptimizer.java   |   5 +-
 .../entity/update/BaseEntityIndexer.java        |  25 +-
 .../indexing/entity/update/EntityIndexer.java   |   7 +-
 .../entity/update/mongo/MongoEntityIndexer.java |  43 +-
 .../indexing/mongodb/AbstractMongoIndexer.java  |  10 +-
 .../mongodb/freetext/MongoFreeTextIndexer.java  |   2 +-
 .../apache/rya/sail/config/RyaSailFactory.java  |   6 +-
 .../update/mongo/MongoEntityIndexerIT.java      | 457 +++++++-------
 .../rya/indexing/mongo/MongoEntityIndexIT.java  |  40 +-
 .../metadata/MongoStatementMetadataIT.java      | 401 ++++++------
 .../MongoStatementMetadataNodeTest.java         | 623 ++++++++++---------
 .../src/main/java/MongoRyaDirectExample.java    |  14 +-
 .../client/conf/MergeConfigHadoopAdapter.java   |   4 +-
 .../client/merge/StatementStoreFactory.java     |   7 +-
 .../rya/indexing/export/StoreToStoreIT.java     |   9 +-
 .../GeoEnabledFilterFunctionOptimizer.java      |  27 +-
 .../apache/rya/indexing/GeoRyaSailFactory.java  |  99 ++-
 .../geotemporal/GeoTemporalIndexer.java         |  11 +-
 .../geotemporal/GeoTemporalIndexerFactory.java  |  21 +-
 .../geotemporal/GeoTemporalOptimizer.java       |   2 +-
 extras/rya.geoindexing/geo.mongo/pom.xml        |   6 +
 .../geoExamples/RyaMongoGeoDirectExample.java   |   5 -
 .../mongo/MongoGeoTemporalIndexer.java          |  27 +-
 .../geotemporal/GeoTemporalProviderTest.java    |   2 +-
 .../geotemporal/GeoTemporalTestBase.java        | 140 -----
 .../geotemporal/GeoTemporalTestUtils.java       | 140 +++++
 .../geotemporal/MongoGeoTemporalIndexIT.java    | 257 ++++----
 .../geotemporal/model/EventQueryNodeTest.java   |   6 +-
 .../GeoTemporalMongoDBStorageStrategyTest.java  |   4 +-
 .../mongo/MongoEventStorageTest.java            |   3 +-
 .../mongo/MongoGeoTemporalIndexerIT.java        |   7 +-
 .../indexing/geotemporal/mongo/MongoITBase.java |  64 --
 .../indexing/mongo/MongoGeoIndexerFilterIT.java | 479 +++++++-------
 .../indexing/mongo/MongoGeoIndexerSfTest.java   | 102 +--
 .../rya/indexing/mongo/MongoGeoIndexerTest.java |  95 +--
 .../indexing/mongo/MongoIndexerDeleteIT.java    | 146 ++---
 .../rya/rdftriplestore/RdfCloudTripleStore.java |  13 +-
 .../RdfCloudTripleStoreConnection.java          |   6 +-
 42 files changed, 1683 insertions(+), 1707 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 01bbee7..4da36d5 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
@@ -103,7 +103,6 @@ public final class MongoDBRyaDAO implements RyaDAO<StatefulMongoDBRdfConfigurati
         secondaryIndexers = conf.getAdditionalIndexers();
         for(final MongoSecondaryIndex index: secondaryIndexers) {
             index.setConf(conf);
-            index.setClient(conf.getMongoClient());
         }
 
         db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
index 3d3e22f..3be6da8 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
@@ -1,6 +1,4 @@
-package org.apache.rya.mongodb;
-
-/*
+/**
  * 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
@@ -8,9 +6,9 @@ package org.apache.rya.mongodb;
  * 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,14 +16,14 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
-
-import com.mongodb.MongoClient;
+package org.apache.rya.mongodb;
 
 import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
 
+/**
+ *TODO: doc me.
+ */
 public interface MongoSecondaryIndex extends RyaSecondaryIndexer{
-    public void init();    
-
-    public void setClient(MongoClient client);
-	
+    @Override
+	public void init();    
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
index 1263166..12bc5e3 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.rya.mongodb;
 
 import static java.util.Objects.requireNonNull;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -43,9 +44,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
 
     private final MongoClient mongoClient;
-    private final List<MongoSecondaryIndex> indexers;
-    // TODO optimizers? They're causing problems because they aren't getting this configuration object
-    //      like the indexers do.
+    private List<MongoSecondaryIndex> indexers;
 
     /**
      * Constructs an instance of {@link StatefulMongoDBRdfConfiguration} pre-loaded with values.
@@ -62,8 +61,28 @@ public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
         this.mongoClient = requireNonNull(mongoClient);
         this.indexers = requireNonNull(indexers);
     }
+    
+    /**
+     * Constructs an instance of {@link StatefulMongoDBRdfConfiguration} pre-loaded with values.
+     *
+     * @param other - The values that will be cloned into the constructed object. (not null)
+     * @param mongoClient - The {@link MongoClient} that Rya will use. (not null)
+     */
+    public StatefulMongoDBRdfConfiguration(
+            final Configuration other,
+            final MongoClient mongoClient) {
+        this(other, mongoClient, new ArrayList<>());
+    }
 
     /**
+     * TODO doc
+     * @param indexers (not null)
+     */
+    public void setIndexers(final List<MongoSecondaryIndex> indexers) {
+    	this.indexers = requireNonNull(indexers);
+    }
+    
+    /**
      * @return The {@link MongoClient} that Rya will use.
      */
     public MongoClient getMongoClient() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
index 1bab129..58ec29a 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
@@ -34,6 +34,14 @@ import org.apache.commons.lang.Validate;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
+import org.apache.rya.indexing.accumulo.freetext.FreeTextTupleSet;
+import org.apache.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
+import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.model.Resource;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -58,15 +66,6 @@ import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
 
 import com.google.common.collect.Lists;
 
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.accumulo.freetext.AccumuloFreeTextIndexer;
-import org.apache.rya.indexing.accumulo.freetext.FreeTextTupleSet;
-import org.apache.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
-import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import org.apache.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
-
 public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     private static final Logger LOG = Logger.getLogger(FilterFunctionOptimizer.class);
     private final ValueFactory valueFactory = new ValueFactoryImpl();
@@ -98,10 +97,14 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     private synchronized void init() {
         if (!init) {
             if (ConfigUtils.getUseMongo(conf)) {
-                    freeTextIndexer = new MongoFreeTextIndexer();
-                    freeTextIndexer.setConf(conf);
-                    temporalIndexer = new MongoTemporalIndexer();
-                    temporalIndexer.setConf(conf);
+            	StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
+            	for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
+        			if(indexer instanceof FreeTextIndexer) {
+        				freeTextIndexer = (FreeTextIndexer) indexer;
+        			} else if(indexer instanceof TemporalIndexer) {
+        				temporalIndexer = (TemporalIndexer) indexer;
+        			}
+            	}
             } else {
                  freeTextIndexer = new AccumuloFreeTextIndexer();
                 freeTextIndexer.setConf(conf);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
index 507fde2..cec776b 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
@@ -75,10 +75,11 @@ public class EntityIndexOptimizer extends AbstractExternalSetOptimizer<EntityQue
 
         this.conf = conf;
         indexer.setConf(conf);
+        indexer.init();
 
-        typeStorage = indexer.getTypeStorage(conf);
+        typeStorage = indexer.getTypeStorage();
         try {
-            entityStorage = indexer.getEntityStorage(conf);
+            entityStorage = indexer.getEntityStorage();
         } catch (final EntityStorageException e) {
             log.error("Error getting entity storage", e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
index e73eeb3..2ca6761 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.rya.indexing.entity.update;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import static java.util.Collections.singleton;
 import static java.util.Objects.requireNonNull;
@@ -46,8 +47,8 @@ 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.ConvertingCursor;
 import org.apache.rya.indexing.mongodb.IndexingException;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.model.URI;
 import org.openrdf.model.vocabulary.RDF;
 
@@ -69,19 +70,27 @@ public abstract class BaseEntityIndexer implements EntityIndexer, MongoSecondary
      */
     private static final RyaURI TYPE_URI = new RyaURI( RDF.TYPE.toString() );
 
-    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
+    protected final AtomicReference<StatefulMongoDBRdfConfiguration> configuration = new AtomicReference<>();
     private final AtomicReference<EntityStorage> entities = new AtomicReference<>();
     private final AtomicReference<TypeStorage> types = new AtomicReference<>();
 
     @Override
+	public void init() {
+    	try {
+    		entities.set(getEntityStorage());
+    	} catch (final EntityStorageException e) {
+    		log.error("Unable to set entity storage.");
+    	}
+    	types.set(getTypeStorage());
+    }
+    
+    @Override
     public void setConf(final Configuration conf) {
         requireNonNull(conf);
-        try {
-            entities.set( getEntityStorage(conf) );
-        } catch (final EntityStorageException e) {
-            log.error("Unable to set entity storage.");
-        }
-        types.set( getTypeStorage(conf) );
+        checkArgument(conf instanceof StatefulMongoDBRdfConfiguration, 
+        		"The configuration provided must be a StatefulMongoDBRdfConfiguration, found: " 
+        	    + conf.getClass().getSimpleName());
+        configuration.set((StatefulMongoDBRdfConfiguration) conf);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityIndexer.java
index aeb5a41..0c816bf 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityIndexer.java
@@ -18,7 +18,6 @@
  */
 package org.apache.rya.indexing.entity.update;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
@@ -36,17 +35,15 @@ public interface EntityIndexer extends RyaSecondaryIndexer {
     /**
      * Creates the {@link EntityStorage} that will be used by the indexer.
      *
-     * @param conf - Indicates how the {@link EntityStorage} is initialized. (not null)
      * @return The {@link EntityStorage} that will be used by this indexer.
      * @throws EntityStorageException
      */
-    public @Nullable EntityStorage getEntityStorage(Configuration conf) throws EntityStorageException;
+    public @Nullable EntityStorage getEntityStorage() throws EntityStorageException;
 
     /**
      * Creates the {@link TypeStorage} that will be used by the indexer.
      *
-     * @param conf - Indicates how the {@link TypeStorage} is initialized. (not null)
      * @return The {@link TypeStorage} that will be used by this indexer.
      */
-    public @Nullable TypeStorage getTypeStorage(Configuration conf);
+    public @Nullable TypeStorage getTypeStorage();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
index d1d00fb..ea28388 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
@@ -18,9 +18,6 @@
  */
 package org.apache.rya.indexing.entity.update.mongo;
 
-import static com.google.common.base.Preconditions.checkState;
-
-import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
@@ -30,8 +27,6 @@ import org.apache.rya.indexing.entity.update.BaseEntityIndexer;
 import org.apache.rya.indexing.entity.update.EntityIndexer;
 import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
-import com.mongodb.MongoClient;
-
 import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
 import edu.umd.cs.findbugs.annotations.NonNull;
 
@@ -40,42 +35,16 @@ import edu.umd.cs.findbugs.annotations.NonNull;
  */
 @DefaultAnnotation(NonNull.class)
 public class MongoEntityIndexer extends BaseEntityIndexer {
-    private MongoClient client;
-
-    @Override
-    public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
-        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
-                "The provided Configuration must be of type StatefulMongoDBRdfConfiguration, but was "  + conf.getClass().getName());
-
-        final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
-        if (client == null) {
-            client = mongoConf.getMongoClient();
-        }
-        final String ryaInstanceName = mongoConf.getMongoDBName();
-        return new MongoEntityStorage(client, ryaInstanceName);
-    }
-
-    @Override
-    public TypeStorage getTypeStorage(final Configuration conf) {
-        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
-                "The provided Configuration must be of type StatefulMongoDBRdfConfiguration, but was "  + conf.getClass().getName());
-
-        final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
-        if (client == null) {
-            client = mongoConf.getMongoClient();
-        }
-        final String ryaInstanceName = mongoConf.getMongoDBName();
-        return new MongoTypeStorage(client, ryaInstanceName);
-    }
-
 
     @Override
-    public void init() {
-        //nothing to init.
+    public EntityStorage getEntityStorage() throws EntityStorageException {
+    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
+        return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstance());
     }
 
     @Override
-    public void setClient(final MongoClient client) {
-        this.client = client;
+    public TypeStorage getTypeStorage() {
+    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
+        return new MongoTypeStorage(conf.getMongoClient(), conf.getRyaInstance());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 a815040..1c4c2fa 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
@@ -76,6 +76,7 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
 
     protected void initCore() {
         dbName = conf.getMongoDBName();
+        this.mongoClient = conf.getMongoClient();
         db = this.mongoClient.getDB(dbName);
         final String collectionName = conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + getCollectionName();
         collection = db.getCollection(collectionName);
@@ -92,19 +93,10 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
     }
 
     @Override
-    public void setClient(final MongoClient client){
-        this.mongoClient = client;
-    }
-
-    @Override
     public void setConf(final Configuration conf) {
         checkState(conf instanceof StatefulMongoDBRdfConfiguration,
                 "The provided Configuration must be a StatefulMongoDBRdfConfiguration, but it was " + conf.getClass().getName());
         this.conf = (StatefulMongoDBRdfConfiguration) conf;
-        if (!isInit){
-            setClient(this.conf.getMongoClient());
-            init();
-        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/freetext/MongoFreeTextIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/freetext/MongoFreeTextIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/freetext/MongoFreeTextIndexer.java
index 47f9022..913a4fd 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/freetext/MongoFreeTextIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/mongodb/freetext/MongoFreeTextIndexer.java
@@ -35,7 +35,7 @@ import info.aduna.iteration.CloseableIteration;
 public class MongoFreeTextIndexer extends AbstractMongoIndexer<TextMongoDBStorageStrategy> implements FreeTextIndexer {
     private static final String COLLECTION_SUFFIX = "freetext";
     private static final Logger logger = Logger.getLogger(MongoFreeTextIndexer.class);
-
+    
     @Override
     public void init() {
         initCore();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
index 1ec8543..ffde9b2 100644
--- a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
+++ b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
@@ -96,8 +96,6 @@ public class RyaSailFactory {
             ConfigUtils.setIndexers(mongoConfig);
 
             // Initialize the indexer and optimizer objects that will be used within the Sail object.
-            final List<MongoSecondaryIndex> indexers = mongoConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
-            // TODO Optimizers the same way. They're getting the wrong configuration somehow.
 
             // Populate the configuration using previously stored Rya Details if this instance uses them.
             try {
@@ -108,7 +106,9 @@ public class RyaSailFactory {
             }
 
             // Set the configuration to the stateful configuration that is used to pass the constructed objects around.
-            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client, indexers);
+            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client);
+            final List<MongoSecondaryIndex> indexers = statefulConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+            statefulConfig.setIndexers(indexers);
             rdfConfig = statefulConfig;
 
             // Create the DAO that is able to interact with MongoDB.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 875d54e..88b2ed0 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
@@ -33,7 +33,6 @@ 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.MongoTypeStorage;
 import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.RDF;
 import org.openrdf.model.vocabulary.XMLSchema;
@@ -46,226 +45,238 @@ import com.google.common.collect.Sets;
  */
 public class MongoEntityIndexerIT extends MongoTestBase {
 
-    private static final String RYA_INSTANCE_NAME = "testDB";
-
-    private static final Type PERSON_TYPE =
-            new Type(new RyaURI("urn:person"),
-                ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:age"))
-                    .add(new RyaURI("urn:eye"))
-                    .build());
-
-    private static final Type EMPLOYEE_TYPE =
-            new Type(new RyaURI("urn:employee"),
-                ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:hoursPerWeek"))
-                    .build());
-
-    private MongoEntityIndexer indexer;
-
-    @Before
-    public void setup() {
-        indexer = new MongoEntityIndexer();
-        indexer.setClient(getMongoClient());
-        indexer.setConf(conf);
-        indexer.init();
-    }
-
-    @Test
-    public void addStatement_setsType() throws Exception {
-        // Load a type into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-
-        // Index a RyaStatement that will create an Entity with an explicit type.
-        final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person"));
-        indexer.storeStatement(statement);
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-        final Entity expected = Entity.builder()
-                .setSubject(new RyaURI("urn:SSN/111-11-1111"))
-                .setExplicitType(new RyaURI("urn:person"))
-                .build();
-
-        assertEquals(expected, entity);
-    }
-
-    @Test
-    public void addStatement_setsProperty() throws Exception {
-        // Load the types into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-        types.create(EMPLOYEE_TYPE);
-
-        // Index a RyaStatement that will create an Entity with two implicit types.
-        final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice"));
-        indexer.storeStatement(statement);
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-        final Entity expected = Entity.builder()
-                .setSubject(new RyaURI("urn:SSN/111-11-1111"))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .build();
-
-        assertEquals(expected, entity);
-    }
-
-    @Test
-    public void addStatement_manyUpdates() throws Exception {
-        // Load the types into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-        types.create(EMPLOYEE_TYPE);
-
-        // Index a bunch of RyaStatements.
-        final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-        indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
-        indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
-        indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")));
-        indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")));
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-        final Entity expected = Entity.builder()
-                .setSubject(aliceSSN)
-                .setExplicitType(new RyaURI("urn:person"))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setVersion( entity.getVersion() )
-                .build();
-
-        assertEquals(expected, entity);
-    }
-
-    @Test
-    public void addStatements() throws Exception {
-        // Load the types into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-        types.create(EMPLOYEE_TYPE);
-
-        // Index a bunch of RyaStatements.
-        final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-        final RyaURI bobSSN = new RyaURI("urn:SSN/222-22-2222");
-
-        indexer.storeStatements(Sets.newHashSet(
-                new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")),
-
-                new RyaStatement(bobSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")),
-                new RyaStatement(bobSSN, new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")),
-                new RyaStatement(bobSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:employee"))));
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-
-        final Entity alice = entities.get(aliceSSN).get();
-        final Entity bob = entities.get(bobSSN).get();
-        final Set<Entity> storedEntities = Sets.newHashSet(alice, bob);
-
-        final Entity expectedAlice = Entity.builder()
-                .setSubject(aliceSSN)
-                .setExplicitType(new RyaURI("urn:person"))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setVersion( alice.getVersion() )
-                .build();
-        final Entity expectedBob = Entity.builder()
-                .setSubject(bobSSN)
-                .setExplicitType(new RyaURI("urn:employee"))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
-                .setVersion( bob.getVersion() )
-                .build();
-        final Set<Entity> expected = Sets.newHashSet(expectedAlice, expectedBob);
-
-        assertEquals(expected, storedEntities);
-    }
-
-    @Test
-    public void deleteStatement_deletesType() throws Exception {
-        // Load the type into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-        types.create(EMPLOYEE_TYPE);
-
-        // Index a bunch of RyaStatements.
-        final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-
-        indexer.storeStatements(Sets.newHashSet(
-                new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
-
-        // Remove the explicit type from Alice.
-        indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-        final Entity expected = Entity.builder()
-                .setSubject(aliceSSN)
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-                .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-                .setVersion( entity.getVersion() )
-                .build();
-
-        assertEquals(expected, entity);
-    }
-
-    @Test
-    public void deleteStatement_deletesProperty() throws Exception {
-        // Load the type into the TypeStorage.
-        final TypeStorage types = new MongoTypeStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        types.create(PERSON_TYPE);
-        types.create(EMPLOYEE_TYPE);
-
-        // Index a bunch of RyaStatements.
-        final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-
-        indexer.storeStatements(Sets.newHashSet(
-                new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-                new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
-
-        // Remove the name property from Alice.
-        indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
-
-        // Fetch the Entity from storage and ensure it looks correct.
-        final EntityStorage entities = new MongoEntityStorage(getMongoClient(), RYA_INSTANCE_NAME);
-        final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-        final Entity expected = Entity.builder()
-                .setSubject(aliceSSN)
-                .setExplicitType(new RyaURI("urn:person"))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-                .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-                .setVersion( entity.getVersion() )
-                .build();
-
-        assertEquals(expected, entity);
-    }
+	private static final Type PERSON_TYPE =
+			new Type(new RyaURI("urn:person"),
+					ImmutableSet.<RyaURI>builder()
+					.add(new RyaURI("urn:name"))
+					.add(new RyaURI("urn:age"))
+					.add(new RyaURI("urn:eye"))
+					.build());
+
+	private static final Type EMPLOYEE_TYPE =
+			new Type(new RyaURI("urn:employee"),
+					ImmutableSet.<RyaURI>builder()
+					.add(new RyaURI("urn:name"))
+					.add(new RyaURI("urn:hoursPerWeek"))
+					.build());
+
+	@Test
+	public void addStatement_setsType() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load a type into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+
+			// Index a RyaStatement that will create an Entity with an explicit type.
+			final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person"));
+			indexer.storeStatement(statement);
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+			final Entity expected = Entity.builder()
+					.setSubject(new RyaURI("urn:SSN/111-11-1111"))
+					.setExplicitType(new RyaURI("urn:person"))
+					.build();
+
+			assertEquals(expected, entity);
+		}
+	}
+
+	@Test
+	public void addStatement_setsProperty() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load the types into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+			types.create(EMPLOYEE_TYPE);
+
+			// Index a RyaStatement that will create an Entity with two implicit types.
+			final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice"));
+			indexer.storeStatement(statement);
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+			final Entity expected = Entity.builder()
+					.setSubject(new RyaURI("urn:SSN/111-11-1111"))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.build();
+
+			assertEquals(expected, entity);
+		}
+	}
+
+	@Test
+	public void addStatement_manyUpdates() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load the types into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+			types.create(EMPLOYEE_TYPE);
+
+			// Index a bunch of RyaStatements.
+			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
+			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
+			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")));
+			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")));
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+			final Entity expected = Entity.builder()
+					.setSubject(aliceSSN)
+					.setExplicitType(new RyaURI("urn:person"))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setVersion( entity.getVersion() )
+					.build();
+
+			assertEquals(expected, entity);
+		}
+	}
+
+	@Test
+	public void addStatements() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load the types into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+			types.create(EMPLOYEE_TYPE);
+
+			// Index a bunch of RyaStatements.
+			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+			final RyaURI bobSSN = new RyaURI("urn:SSN/222-22-2222");
+
+			indexer.storeStatements(Sets.newHashSet(
+					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")),
+
+					new RyaStatement(bobSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")),
+					new RyaStatement(bobSSN, new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")),
+					new RyaStatement(bobSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:employee"))));
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+
+			final Entity alice = entities.get(aliceSSN).get();
+			final Entity bob = entities.get(bobSSN).get();
+			final Set<Entity> storedEntities = Sets.newHashSet(alice, bob);
+
+			final Entity expectedAlice = Entity.builder()
+					.setSubject(aliceSSN)
+					.setExplicitType(new RyaURI("urn:person"))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setVersion( alice.getVersion() )
+					.build();
+			final Entity expectedBob = Entity.builder()
+					.setSubject(bobSSN)
+					.setExplicitType(new RyaURI("urn:employee"))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
+					.setVersion( bob.getVersion() )
+					.build();
+			final Set<Entity> expected = Sets.newHashSet(expectedAlice, expectedBob);
+
+			assertEquals(expected, storedEntities);
+		}
+	}
+
+	@Test
+	public void deleteStatement_deletesType() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load the type into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+			types.create(EMPLOYEE_TYPE);
+
+			// Index a bunch of RyaStatements.
+			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+
+			indexer.storeStatements(Sets.newHashSet(
+					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
+
+			// Remove the explicit type from Alice.
+			indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+			final Entity expected = Entity.builder()
+					.setSubject(aliceSSN)
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+					.setVersion( entity.getVersion() )
+					.build();
+
+			assertEquals(expected, entity);
+		}
+	}
+
+	@Test
+	public void deleteStatement_deletesProperty() throws Exception {
+		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+			// Load the type into the TypeStorage.
+			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
+			types.create(PERSON_TYPE);
+			types.create(EMPLOYEE_TYPE);
+
+			// Index a bunch of RyaStatements.
+			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+
+			indexer.storeStatements(Sets.newHashSet(
+					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
+
+			// Remove the name property from Alice.
+			indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
+
+			// Fetch the Entity from storage and ensure it looks correct.
+			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
+			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+			final Entity expected = Entity.builder()
+					.setSubject(aliceSSN)
+					.setExplicitType(new RyaURI("urn:person"))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+					.setVersion( entity.getVersion() )
+					.build();
+
+			assertEquals(expected, entity);
+		}
+	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 36e1445..c33a5bb 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
@@ -60,33 +60,6 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conf.setBoolean(ConfigUtils.USE_ENTITY, true);
     }
 
-//    private SailRepositoryConnection conn;
-//    private MongoEntityIndexer indexer;
-
-//    @Before
-//    public void setUp() throws Exception {
-//        conf.setBoolean(ConfigUtils.USE_MONGO, true);
-//        conf.setBoolean(ConfigUtils.USE_ENTITY, true);
-//
-//        final Sail sail = RyaSailFactory.getInstance(conf);
-//        conn = new SailRepository(sail).getConnection();
-//        conn.begin();
-//
-//        indexer = new MongoEntityIndexer();
-//        indexer.setConf(conf);
-//        indexer.init();
-//    }
-//
-//    @After
-//    public void tearDown() throws Exception {
-//        if (conn != null) {
-//            conn.clear();
-//        }
-//        if (indexer != null) {
-//            indexer.close();
-//        }
-//    }
-
     @Test
     public void ensureInEntityStore_Test() throws Exception {
         final Sail sail = RyaSailFactory.getInstance(conf);
@@ -94,10 +67,13 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+        	indexer.setConf(conf);
+        	indexer.init();
+
             setupTypes(indexer);
             addStatements(conn);
 
-            final EntityStorage entities = indexer.getEntityStorage(conf);
+            final EntityStorage entities = indexer.getEntityStorage();
             final RyaURI subject = new RyaURI("urn:alice");
             final Optional<Entity> alice = entities.get(subject);
             assertTrue(alice.isPresent());
@@ -113,6 +89,9 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+        	indexer.setConf(conf);
+        	indexer.init();
+        	
             setupTypes(indexer);
             addStatements(conn);
 
@@ -146,6 +125,9 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+        	indexer.setConf(conf);
+        	indexer.init();
+        	
             setupTypes(indexer);
             addStatements(conn);
             conn.commit();
@@ -177,7 +159,7 @@ public class MongoEntityIndexIT extends MongoTestBase {
     }
 
     private void setupTypes(MongoEntityIndexer indexer) throws Exception {
-        final TypeStorage typeStore = indexer.getTypeStorage(conf);
+        final TypeStorage typeStore = indexer.getTypeStorage();
         // Add some Types to the storage.
         final Type cat = new Type(new RyaURI("urn:cat"),
                 ImmutableSet.<RyaURI>builder()

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 52dd064..e62404c 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
@@ -35,7 +35,6 @@ 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.Assert;
 import org.junit.Test;
 import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.impl.URIImpl;
@@ -52,201 +51,207 @@ import org.openrdf.sail.Sail;
 
 public class MongoStatementMetadataIT extends MongoTestBase {
 
-//    private Sail sail;
-//    private SailRepository repo;
-//    private SailRepositoryConnection conn;
-//    private MongoDBRyaDAO dao;
-
-    private final String query1 = "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 ?a ?b ?c where {_:blankNode1 rdf:type owl:Annotation; ano:Source ?a; "
-            + "ano:Property <http://worksAt>; ano:Target <http://BurgerShack>; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date. "
-            + "_:blankNode2 rdf:type owl:Annotation; ano:Source ?a; "
-            + "ano:Property <http://talksTo>; ano:Target ?b; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date }";
-
-    @Override
-    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
-        final Set<RyaURI> propertySet = new HashSet<>(
-                Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-        conf.setUseStatementMetadata(true);
-        conf.setStatementMetadataProperties(propertySet);
-    }
-
-//    @Before
-//    public void init() throws Exception {
-//        final Set<RyaURI> propertySet = new HashSet<>(
-//                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();
-//        dao.setConf(conf);
-//        dao.init();
-//    }
-
-    @Test
-    public void simpleQueryWithoutBindingSet() throws Exception {
-        Sail sail = RyaSailFactory.getInstance(conf);
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-
-            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"));
-
-            final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            dao.add(statement);
-
-            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
-
-            final QueryBindingSet bs = new QueryBindingSet();
-            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-            bs.addBinding("y", new LiteralImpl("Joe"));
-
-            final List<BindingSet> bsList = new ArrayList<>();
-            while (result.hasNext()) {
-                bsList.add(result.next());
-            }
-
-            System.out.println(bsList);
-            assertEquals(1, bsList.size());
-            assertEquals(bs, bsList.get(0));
-            dao.delete(statement, conf);
-        } finally {
-//            dao.destroy();
-//            sail.shutDown();
-        }
-    }
-
-    /**
-     * 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 {
-        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"));
-
-        final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement);
-
-        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
-
-        final List<BindingSet> bsList = new ArrayList<>();
-        while (result.hasNext()) {
-            bsList.add(result.next());
-        }
-        Assert.assertEquals(0, bsList.size());
-        dao.delete(statement, conf);
-    }
-
-    @Test
-    public void simpleQueryWithBindingSet() throws Exception {
-
-        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"));
-
-        final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        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);
-
-        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
-
-        final Set<BindingSet> expected = new HashSet<>();
-        final QueryBindingSet expected1 = new QueryBindingSet();
-        expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
-        expected1.addBinding("y", new LiteralImpl("Joe"));
-        final QueryBindingSet expected2 = new QueryBindingSet();
-        expected2.addBinding("x", new LiteralImpl("HardwareStore"));
-        expected2.addBinding("y", new LiteralImpl("Joe"));
-        expected.add(expected1);
-        expected.add(expected2);
-
-        final Set<BindingSet> bsSet = new HashSet<>();
-        while (result.hasNext()) {
-            bsSet.add(result.next());
-        }
-
-        Assert.assertEquals(expected, bsSet);
-
-        dao.delete(statement1, conf);
-        dao.delete(statement2, conf);
-    }
-
-    /**
-     * Tests to see if correct result is passed back when a metadata statement
-     * is joined with a StatementPattern statement (i.e. a common variable
-     * appears in a StatementPattern statement and a metadata statement).
-     * StatementPattern statements have either rdf:subject, rdf:predicate, or
-     * rdf:object as the predicate while a metadata statement is any statement
-     * in the reified query whose predicate is not rdf:type and not a
-     * StatementPattern predicate.
-     *
-     * @throws MalformedQueryException
-     * @throws QueryEvaluationException
-     * @throws RyaDAOException
-     */
-    @Test
-    public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
-
-        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"));
-        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"));
-
-        final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-                new RyaURI("http://BurgerShack"), new RyaURI("http://context"), "", metadata1);
-        final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
-                new RyaURI("http://Betty"), new RyaURI("http://context"), "", metadata1);
-        final RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
-                new RyaURI("http://Amanda"), new RyaURI("http://context"), "", metadata1);
-        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);
-
-        final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
-
-        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);
-
-        final Set<BindingSet> bsSet = new HashSet<>();
-        while (result.hasNext()) {
-            bsSet.add(result.next());
-        }
-
-        Assert.assertEquals(expected, bsSet);
-
-        dao.delete(statement1, conf);
-        dao.delete(statement2, conf);
-        dao.delete(statement3, conf);
-        dao.delete(statement4, conf);
-    }
+	private final String query1 = "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 ?a ?b ?c where {_:blankNode1 rdf:type owl:Annotation; ano:Source ?a; "
+			+ "ano:Property <http://worksAt>; ano:Target <http://BurgerShack>; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date. "
+			+ "_:blankNode2 rdf:type owl:Annotation; ano:Source ?a; "
+			+ "ano:Property <http://talksTo>; ano:Target ?b; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date }";
+
+	@Override
+	protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+		final Set<RyaURI> propertySet = new HashSet<>(
+				Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+		conf.setUseStatementMetadata(true);
+		conf.setStatementMetadataProperties(propertySet);
+	}
+
+	@Test
+	public void simpleQueryWithoutBindingSet() throws Exception {
+		Sail sail = RyaSailFactory.getInstance(conf);
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+
+			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"));
+
+			final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement);
+
+			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+
+			final QueryBindingSet bs = new QueryBindingSet();
+			bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+			bs.addBinding("y", new LiteralImpl("Joe"));
+
+			final List<BindingSet> bsList = new ArrayList<>();
+			while (result.hasNext()) {
+				bsList.add(result.next());
+			}
+
+			assertEquals(1, bsList.size());
+			assertEquals(bs, bsList.get(0));
+			dao.delete(statement, conf);
+		} finally {
+			dao.destroy();
+			sail.shutDown();
+		}
+	}
+
+	/**
+	 * 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 {
+		Sail sail = RyaSailFactory.getInstance(conf);
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"));
+
+			final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement);
+
+			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+
+			final List<BindingSet> bsList = new ArrayList<>();
+			while (result.hasNext()) {
+				bsList.add(result.next());
+			}
+			assertEquals(0, bsList.size());
+			dao.delete(statement, conf);
+		} finally {
+			dao.destroy();
+			sail.shutDown();
+		}
+	}
+
+	@Test
+	public void simpleQueryWithBindingSet() throws Exception {
+		Sail sail = RyaSailFactory.getInstance(conf);
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"));
+
+			final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			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);
+
+			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+
+			final Set<BindingSet> expected = new HashSet<>();
+			final QueryBindingSet expected1 = new QueryBindingSet();
+			expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
+			expected1.addBinding("y", new LiteralImpl("Joe"));
+			final QueryBindingSet expected2 = new QueryBindingSet();
+			expected2.addBinding("x", new LiteralImpl("HardwareStore"));
+			expected2.addBinding("y", new LiteralImpl("Joe"));
+			expected.add(expected1);
+			expected.add(expected2);
+
+			final Set<BindingSet> bsSet = new HashSet<>();
+			while (result.hasNext()) {
+				bsSet.add(result.next());
+			}
+
+			assertEquals(expected, bsSet);
+
+			dao.delete(statement1, conf);
+			dao.delete(statement2, conf);
+		} finally {
+			dao.destroy();
+			sail.shutDown();
+		}
+	}
+
+	/**
+	 * Tests to see if correct result is passed back when a metadata statement
+	 * is joined with a StatementPattern statement (i.e. a common variable
+	 * appears in a StatementPattern statement and a metadata statement).
+	 * StatementPattern statements have either rdf:subject, rdf:predicate, or
+	 * rdf:object as the predicate while a metadata statement is any statement
+	 * in the reified query whose predicate is not rdf:type and not a
+	 * StatementPattern predicate.
+	 *
+	 * @throws MalformedQueryException
+	 * @throws QueryEvaluationException
+	 * @throws RyaDAOException
+	 */
+	@Test
+	public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
+		Sail sail = RyaSailFactory.getInstance(conf);
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"));
+			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"));
+
+			final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+					new RyaURI("http://BurgerShack"), new RyaURI("http://context"), "", metadata1);
+			final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
+					new RyaURI("http://Betty"), new RyaURI("http://context"), "", metadata1);
+			final RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
+					new RyaURI("http://Amanda"), new RyaURI("http://context"), "", metadata1);
+			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);
+
+			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
+
+			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);
+
+			final Set<BindingSet> bsSet = new HashSet<>();
+			while (result.hasNext()) {
+				bsSet.add(result.next());
+			}
+
+			assertEquals(expected, bsSet);
+
+			dao.delete(statement1, conf);
+			dao.delete(statement2, conf);
+			dao.delete(statement3, conf);
+			dao.delete(statement4, conf);
+		} finally {
+			dao.destroy();
+			sail.shutDown();
+		}
+	}
 }


[17/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
RYA-414 Code review. closes #256


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

Branch: refs/heads/master
Commit: 4576f556af04326b68d80679df233ccfb36d3382
Parents: 157c064
Author: kchilton2 <ke...@gmail.com>
Authored: Wed Jan 3 14:56:08 2018 -0500
Committer: David Lotts <da...@parsons.com>
Committed: Thu Jan 4 17:05:04 2018 -0500

----------------------------------------------------------------------
 .../AbstractMongoDBRdfConfigurationBuilder.java |    2 +-
 .../rya/mongodb/EmbeddedMongoFactory.java       |   11 +-
 .../rya/mongodb/MongoDBRdfConfiguration.java    |    4 +-
 .../StatefulMongoDBRdfConfiguration.java        |    2 +-
 .../dao/SimpleMongoDBNamespaceManager.java      |    1 -
 .../rya/mongodb/EmbeddedMongoSingleton.java     |    3 +
 .../rya/mongodb/MongoDBQueryEngineIT.java       |  185 ++
 .../rya/mongodb/MongoDBQueryEngineTest.java     |  185 --
 .../rya/mongodb/MongoDBRyaBatchWriterIT.java    |    2 +-
 .../apache/rya/mongodb/MongoDBRyaDAO2IT.java    |  138 ++
 .../org/apache/rya/mongodb/MongoDBRyaDAOIT.java |   18 +-
 .../apache/rya/mongodb/MongoDBRyaDAOTest.java   |  138 --
 .../org/apache/rya/mongodb/MongoITBase.java     |   94 +
 .../org/apache/rya/mongodb/MongoTestBase.java   |   94 -
 .../instance/MongoRyaDetailsRepositoryIT.java   |    4 +-
 .../entity/update/mongo/MongoEntityIndexer.java |    4 +-
 .../matching/RyaQueryEngineFactory.java         |    3 +-
 .../entity/query/EntityQueryNodeIT.java         |  287 +++
 .../entity/query/EntityQueryNodeTest.java       |  287 ---
 .../storage/mongo/MongoEntityStorageIT.java     |    4 +-
 .../storage/mongo/MongoTypeStorageIT.java       |    4 +-
 .../update/mongo/MongoEntityIndexerIT.java      |    4 +-
 .../rya/indexing/mongo/MongoDbSmartUriIT.java   |    4 +-
 .../rya/indexing/mongo/MongoEntityIndex2IT.java |  248 +++
 .../rya/indexing/mongo/MongoEntityIndexIT.java  |    4 +-
 .../indexing/mongo/MongoEntityIndexTest.java    |  248 ---
 .../indexing/mongo/MongoFreeTextIndexerIT.java  |  188 ++
 .../mongo/MongoFreeTextIndexerTest.java         |  188 --
 .../indexing/mongo/MongoTemporalIndexerIT.java  |  722 ++++++
 .../mongo/MongoTemporalIndexerTest.java         |  722 ------
 .../duplication/DuplicateDataDetectorIT.java    | 2053 ++++++++++++++++++
 .../duplication/DuplicateDataDetectorTest.java  | 2053 ------------------
 .../metadata/MongoStatementMetadataIT.java      |    4 +-
 .../metadata/MongoStatementMetadataNodeIT.java  |  368 ++++
 .../MongoStatementMetadataNodeTest.java         |  368 ----
 .../geotemporal/GeoTemporalIndexer.java         |    2 +-
 .../geotemporal/GeoTemporalProviderTest.java    |    4 +-
 .../geotemporal/GeoTemporalTestUtils.java       |    7 +-
 .../geotemporal/MongoGeoTemporalIndexIT.java    |    4 +-
 .../geotemporal/model/EventQueryNode2IT.java    |  364 ++++
 .../geotemporal/model/EventQueryNodeTest.java   |  364 ----
 .../GeoTemporalMongoDBStorageStrategyTest.java  |    7 +-
 .../geotemporal/mongo/MongoEventStorageIT.java  |  198 ++
 .../mongo/MongoEventStorageTest.java            |  198 --
 .../mongo/MongoGeoTemporalIndexerIT.java        |    4 +-
 .../indexing/mongo/MongoGeoIndexerFilterIT.java |    4 +-
 .../rya/indexing/mongo/MongoGeoIndexerIT.java   |  377 ++++
 .../rya/indexing/mongo/MongoGeoIndexerSfIT.java |  262 +++
 .../indexing/mongo/MongoGeoIndexerSfTest.java   |  262 ---
 .../rya/indexing/mongo/MongoGeoIndexerTest.java |  377 ----
 .../indexing/mongo/MongoIndexerDeleteIT.java    |    4 +-
 .../rya/rdftriplestore/RdfCloudTripleStore.java |    7 +-
 52 files changed, 5549 insertions(+), 5540 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
index 277ba5e..bb14a39 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
@@ -121,7 +121,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * an instance of MongoDB and will default to "rya_" is no value is
      * specified.
      *
-     * @param name - name of Collection to connect to
+     * @param prefix - name of Collection to connect to
      * @return specified builder for chaining method invocations
      */
     public B setMongoCollectionPrefix(String prefix) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
index ced8aa5..b3cfaa4 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
@@ -67,7 +67,7 @@ public class EmbeddedMongoFactory {
     }
 
     private IMongodConfig newMongodConfig(final IFeatureAwareVersion version) throws UnknownHostException, IOException {
-        Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
+        final Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
         return new MongodConfigBuilder().version(version).net(net).build();
     }
 
@@ -78,17 +78,16 @@ public class EmbeddedMongoFactory {
     }
 
     /**
-     * Creates a new Mongo connection.
-     *
-     * @throws MongoException
-     * @throws UnknownHostException
+     * @return A new Mongo client that is connected to the embedded MongoDB Server.
+     * @throws UnknownHostException The hostname used was unknown.
+     * @throws MongoException Couldn't create the client.
      */
     public MongoClient newMongoClient() throws UnknownHostException, MongoException {
         return new MongoClient(new ServerAddress(mongodProcess.getConfig().net().getServerAddress(), mongodProcess.getConfig().net().getPort()));
     }
 
     /**
-     * Gives access to the process configuration.
+     * @return The process configuration.
      *
      */
     public IMongodConfig getMongoServerDetails() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
index 269a73c..835ed27 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
@@ -97,7 +97,7 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
      * Set whether the Rya client should spin up an embedded MongoDB instance and connect to that
      * or if it should connect to a MongoDB Server that is running somewhere.
      *
-     * @param useMock - {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
+     * @param useMock - {@code true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
      */
     public void setUseMock(final boolean useMock) {
         this.setBoolean(USE_MOCK_MONGO, useMock);
@@ -107,7 +107,7 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
      * Indicates whether the Rya client should spin up an embedded MongoDB instance and connect to that
      * or if it should connect to a MongoDB Server that is running somewhere.
      *
-     * @return {@true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
+     * @return {@code true} to use an embedded Mongo DB instance; {@code false} to connect to a real server.
      */
     public boolean getUseMock() {
         return getBoolean(USE_MOCK_MONGO, false);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
index 61c349e..e334194 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
@@ -33,7 +33,7 @@ import edu.umd.cs.findbugs.annotations.NonNull;
 /**
  * A {@link MongoDBRdfConfiguration} that is used to hold onto state that is pass into Rya components that accept
  * {@link Configuration} objects.
- * </p>
+ * <p>
  * HACK:
  * This class is part of a hack to get around how Rya uses reflection to initialize indexers, optimizers, etc.
  * Those classes have empty constructors, so they are not able to receive Mongo specific components at construction

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
index 91ee064..2be0785 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
@@ -142,7 +142,6 @@ public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
             final MessageDigest digest = MessageDigest.getInstance("SHA-1");
             bytes = digest.digest(bytes);
         } catch (final NoSuchAlgorithmException e) {
-            // TODO Auto-generated catch block
             e.printStackTrace();
         }
         final BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
index a481ccf..235d0c5 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/EmbeddedMongoSingleton.java
@@ -54,6 +54,9 @@ public class EmbeddedMongoSingleton {
         return client;
     }
 
+    /**
+     * @return The singleton Mongo DB instance's server details.
+     */
     public static IMongodConfig getMongodConfig() {
         return InstanceHolder.SINGLETON.mongodConfig;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineIT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineIT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineIT.java
new file mode 100644
index 0000000..3a216ec
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineIT.java
@@ -0,0 +1,185 @@
+/*
+* * 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 static org.junit.Assert.assertEquals;
+
+/*
+ * 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.Collection;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.rya.api.RdfCloudTripleStoreUtils;
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
+import org.apache.rya.api.domain.RyaURI;
+import org.junit.Test;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.google.common.collect.Lists;
+
+import info.aduna.iteration.CloseableIteration;
+/**
+ * Integration tests the methods of {@link MongoDBQueryEngine}.
+ */
+public class MongoDBQueryEngineIT extends MongoITBase {
+
+    private RyaStatement getStatement(final String s, final String p, final String o) {
+        final RyaStatementBuilder builder = new RyaStatementBuilder();
+        if (s != null) {
+            builder.setSubject(new RyaURI(s));
+        }
+        if (p != null) {
+            builder.setPredicate(new RyaURI(p));
+        }
+        if (o != null) {
+            builder.setObject(new RyaURI(o));
+        }
+        return builder.build();
+    }
+
+    public int size(final CloseableIteration<?, ?> iter) throws Exception {
+        int i = 0;
+        while (iter.hasNext()) {
+            i++;
+            iter.next();
+        }
+        return i;
+    }
+
+    @Test
+    public void statementQuery() throws Exception {
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
+
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
+
+            final RyaStatement s = getStatement("u:a", null, null);
+            assertEquals(2, size(engine.query(s, conf)));
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void batchbindingSetsQuery() throws Exception {
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
+
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
+
+            // Run the test.
+            final RyaStatement s1 = getStatement(null, null, "u:b");
+
+            final MapBindingSet bs1 = new MapBindingSet();
+            bs1.addBinding("foo", new URIImpl("u:x"));
+
+            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs1);
+            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
+            assertEquals(1, size(engine.queryWithBindingSet(stmts1, conf)));
+
+
+            final MapBindingSet bs2 = new MapBindingSet();
+            bs2.addBinding("foo", new URIImpl("u:y"));
+
+            final RyaStatement s2 = getStatement(null, null, "u:c");
+
+            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs2);
+
+            final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
+            assertEquals(2, size(engine.queryWithBindingSet(stmts2, conf)));
+
+
+            final Map.Entry<RyaStatement, BindingSet> e3 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs1);
+            final Map.Entry<RyaStatement, BindingSet> e4 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs2);
+
+            final Collection<Entry<RyaStatement, BindingSet>> stmts3 = Lists.newArrayList(e1, e2, e3, e4);
+            assertEquals(4, size(engine.queryWithBindingSet(stmts3, conf)));
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void bindingSetsQuery() throws Exception {
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
+            engine.setConf(conf);
+
+            // Add data.
+            dao.setConf(conf);
+            dao.init();
+            dao.add(getStatement("u:a", "u:tt", "u:b"));
+            dao.add(getStatement("u:a", "u:tt", "u:c"));
+
+            // Run the test.
+            final RyaStatement s = getStatement("u:a", null, null);
+
+            final MapBindingSet bs1 = new MapBindingSet();
+            bs1.addBinding("foo", new URIImpl("u:x"));
+
+            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs1);
+            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
+            assertEquals(2, size(engine.queryWithBindingSet(stmts1, conf)));
+
+
+            final MapBindingSet bs2 = new MapBindingSet();
+            bs2.addBinding("foo", new URIImpl("u:y"));
+
+            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs2);
+
+            final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
+            assertEquals(4, size(engine.queryWithBindingSet(stmts2, conf)));
+        } finally {
+            dao.destroy();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index f7235a6..0000000
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBQueryEngineTest.java
+++ /dev/null
@@ -1,185 +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.mongodb;
-
-import static org.junit.Assert.assertEquals;
-
-/*
- * 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.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.rya.api.RdfCloudTripleStoreUtils;
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
-import org.apache.rya.api.domain.RyaURI;
-import org.junit.Test;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.google.common.collect.Lists;
-
-import info.aduna.iteration.CloseableIteration;
-/**
- * Integration tests the methods of {@link MongoDBQueryEngine}.
- */
-public class MongoDBQueryEngineTest extends MongoTestBase {
-
-    private RyaStatement getStatement(final String s, final String p, final String o) {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        if (s != null) {
-            builder.setSubject(new RyaURI(s));
-        }
-        if (p != null) {
-            builder.setPredicate(new RyaURI(p));
-        }
-        if (o != null) {
-            builder.setObject(new RyaURI(o));
-        }
-        return builder.build();
-    }
-
-    public int size(final CloseableIteration<?, ?> iter) throws Exception {
-        int i = 0;
-        while (iter.hasNext()) {
-            i++;
-            iter.next();
-        }
-        return i;
-    }
-
-    @Test
-    public void statementQuery() throws Exception {
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
-            engine.setConf(conf);
-
-            // Add data.
-            dao.setConf(conf);
-            dao.init();
-            dao.add(getStatement("u:a", "u:tt", "u:b"));
-            dao.add(getStatement("u:a", "u:tt", "u:c"));
-
-            final RyaStatement s = getStatement("u:a", null, null);
-            assertEquals(2, size(engine.query(s, conf)));
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void batchbindingSetsQuery() throws Exception {
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
-            engine.setConf(conf);
-
-            // Add data.
-            dao.setConf(conf);
-            dao.init();
-            dao.add(getStatement("u:a", "u:tt", "u:b"));
-            dao.add(getStatement("u:a", "u:tt", "u:c"));
-
-            // Run the test.
-            final RyaStatement s1 = getStatement(null, null, "u:b");
-
-            final MapBindingSet bs1 = new MapBindingSet();
-            bs1.addBinding("foo", new URIImpl("u:x"));
-
-            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs1);
-            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
-            assertEquals(1, size(engine.queryWithBindingSet(stmts1, conf)));
-
-
-            final MapBindingSet bs2 = new MapBindingSet();
-            bs2.addBinding("foo", new URIImpl("u:y"));
-
-            final RyaStatement s2 = getStatement(null, null, "u:c");
-
-            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs2);
-
-            final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
-            assertEquals(2, size(engine.queryWithBindingSet(stmts2, conf)));
-
-
-            final Map.Entry<RyaStatement, BindingSet> e3 = new RdfCloudTripleStoreUtils.CustomEntry<>(s2, bs1);
-            final Map.Entry<RyaStatement, BindingSet> e4 = new RdfCloudTripleStoreUtils.CustomEntry<>(s1, bs2);
-
-            final Collection<Entry<RyaStatement, BindingSet>> stmts3 = Lists.newArrayList(e1, e2, e3, e4);
-            assertEquals(4, size(engine.queryWithBindingSet(stmts3, conf)));
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void bindingSetsQuery() throws Exception {
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try(final MongoDBQueryEngine engine =new MongoDBQueryEngine()) {
-            engine.setConf(conf);
-
-            // Add data.
-            dao.setConf(conf);
-            dao.init();
-            dao.add(getStatement("u:a", "u:tt", "u:b"));
-            dao.add(getStatement("u:a", "u:tt", "u:c"));
-
-            // Run the test.
-            final RyaStatement s = getStatement("u:a", null, null);
-
-            final MapBindingSet bs1 = new MapBindingSet();
-            bs1.addBinding("foo", new URIImpl("u:x"));
-
-            final Map.Entry<RyaStatement, BindingSet> e1 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs1);
-            final Collection<Entry<RyaStatement, BindingSet>> stmts1 = Lists.newArrayList(e1);
-            assertEquals(2, size(engine.queryWithBindingSet(stmts1, conf)));
-
-
-            final MapBindingSet bs2 = new MapBindingSet();
-            bs2.addBinding("foo", new URIImpl("u:y"));
-
-            final Map.Entry<RyaStatement, BindingSet> e2 = new RdfCloudTripleStoreUtils.CustomEntry<>(s, bs2);
-
-            final Collection<Entry<RyaStatement, BindingSet>> stmts2 = Lists.newArrayList(e1, e2);
-            assertEquals(4, size(engine.queryWithBindingSet(stmts2, conf)));
-        } finally {
-            dao.destroy();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
index afd7308..d024cc4 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaBatchWriterIT.java
@@ -44,7 +44,7 @@ import com.mongodb.DBObject;
 /**
  * Integration tests for the {@link MongoDbBatchWriter}.
  */
-public class MongoDBRyaBatchWriterIT extends MongoTestBase {
+public class MongoDBRyaBatchWriterIT extends MongoITBase {
 
     private static void setupLogging() {
         BasicConfigurator.configure();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAO2IT.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAO2IT.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAO2IT.java
new file mode 100644
index 0000000..f2d24ad
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAO2IT.java
@@ -0,0 +1,138 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.persist.RyaDAOException;
+import org.bson.Document;
+import org.junit.Test;
+
+import com.mongodb.MongoException;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.MongoDatabase;
+
+/**
+ * Integration tests the methods of {@link MongoDBRyaDAO}.
+ */
+public class MongoDBRyaDAO2IT extends MongoITBase {
+
+    @Test
+    public void testDeleteWildcard() throws RyaDAOException {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            dao.delete(builder.build(), conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+
+    @Test
+    public void testAdd() throws RyaDAOException, MongoException, IOException {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(builder.build());
+
+            assertEquals(coll.count(),1);
+        }  finally {
+            dao.destroy();
+        }
+    }
+
+    @Test
+    public void testDelete() throws RyaDAOException, MongoException, IOException {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            final RyaStatement statement = builder.build();
+
+            final MongoDatabase db = conf.getMongoClient().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, conf);
+            assertEquals(coll.count(),0);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    @Test
+    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            builder.setContext(new RyaURI("http://context.com"));
+            final RyaStatement statement = builder.build();
+
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(statement);
+            assertEquals(coll.count(),1);
+
+            final RyaStatementBuilder builder2 = new RyaStatementBuilder();
+            builder2.setPredicate(new RyaURI("http://temp.com"));
+            builder2.setObject(new RyaURI("http://object.com"));
+            builder2.setContext(new RyaURI("http://context3.com"));
+            final RyaStatement query = builder2.build();
+
+            dao.delete(query, conf);
+            assertEquals(coll.count(),1);
+        } finally {
+            dao.destroy();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 7043870..a05393c 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
@@ -42,7 +42,7 @@ import com.mongodb.MongoException;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
 
-public class MongoDBRyaDAOIT extends MongoTestBase {
+public class MongoDBRyaDAOIT extends MongoITBase {
 
     @Override
     protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
@@ -51,7 +51,7 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
 
     @Test
     public void testDeleteWildcard() throws RyaDAOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
         try {
             dao.setConf(conf);
             dao.init();
@@ -67,7 +67,7 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
 
     @Test
     public void testAdd() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
         try {
             dao.setConf(conf);
             dao.init();
@@ -95,7 +95,7 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
 
     @Test
     public void testDelete() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
         try {
             dao.setConf(conf);
             dao.init();
@@ -121,7 +121,7 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
 
     @Test
     public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
         try {
             dao.setConf(conf);
             dao.init();
@@ -155,7 +155,7 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
 
     @Test
     public void testVisibility() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
         try {
             dao.setConf(conf);
             dao.init();
@@ -519,9 +519,11 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
     /**
      * Generates a test statement with the provided document visibility to
      * determine if the specified user authorization can view the statement.
-     * @param documentVisibility the document visibility boolean expression
+     *
+     * @param dao - The DAO that will be used by the test.
+     * @param documentVisibility - The document visibility boolean expression
      * string.
-     * @param userAuthorizations the user authorization strings.
+     * @param userAuthorizations - The user authorization strings.
      * @return {@code true} if provided authorization could access the document
      * in the collection. {@code false} otherwise.
      * @throws RyaDAOException

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 200be9a..0000000
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java
+++ /dev/null
@@ -1,138 +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.mongodb;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.api.persist.RyaDAOException;
-import org.bson.Document;
-import org.junit.Test;
-
-import com.mongodb.MongoException;
-import com.mongodb.client.MongoCollection;
-import com.mongodb.client.MongoDatabase;
-
-/**
- * Integration tests the methods of {@link MongoDBRyaDAO}.
- */
-public class MongoDBRyaDAOTest extends MongoTestBase {
-
-    @Test
-    public void testDeleteWildcard() throws RyaDAOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-
-            final RyaStatementBuilder builder = new RyaStatementBuilder();
-            builder.setPredicate(new RyaURI("http://temp.com"));
-            dao.delete(builder.build(), conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-
-    @Test
-    public void testAdd() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-
-            final RyaStatementBuilder builder = new RyaStatementBuilder();
-            builder.setPredicate(new RyaURI("http://temp.com"));
-            builder.setSubject(new RyaURI("http://subject.com"));
-            builder.setObject(new RyaURI("http://object.com"));
-
-            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
-
-            dao.add(builder.build());
-
-            assertEquals(coll.count(),1);
-        }  finally {
-            dao.destroy();
-        }
-    }
-
-    @Test
-    public void testDelete() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-
-            final RyaStatementBuilder builder = new RyaStatementBuilder();
-            builder.setPredicate(new RyaURI("http://temp.com"));
-            builder.setSubject(new RyaURI("http://subject.com"));
-            builder.setObject(new RyaURI("http://object.com"));
-            final RyaStatement statement = builder.build();
-
-            final MongoDatabase db = conf.getMongoClient().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, conf);
-            assertEquals(coll.count(),0);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    @Test
-    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-
-            final RyaStatementBuilder builder = new RyaStatementBuilder();
-            builder.setPredicate(new RyaURI("http://temp.com"));
-            builder.setSubject(new RyaURI("http://subject.com"));
-            builder.setObject(new RyaURI("http://object.com"));
-            builder.setContext(new RyaURI("http://context.com"));
-            final RyaStatement statement = builder.build();
-
-            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
-
-            dao.add(statement);
-            assertEquals(coll.count(),1);
-
-            final RyaStatementBuilder builder2 = new RyaStatementBuilder();
-            builder2.setPredicate(new RyaURI("http://temp.com"));
-            builder2.setObject(new RyaURI("http://object.com"));
-            builder2.setContext(new RyaURI("http://context3.com"));
-            final RyaStatement query = builder2.build();
-
-            dao.delete(query, conf);
-            assertEquals(coll.count(),1);
-        } finally {
-            dao.destroy();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoITBase.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoITBase.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoITBase.java
new file mode 100644
index 0000000..1aaf6c2
--- /dev/null
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoITBase.java
@@ -0,0 +1,94 @@
+/**
+ * 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.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.bson.Document;
+import org.junit.Before;
+
+import com.mongodb.DBCollection;
+import com.mongodb.MongoClient;
+import com.mongodb.client.MongoCollection;
+
+/**
+ * A base class that may be used when implementing Mongo DB tests that use the
+ * JUnit framework.
+ */
+public class MongoITBase {
+
+    private MongoClient mongoClient = null;
+    protected StatefulMongoDBRdfConfiguration conf;
+
+    @Before
+    public void setupTest() throws Exception {
+        // Setup the configuration that will be used within the test.
+        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
+        conf.setBoolean("sc.useMongo", true);
+        conf.setTablePrefix("test_");
+        conf.setMongoDBName("testDB");
+        conf.setMongoHostname(EmbeddedMongoSingleton.getMongodConfig().net().getServerAddress().getHostAddress());
+        conf.setMongoPort(Integer.toString(EmbeddedMongoSingleton.getMongodConfig().net().getPort()));
+
+        // Let tests update the configuration.
+        updateConfiguration(conf);
+
+        // Create the stateful configuration object.
+        mongoClient = EmbeddedMongoSingleton.getNewMongoClient();
+        final List<MongoSecondaryIndex> indexers = conf.getInstances("ac.additional.indexers", MongoSecondaryIndex.class);
+        this.conf = new StatefulMongoDBRdfConfiguration(conf, mongoClient, indexers);
+
+        // Remove any DBs that were created by previous tests.
+        for(final String dbName : mongoClient.listDatabaseNames()) {
+            mongoClient.dropDatabase(dbName);
+        }
+    }
+
+    /**
+     * Override this method if you would like to augment the configuration object that
+     * will be used to initialize indexers and create the mongo client prior to running a test.
+     *
+     * @param conf - The configuration object that may be updated. (not null)
+     */
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        // By default, do nothing.
+    }
+
+    /**
+     * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
+     */
+    public MongoClient getMongoClient() {
+        return mongoClient;
+    }
+
+    /**
+     * @return The Rya triples {@link MongoCollection}.
+     */
+    public MongoCollection<Document> getRyaCollection() {
+        return mongoClient.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName());
+    }
+
+    /**
+     * @return The Rya triples {@link DBCollection}.
+     */
+    public DBCollection getRyaDbCollection() {
+        return mongoClient.getDB(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 6dbd19b..0000000
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoTestBase.java
+++ /dev/null
@@ -1,94 +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.mongodb;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.bson.Document;
-import org.junit.Before;
-
-import com.mongodb.DBCollection;
-import com.mongodb.MongoClient;
-import com.mongodb.client.MongoCollection;
-
-/**
- * A base class that may be used when implementing Mongo DB tests that use the
- * JUnit framework.
- */
-public class MongoTestBase {
-
-    private MongoClient mongoClient = null;
-    protected StatefulMongoDBRdfConfiguration conf;
-
-    @Before
-    public void setupTest() throws Exception {
-        // Setup the configuration that will be used within the test.
-        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
-        conf.setBoolean("sc.useMongo", true);
-        conf.setTablePrefix("test_");
-        conf.setMongoDBName("testDB");
-        conf.setMongoHostname(EmbeddedMongoSingleton.getMongodConfig().net().getServerAddress().getHostAddress());
-        conf.setMongoPort(Integer.toString(EmbeddedMongoSingleton.getMongodConfig().net().getPort()));
-
-        // Let tests update the configuration.
-        updateConfiguration(conf);
-
-        // Create the stateful configuration object.
-        mongoClient = EmbeddedMongoSingleton.getNewMongoClient();
-        final List<MongoSecondaryIndex> indexers = conf.getInstances("ac.additional.indexers", MongoSecondaryIndex.class);
-        this.conf = new StatefulMongoDBRdfConfiguration(conf, mongoClient, indexers);
-
-        // Remove any DBs that were created by previous tests.
-        for(final String dbName : mongoClient.listDatabaseNames()) {
-            mongoClient.dropDatabase(dbName);
-        }
-    }
-
-    /**
-     * Override this method if you would like to augment the configuration object that
-     * will be used to initialize indexers and create the mongo client prior to running a test.
-     *
-     * @param conf - The configuration object that may be updated. (not null)
-     */
-    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
-        // By default, do nothing.
-    }
-
-    /**
-     * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
-     */
-    public MongoClient getMongoClient() {
-        return mongoClient;
-    }
-
-    /**
-     * @return The Rya triples {@link MongoCollection}.
-     */
-    public MongoCollection<Document> getRyaCollection() {
-        return mongoClient.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName());
-    }
-
-    /**
-     * @return The Rya triples {@link DBCollection}.
-     */
-    public DBCollection getRyaDbCollection() {
-        return mongoClient.getDB(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 cb276cb..dd5088e 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
@@ -40,7 +40,7 @@ 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.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -50,7 +50,7 @@ import com.mongodb.MongoClient;
 /**
  * Tests the methods of {@link AccumuloRyaDetailsRepository} by using a {@link MiniAccumuloCluster}.
  */
-public class MongoRyaDetailsRepositoryIT extends MongoTestBase {
+public class MongoRyaDetailsRepositoryIT extends MongoITBase {
     private MongoClient client;
 
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
index a850557..1507e98 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
@@ -38,13 +38,13 @@ public class MongoEntityIndexer extends BaseEntityIndexer {
 
     @Override
     public EntityStorage getEntityStorage() throws EntityStorageException {
-    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
+        final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
         return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstanceName());
     }
 
     @Override
     public TypeStorage getTypeStorage() {
-    	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
+        final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
         return new MongoTypeStorage(conf.getMongoClient(), conf.getRyaInstanceName());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java b/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
index 1025537..eb35aac 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
@@ -29,6 +29,7 @@ import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.query.AccumuloRyaQueryEngine;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.persist.query.RyaQueryEngine;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.mongodb.MongoDBQueryEngine;
 import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
@@ -60,7 +61,7 @@ public class RyaQueryEngineFactory {
                 throw new RuntimeException(e);
             }
             return (RyaQueryEngine<C>) new AccumuloRyaQueryEngine(conn, aConf);
-        } else if(conf instanceof StatefulMongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
+        } else if(conf instanceof StatefulMongoDBRdfConfiguration && ConfigUtils.getUseMongo(conf)) {
             StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration)conf;
             MongoDBQueryEngine mongoQueryEngine = new MongoDBQueryEngine();
             mongoQueryEngine.setConf(mongoConf);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeIT.java
new file mode 100644
index 0000000..66b79ef
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeIT.java
@@ -0,0 +1,287 @@
+/**
+ * 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.query;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+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.storage.EntityStorage;
+import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+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.helpers.StatementPatternCollector;
+import org.openrdf.query.impl.MapBindingSet;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import com.google.common.collect.ImmutableSet;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Integration tests the methods of {@link EntityQueryNode}.
+ */
+public class EntityQueryNodeIT extends MongoITBase {
+
+    private static final Type PERSON_TYPE =
+            new Type(new RyaURI("urn:person"),
+                ImmutableSet.<RyaURI>builder()
+                    .add(new RyaURI("urn:name"))
+                    .add(new RyaURI("urn:age"))
+                    .add(new RyaURI("urn:eye"))
+                    .build());
+
+    private static final Type EMPLOYEE_TYPE =
+            new Type(new RyaURI("urn:employee"),
+                ImmutableSet.<RyaURI>builder()
+                    .add(new RyaURI("urn:name"))
+                    .add(new RyaURI("urn:hoursPerWeek"))
+                    .build());
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_differentSubjects() throws Exception {
+        // A pattern that has two different subjects.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                    "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                    "<urn:SSN:222-22-2222> <urn:age> ?age . " +
+                    "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
+                    "<urn:SSN:222-22-2222> <urn:name> ?name . " +
+                "}");
+
+
+        // This will fail.
+        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_variablePredicate() throws Exception {
+        // A pattern that has a variable for its predicate.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
+                    "?subject ?variableProperty ?value . " +
+                    "?subject <urn:eye> ?eye . " +
+                    "?subject <urn:name> ?name . " +
+                "}");
+
+
+        // This will fail.
+        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_predicateNotPartOfType() throws Exception {
+        // A pattern that does uses a predicate that is not part of the type.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
+                    "?subject <urn:age> ?age . " +
+                    "?subject <urn:eye> ?eye . " +
+                    "?subject <urn:name> ?name . " +
+                    "?subject <urn:notPartOfType> ?value . " +
+                "}");
+
+        // This will fail.
+        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_typeMissing() throws Exception {
+        // A pattern that does uses a predicate that is not part of the type.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "?subject <urn:age> ?age . " +
+                    "?subject <urn:eye> ?eye . " +
+                    "?subject <urn:name> ?name . " +
+                "}");
+
+        // This will fail.
+        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_wrongType() throws Exception {
+        // A pattern that does uses a predicate that is not part of the type.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
+                    "?subject <urn:age> ?age . " +
+                    "?subject <urn:eye> ?eye . " +
+                    "?subject <urn:name> ?name . " +
+                "}");
+
+        // This will fail.
+        new EntityQueryNode(EMPLOYEE_TYPE, patterns, mock(EntityStorage.class));
+    }
+
+    @Test
+    public void evaluate_constantSubject() throws Exception {
+        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()
+            .setSubject(subject)
+            .setExplicitType(PERSON_TYPE.getId())
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
+            .build();
+
+        storage.create(entity);
+        // A set of patterns that match a sepecific Entity subject.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                    "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                "}");
+
+        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("age", vf.createLiteral("20"));
+        expected.addBinding("eye", vf.createLiteral("blue"));
+        expected.addBinding("name", vf.createLiteral("Bob"));
+        while(rez.hasNext()) {
+            assertEquals(expected, rez.next());
+            break;
+        }
+    }
+
+    @Test
+    public void evaluate_variableSubject() throws Exception {
+        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()
+                .setSubject(subject)
+                .setExplicitType(PERSON_TYPE.getId())
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
+                .build();
+
+        subject = new RyaURI("urn:SSN:222-22-2222");
+        final Entity fred = Entity.builder()
+                .setSubject(subject)
+                .setExplicitType(PERSON_TYPE.getId())
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(25))))
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("brown"))))
+                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Fred"))))
+                .build();
+
+        storage.create(bob);
+        storage.create(fred);
+        // A set of patterns that match a sepecific Entity subject.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "?ssn <" + RDF.TYPE + "> <urn:person> ."+
+                    "?ssn <urn:age> ?age . " +
+                    "?ssn <urn:eye> ?eye . " +
+                    "?ssn <urn:name> ?name . " +
+                "}");
+
+        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
+        final List<BindingSet> expectedBindings = new ArrayList<>();
+        final MapBindingSet expectedBob = new MapBindingSet();
+        expectedBob.addBinding("age", vf.createLiteral("20"));
+        expectedBob.addBinding("eye", vf.createLiteral("blue"));
+        expectedBob.addBinding("name", vf.createLiteral("Bob"));
+
+        final MapBindingSet expectedFred = new MapBindingSet();
+        expectedFred.addBinding("age", vf.createLiteral("25"));
+        expectedFred.addBinding("eye", vf.createLiteral("brown"));
+        expectedFred.addBinding("name", vf.createLiteral("Fred"));
+        expectedBindings.add(expectedBob);
+        expectedBindings.add(expectedFred);
+        while(rez.hasNext()) {
+            final BindingSet bs = rez.next();
+            assertTrue(expectedBindings.contains(bs));
+        }
+    }
+
+    @Test
+    public void evaluate_constantObject() throws Exception {
+        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()
+            .setSubject(subject)
+            .setExplicitType(PERSON_TYPE.getId())
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
+            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
+            .build();
+
+        storage.create(entity);
+        // A set of patterns that match a sepecific Entity subject.
+        final List<StatementPattern> patterns = getSPs(
+                "SELECT * WHERE { " +
+                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                    "<urn:SSN:111-11-1111> <urn:eye> \"blue\" . " +
+                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                "}");
+
+        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("age", vf.createLiteral("20"));
+        expected.addBinding("-const-blue", vf.createLiteral("blue"));
+        expected.addBinding("name", vf.createLiteral("Bob"));
+        while(rez.hasNext()) {
+            assertEquals(expected, rez.next());
+            break;
+        }
+    }
+
+    /**
+     * TODO doc
+     *
+     * @param sparql
+     * @return
+     * @throws MalformedQueryException
+     */
+    private static List<StatementPattern> getSPs(final String sparql) throws MalformedQueryException {
+        final StatementPatternCollector spCollector = new StatementPatternCollector();
+        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
+        return spCollector.getStatementPatterns();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 874a40e..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
+++ /dev/null
@@ -1,287 +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.query;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.api.resolver.RdfToRyaConversions;
-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.storage.EntityStorage;
-import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Test;
-import org.openrdf.model.ValueFactory;
-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.helpers.StatementPatternCollector;
-import org.openrdf.query.impl.MapBindingSet;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import com.google.common.collect.ImmutableSet;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Unit tests the methods of {@link EntityQueryNode}.
- */
-public class EntityQueryNodeTest extends MongoTestBase {
-
-    private static final Type PERSON_TYPE =
-            new Type(new RyaURI("urn:person"),
-                ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:age"))
-                    .add(new RyaURI("urn:eye"))
-                    .build());
-
-    private static final Type EMPLOYEE_TYPE =
-            new Type(new RyaURI("urn:employee"),
-                ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:hoursPerWeek"))
-                    .build());
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_differentSubjects() throws Exception {
-        // A pattern that has two different subjects.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                    "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                    "<urn:SSN:222-22-2222> <urn:age> ?age . " +
-                    "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
-                    "<urn:SSN:222-22-2222> <urn:name> ?name . " +
-                "}");
-
-
-        // This will fail.
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_variablePredicate() throws Exception {
-        // A pattern that has a variable for its predicate.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
-                    "?subject ?variableProperty ?value . " +
-                    "?subject <urn:eye> ?eye . " +
-                    "?subject <urn:name> ?name . " +
-                "}");
-
-
-        // This will fail.
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_predicateNotPartOfType() throws Exception {
-        // A pattern that does uses a predicate that is not part of the type.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
-                    "?subject <urn:age> ?age . " +
-                    "?subject <urn:eye> ?eye . " +
-                    "?subject <urn:name> ?name . " +
-                    "?subject <urn:notPartOfType> ?value . " +
-                "}");
-
-        // This will fail.
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_typeMissing() throws Exception {
-        // A pattern that does uses a predicate that is not part of the type.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "?subject <urn:age> ?age . " +
-                    "?subject <urn:eye> ?eye . " +
-                    "?subject <urn:name> ?name . " +
-                "}");
-
-        // This will fail.
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_wrongType() throws Exception {
-        // A pattern that does uses a predicate that is not part of the type.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
-                    "?subject <urn:age> ?age . " +
-                    "?subject <urn:eye> ?eye . " +
-                    "?subject <urn:name> ?name . " +
-                "}");
-
-        // This will fail.
-        new EntityQueryNode(EMPLOYEE_TYPE, patterns, mock(EntityStorage.class));
-    }
-
-    @Test
-    public void evaluate_constantSubject() throws Exception {
-        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()
-            .setSubject(subject)
-            .setExplicitType(PERSON_TYPE.getId())
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
-            .build();
-
-        storage.create(entity);
-        // A set of patterns that match a sepecific Entity subject.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                    "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                "}");
-
-        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("age", vf.createLiteral("20"));
-        expected.addBinding("eye", vf.createLiteral("blue"));
-        expected.addBinding("name", vf.createLiteral("Bob"));
-        while(rez.hasNext()) {
-            assertEquals(expected, rez.next());
-            break;
-        }
-    }
-
-    @Test
-    public void evaluate_variableSubject() throws Exception {
-        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()
-                .setSubject(subject)
-                .setExplicitType(PERSON_TYPE.getId())
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
-                .build();
-
-        subject = new RyaURI("urn:SSN:222-22-2222");
-        final Entity fred = Entity.builder()
-                .setSubject(subject)
-                .setExplicitType(PERSON_TYPE.getId())
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(25))))
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("brown"))))
-                .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Fred"))))
-                .build();
-
-        storage.create(bob);
-        storage.create(fred);
-        // A set of patterns that match a sepecific Entity subject.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "?ssn <" + RDF.TYPE + "> <urn:person> ."+
-                    "?ssn <urn:age> ?age . " +
-                    "?ssn <urn:eye> ?eye . " +
-                    "?ssn <urn:name> ?name . " +
-                "}");
-
-        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
-        final List<BindingSet> expectedBindings = new ArrayList<>();
-        final MapBindingSet expectedBob = new MapBindingSet();
-        expectedBob.addBinding("age", vf.createLiteral("20"));
-        expectedBob.addBinding("eye", vf.createLiteral("blue"));
-        expectedBob.addBinding("name", vf.createLiteral("Bob"));
-
-        final MapBindingSet expectedFred = new MapBindingSet();
-        expectedFred.addBinding("age", vf.createLiteral("25"));
-        expectedFred.addBinding("eye", vf.createLiteral("brown"));
-        expectedFred.addBinding("name", vf.createLiteral("Fred"));
-        expectedBindings.add(expectedBob);
-        expectedBindings.add(expectedFred);
-        while(rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            assertTrue(expectedBindings.contains(bs));
-        }
-    }
-
-    @Test
-    public void evaluate_constantObject() throws Exception {
-        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()
-            .setSubject(subject)
-            .setExplicitType(PERSON_TYPE.getId())
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:age"), RdfToRyaConversions.convertLiteral(vf.createLiteral(20))))
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:eye"), RdfToRyaConversions.convertLiteral(vf.createLiteral("blue"))))
-            .setProperty(PERSON_TYPE.getId(), new Property(new RyaURI("urn:name"), RdfToRyaConversions.convertLiteral(vf.createLiteral("Bob"))))
-            .build();
-
-        storage.create(entity);
-        // A set of patterns that match a sepecific Entity subject.
-        final List<StatementPattern> patterns = getSPs(
-                "SELECT * WHERE { " +
-                    "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                    "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                    "<urn:SSN:111-11-1111> <urn:eye> \"blue\" . " +
-                    "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                "}");
-
-        final EntityQueryNode node = new EntityQueryNode(PERSON_TYPE, patterns, storage);
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("age", vf.createLiteral("20"));
-        expected.addBinding("-const-blue", vf.createLiteral("blue"));
-        expected.addBinding("name", vf.createLiteral("Bob"));
-        while(rez.hasNext()) {
-            assertEquals(expected, rez.next());
-            break;
-        }
-    }
-
-    /**
-     * TODO doc
-     *
-     * @param sparql
-     * @return
-     * @throws MalformedQueryException
-     */
-    private static List<StatementPattern> getSPs(final String sparql) throws MalformedQueryException {
-        final StatementPatternCollector spCollector = new StatementPatternCollector();
-        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
-        return spCollector.getStatementPatterns();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 d8856fe..8e33d37 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,7 +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.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.XMLSchema;
 
@@ -46,7 +46,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoEntityStorage}.
  */
-public class MongoEntityStorageIT extends MongoTestBase {
+public class MongoEntityStorageIT extends MongoITBase {
 
     private static final String RYA_INSTANCE_NAME = "testInstance";
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 cff4f26..8212dbf 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,7 +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.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
@@ -39,7 +39,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoTypeStorage}.
  */
-public class MongoTypeStorageIT extends MongoTestBase {
+public class MongoTypeStorageIT extends MongoITBase {
 
     private static final String RYA_INSTANCE_NAME = "testInstance";
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 8b62a57..28fd330 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
@@ -32,7 +32,7 @@ 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.MongoTypeStorage;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.RDF;
 import org.openrdf.model.vocabulary.XMLSchema;
@@ -43,7 +43,7 @@ import com.google.common.collect.Sets;
 /**
  * Integration tests the methods of {@link MongoEntityIndexer}.
  */
-public class MongoEntityIndexerIT extends MongoTestBase {
+public class MongoEntityIndexerIT extends MongoITBase {
 
     private static final Type PERSON_TYPE =
             new Type(new RyaURI("urn:person"),



[10/17] incubator-rya git commit: RYA-414 Fixing broken tests, cleaning up documentate, cleaning up whitespace.

Posted by dl...@apache.org.
RYA-414 Fixing broken tests, cleaning up documentate, cleaning up whitespace.


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

Branch: refs/heads/master
Commit: 157c06491cd814a1d6e445ebfe77fc63226e5739
Parents: 17cebae
Author: kchilton2 <ke...@gmail.com>
Authored: Tue Dec 26 18:42:33 2017 -0500
Committer: kchilton2 <ke...@gmail.com>
Committed: Wed Dec 27 14:45:18 2017 -0500

----------------------------------------------------------------------
 .../AbstractMongoDBRdfConfigurationBuilder.java |  18 +-
 .../rya/mongodb/EmbeddedMongoFactory.java       |   5 +-
 .../apache/rya/mongodb/MongoDBQueryEngine.java  |  16 +-
 .../rya/mongodb/MongoDBRdfConfiguration.java    |   8 +-
 .../apache/rya/mongodb/MongoSecondaryIndex.java |   5 +-
 .../StatefulMongoDBRdfConfiguration.java        |  22 +-
 .../dao/SimpleMongoDBNamespaceManager.java      | 244 +++----
 .../mongodb/MongoDBRdfConfigurationTest.java    |   4 +-
 .../org/apache/rya/mongodb/MongoTestBase.java   |   5 +-
 .../rya/indexing/FilterFunctionOptimizer.java   |  24 +-
 .../entity/update/BaseEntityIndexer.java        |  24 +-
 .../entity/update/mongo/MongoEntityIndexer.java |   4 +-
 .../rya/indexing/mongodb/MongoDbSmartUri.java   |   2 +-
 .../apache/rya/sail/config/RyaSailFactory.java  |  64 +-
 .../update/mongo/MongoEntityIndexerIT.java      | 468 +++++++-------
 .../rya/indexing/mongo/MongoEntityIndexIT.java  |  50 +-
 .../mongo/MongoIndexingConfigurationTest.java   |   4 +-
 .../mongo/MongoTemporalIndexerTest.java         |  23 -
 .../metadata/MongoStatementMetadataIT.java      | 354 +++++------
 .../MongoStatementMetadataNodeTest.java         | 634 +++++++++----------
 .../src/main/java/MongoRyaDirectExample.java    |  12 +-
 .../client/merge/StatementStoreFactory.java     |   3 +-
 .../rya/indexing/export/StoreToStoreIT.java     |   4 +-
 .../GeoEnabledFilterFunctionOptimizer.java      |  60 +-
 .../apache/rya/indexing/GeoRyaSailFactory.java  |  13 +-
 extras/rya.geoindexing/geo.mongo/pom.xml        |   4 +-
 .../mongo/MongoGeoTemporalIndexer.java          |   8 +-
 .../geotemporal/MongoGeoTemporalIndexIT.java    | 257 ++++----
 .../indexing/mongo/MongoGeoIndexerFilterIT.java | 472 +++++++-------
 .../indexing/mongo/MongoGeoIndexerSfTest.java   |  44 +-
 .../rya/indexing/mongo/MongoGeoIndexerTest.java |  22 +-
 .../indexing/mongo/MongoIndexerDeleteIT.java    | 108 ++--
 .../RdfCloudTripleStoreConnection.java          |   6 +-
 .../rya/RdfCloudTripleStoreConnectionTest.java  |  15 +-
 34 files changed, 1459 insertions(+), 1547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
index a2bd03f..277ba5e 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/AbstractMongoDBRdfConfigurationBuilder.java
@@ -63,7 +63,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param user - user name used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoUser(final String user) {
+    public B setMongoUser(String user) {
         this.user = user;
         return confBuilder();
     }
@@ -75,7 +75,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param password - password used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoPassword(final String password) {
+    public B setMongoPassword(String password) {
         this.pass = password;
         return confBuilder();
     }
@@ -87,7 +87,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param port - port used to connect Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoPort(final String port) {
+    public B setMongoPort(String port) {
         this.port = port;
         return confBuilder();
     }
@@ -99,7 +99,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param host - host used to connect to Mongo
      * @return specified builder for chaining method invocations
      */
-    public B setMongoHost(final String host) {
+    public B setMongoHost(String host) {
         this.host = host;
         return confBuilder();
     }
@@ -111,7 +111,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param name - name of MongoDB to connect to
      * @return specified builder for chaining method invocations
      */
-    public B setMongoDBName(final String name) {
+    public B setMongoDBName(String name) {
         this.mongoDBName = name;
         return confBuilder();
     }
@@ -124,7 +124,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param name - name of Collection to connect to
      * @return specified builder for chaining method invocations
      */
-    public B setMongoCollectionPrefix(final String prefix) {
+    public B setMongoCollectionPrefix(String prefix) {
         this.mongoCollectionPrefix = prefix;
         return confBuilder();
     }
@@ -136,7 +136,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param useMock - indicates whether to use embedded Mongo as Rya backing
      * @return specified builder for chaining method invocations
      */
-    public B setUseMockMongo(final boolean useMock) {
+    public B setUseMockMongo(boolean useMock) {
         this.useMock = useMock;
         return confBuilder();
     }
@@ -155,7 +155,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
      * @param conf - Configuration object
      * @return - Configuration object with parameters set
      */
-    private C getConf(final C conf) {
+    private C getConf(C conf) {
 
         conf.setUseMock(useMock);
         conf.set("sc.useMongo", "true");
@@ -167,7 +167,7 @@ public abstract class AbstractMongoDBRdfConfigurationBuilder<B extends AbstractM
             conf.setMongoPassword(pass);
         }
         conf.setMongoDBName(mongoDBName);
-        conf.setRyaInstance(mongoCollectionPrefix);
+        conf.setRyaInstanceName(mongoCollectionPrefix);
         conf.setTablePrefix(mongoCollectionPrefix);
         conf.setMongoHostname(host);
         conf.setMongoPort(port);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
index d695ffa..ced8aa5 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/EmbeddedMongoFactory.java
@@ -58,7 +58,7 @@ public class EmbeddedMongoFactory {
      * Create the testing utility using the specified version of MongoDB.
      *
      * @param version
-     *            - version of MongoDB.
+     *            version of MongoDB.
      */
     private EmbeddedMongoFactory(final IFeatureAwareVersion version) throws IOException {
         final MongodStarter runtime = MongodStarter.getInstance(new RuntimeConfigBuilder().defaultsWithLogger(Command.MongoD, logger).build());
@@ -67,7 +67,7 @@ public class EmbeddedMongoFactory {
     }
 
     private IMongodConfig newMongodConfig(final IFeatureAwareVersion version) throws UnknownHostException, IOException {
-        final Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
+        Net net = new Net(findRandomOpenPortOnAllLocalInterfaces(), false);
         return new MongodConfigBuilder().version(version).net(net).build();
     }
 
@@ -95,7 +95,6 @@ public class EmbeddedMongoFactory {
         return mongodProcess.getConfig();
     }
 
-
     /**
      * Cleans up the resources created by the utility.
      */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
index d107d43..e01fa34 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBQueryEngine.java
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.mongodb;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.IOException;
 import java.util.AbstractMap;
 import java.util.Collection;
@@ -73,11 +75,11 @@ public class MongoDBQueryEngine implements RyaQueryEngine<StatefulMongoDBRdfConf
     public CloseableIteration<RyaStatement, RyaDAOException> query(
             final RyaStatement stmt, final StatefulMongoDBRdfConfiguration conf)
             throws RyaDAOException {
-        Preconditions.checkNotNull(stmt);
-        Preconditions.checkNotNull(conf);
+        checkNotNull(stmt);
+        checkNotNull(conf);
 
-        final Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
-        final Collection<Entry<RyaStatement, BindingSet>> collection = Collections.singleton(entry);
+        Entry<RyaStatement, BindingSet> entry = new AbstractMap.SimpleEntry<>(stmt, new MapBindingSet());
+        Collection<Entry<RyaStatement, BindingSet>> collection = Collections.singleton(entry);
 
         return new RyaStatementCursorIterator(queryWithBindingSet(collection, conf));
     }
@@ -86,8 +88,8 @@ public class MongoDBQueryEngine implements RyaQueryEngine<StatefulMongoDBRdfConf
     public CloseableIteration<? extends Entry<RyaStatement, BindingSet>, RyaDAOException> queryWithBindingSet(
             final Collection<Entry<RyaStatement, BindingSet>> stmts,
             final StatefulMongoDBRdfConfiguration conf) throws RyaDAOException {
-        Preconditions.checkNotNull(stmts);
-        Preconditions.checkNotNull(conf);
+        checkNotNull(stmts);
+        checkNotNull(conf);
 
         final Multimap<RyaStatement, BindingSet> rangeMap = HashMultimap.create();
 
@@ -141,7 +143,7 @@ public class MongoDBQueryEngine implements RyaQueryEngine<StatefulMongoDBRdfConf
             queries.put(stmt, new MapBindingSet());
         }
 
-        final Iterator<RyaStatement> iterator = new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), getConf()));
+        Iterator<RyaStatement> iterator = new RyaStatementCursorIterator(queryWithBindingSet(queries.entrySet(), getConf()));
         return CloseableIterables.wrap((Iterable<RyaStatement>) () -> iterator);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
index 4d06ea1..269a73c 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRdfConfiguration.java
@@ -191,14 +191,14 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
     /**
      * @return The name of the Rya instance to connect to. (default: rya)
      */
-    public String getRyaInstance() {
+    public String getRyaInstanceName() {
         return get(MONGO_COLLECTION_PREFIX, "rya");
     }
 
     /**
      * @param name - The name of the Rya instance to connect to.
      */
-    public void setRyaInstance(final String name) {
+    public void setRyaInstanceName(final String name) {
         requireNonNull(name);
         set(MONGO_COLLECTION_PREFIX, name);
     }
@@ -207,14 +207,14 @@ public class MongoDBRdfConfiguration extends RdfCloudTripleStoreConfiguration {
      * @return The name of the MongoDB Collection that contains Rya statements. (default: rya_triples)
      */
     public String getTriplesCollectionName() {
-        return getRyaInstance() + "_triples";
+        return getRyaInstanceName() + "_triples";
     }
 
     /**
      * @return The name of the MongoDB Collection that contains the Rya namespace. (default: rya_ns)
      */
     public String getNameSpacesCollectionName() {
-        return getRyaInstance() + "_ns";
+        return getRyaInstanceName() + "_ns";
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
index 3be6da8..510e013 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoSecondaryIndex.java
@@ -20,10 +20,7 @@ package org.apache.rya.mongodb;
 
 import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
 
-/**
- *TODO: doc me.
- */
 public interface MongoSecondaryIndex extends RyaSecondaryIndexer{
     @Override
-	public void init();    
+	public void init();
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
index 12bc5e3..61c349e 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/StatefulMongoDBRdfConfiguration.java
@@ -61,7 +61,7 @@ public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
         this.mongoClient = requireNonNull(mongoClient);
         this.indexers = requireNonNull(indexers);
     }
-    
+
     /**
      * Constructs an instance of {@link StatefulMongoDBRdfConfiguration} pre-loaded with values.
      *
@@ -75,14 +75,6 @@ public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
     }
 
     /**
-     * TODO doc
-     * @param indexers (not null)
-     */
-    public void setIndexers(final List<MongoSecondaryIndex> indexers) {
-    	this.indexers = requireNonNull(indexers);
-    }
-    
-    /**
      * @return The {@link MongoClient} that Rya will use.
      */
     public MongoClient getMongoClient() {
@@ -90,9 +82,21 @@ public class StatefulMongoDBRdfConfiguration extends MongoDBRdfConfiguration {
     }
 
     /**
+     * @param indexers - The {@link MongoSecondaryIndex}s that Rya will use. (not null)
+     */
+    public void setIndexers(final List<MongoSecondaryIndex> indexers) {
+        this.indexers = requireNonNull(indexers);
+    }
+
+    /**
      * @return The {@link MongoSecondaryIndex}s that Rya will use.
      */
     public List<MongoSecondaryIndex> getAdditionalIndexers() {
         return indexers;
     }
+
+    @Override
+    public MongoDBRdfConfiguration clone() {
+        return new StatefulMongoDBRdfConfiguration(this, mongoClient, indexers);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
index 88fa2b5..91ee064 100644
--- a/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
+++ b/dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBNamespaceManager.java
@@ -37,145 +37,145 @@ import info.aduna.iteration.CloseableIteration;
 
 public class SimpleMongoDBNamespaceManager implements MongoDBNamespaceManager {
 
-	public class NamespaceImplementation implements Namespace {
+    public class NamespaceImplementation implements Namespace {
 
-		private final String namespace;
-		private final String prefix;
+        private final String namespace;
+        private final String prefix;
 
-		public NamespaceImplementation(final String namespace, final String prefix) {
-			this.namespace = namespace;
-			this.prefix = prefix;
-		}
+        public NamespaceImplementation(final String namespace, final String prefix) {
+            this.namespace = namespace;
+            this.prefix = prefix;
+        }
 
-		@Override
-		public int compareTo(final Namespace o) {
-			if (!namespace.equalsIgnoreCase(o.getName())) {
+        @Override
+        public int compareTo(final Namespace o) {
+            if (!namespace.equalsIgnoreCase(o.getName())) {
                 return namespace.compareTo(o.getName());
             }
-			if (!prefix.equalsIgnoreCase(o.getPrefix())) {
+            if (!prefix.equalsIgnoreCase(o.getPrefix())) {
                 return prefix.compareTo(o.getPrefix());
             }
-			return 0;
-		}
-
-		@Override
-		public String getName() {
-			return namespace;
-		}
-
-		@Override
-		public String getPrefix() {
-			return prefix;
-		}
-
-	}
-
-	public class MongoCursorIteration implements
-			CloseableIteration<Namespace, RyaDAOException> {
-		private final DBCursor cursor;
-
-		public MongoCursorIteration(final DBCursor cursor2) {
-			this.cursor = cursor2;
-		}
-
-		@Override
-		public boolean hasNext() throws RyaDAOException {
-			return cursor.hasNext();
-		}
-
-		@Override
-		public Namespace next() throws RyaDAOException {
-			final DBObject ns = cursor.next();
-			final Map values = ns.toMap();
-			final String namespace = (String) values.get(NAMESPACE);
-			final String prefix = (String) values.get(PREFIX);
-
-			final Namespace temp =  new NamespaceImplementation(namespace, prefix);
-			return temp;
-		}
-
-		@Override
-		public void remove() throws RyaDAOException {
-			next();
-		}
-
-		@Override
-		public void close() throws RyaDAOException {
-			cursor.close();
-		}
-
-	}
-
-	private static final String ID = "_id";
-	private static final String PREFIX = "prefix";
-	private static final String NAMESPACE = "namespace";
-	private StatefulMongoDBRdfConfiguration conf;
-	private final DBCollection nsColl;
-
-
-	public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
-		nsColl = nameSpaceCollection;
-	}
-
-	@Override
-	public void createIndices(final DBCollection coll){
-		coll.createIndex(PREFIX);
-		coll.createIndex(NAMESPACE);
-	}
-
-	@Override
-	public void setConf(final StatefulMongoDBRdfConfiguration conf) {
-		this.conf = conf;
-	}
-
-	@Override
-	public StatefulMongoDBRdfConfiguration getConf() {
-		return conf;
-	}
-
-	@Override
-	public void addNamespace(final String prefix, final String namespace)
-			throws RyaDAOException {
-		final String id = prefix;
-		byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
-		try {
-			final MessageDigest digest = MessageDigest.getInstance("SHA-1");
-			bytes = digest.digest(bytes);
-		} catch (final NoSuchAlgorithmException e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		}
-		final BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))
-		.append(PREFIX, prefix)
-	    .append(NAMESPACE, namespace);
-		nsColl.insert(doc);
-
-	}
-
-	@Override
-	public String getNamespace(final String prefix) throws RyaDAOException {
+            return 0;
+        }
+
+        @Override
+        public String getName() {
+            return namespace;
+        }
+
+        @Override
+        public String getPrefix() {
+            return prefix;
+        }
+
+    }
+
+    public class MongoCursorIteration implements
+    CloseableIteration<Namespace, RyaDAOException> {
+        private final DBCursor cursor;
+
+        public MongoCursorIteration(final DBCursor cursor2) {
+            this.cursor = cursor2;
+        }
+
+        @Override
+        public boolean hasNext() throws RyaDAOException {
+            return cursor.hasNext();
+        }
+
+        @Override
+        public Namespace next() throws RyaDAOException {
+            final DBObject ns = cursor.next();
+            final Map values = ns.toMap();
+            final String namespace = (String) values.get(NAMESPACE);
+            final String prefix = (String) values.get(PREFIX);
+
+            final Namespace temp =  new NamespaceImplementation(namespace, prefix);
+            return temp;
+        }
+
+        @Override
+        public void remove() throws RyaDAOException {
+            next();
+        }
+
+        @Override
+        public void close() throws RyaDAOException {
+            cursor.close();
+        }
+
+    }
+
+    private static final String ID = "_id";
+    private static final String PREFIX = "prefix";
+    private static final String NAMESPACE = "namespace";
+    private StatefulMongoDBRdfConfiguration conf;
+    private final DBCollection nsColl;
+
+
+    public SimpleMongoDBNamespaceManager(final DBCollection nameSpaceCollection) {
+        nsColl = nameSpaceCollection;
+    }
+
+    @Override
+    public void createIndices(final DBCollection coll){
+        coll.createIndex(PREFIX);
+        coll.createIndex(NAMESPACE);
+    }
+
+    @Override
+    public void setConf(final StatefulMongoDBRdfConfiguration conf) {
+        this.conf = conf;
+    }
+
+    @Override
+    public StatefulMongoDBRdfConfiguration getConf() {
+        return conf;
+    }
+
+    @Override
+    public void addNamespace(final String prefix, final String namespace)
+            throws RyaDAOException {
+        final String id = prefix;
+        byte[] bytes = id.getBytes(StandardCharsets.UTF_8);
+        try {
+            final MessageDigest digest = MessageDigest.getInstance("SHA-1");
+            bytes = digest.digest(bytes);
+        } catch (final NoSuchAlgorithmException e) {
+            // TODO Auto-generated catch block
+            e.printStackTrace();
+        }
+        final BasicDBObject doc = new BasicDBObject(ID, new String(Hex.encodeHex(bytes)))
+                .append(PREFIX, prefix)
+                .append(NAMESPACE, namespace);
+        nsColl.insert(doc);
+
+    }
+
+    @Override
+    public String getNamespace(final String prefix) throws RyaDAOException {
         final DBObject query = new BasicDBObject().append(PREFIX, prefix);
         final DBCursor cursor = nsColl.find(query);
         String nameSpace = prefix;
         while (cursor.hasNext()){
-          final DBObject obj = cursor.next();
-          nameSpace = (String) obj.toMap().get(NAMESPACE);
+            final DBObject obj = cursor.next();
+            nameSpace = (String) obj.toMap().get(NAMESPACE);
         }
         return nameSpace;
-	}
+    }
 
-	@Override
-	public void removeNamespace(final String prefix) throws RyaDAOException {
+    @Override
+    public void removeNamespace(final String prefix) throws RyaDAOException {
         final DBObject query = new BasicDBObject().append(PREFIX, prefix);
-		nsColl.remove(query);
-	}
+        nsColl.remove(query);
+    }
 
-	@Override
-	public CloseableIteration<? extends Namespace, RyaDAOException> iterateNamespace()
-			throws RyaDAOException {
+    @Override
+    public CloseableIteration<? extends Namespace, RyaDAOException> iterateNamespace()
+            throws RyaDAOException {
         final DBObject query = new BasicDBObject();
         final DBCursor cursor = nsColl.find(query);
-		return new MongoCursorIteration(cursor);
-	}
+        return new MongoCursorIteration(cursor);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
index 2d28bb5..948c3d5 100644
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
+++ b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRdfConfigurationTest.java
@@ -64,7 +64,7 @@ public class MongoDBRdfConfigurationTest {
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getRyaInstance(), "prefix_");
+        assertEquals(conf.getRyaInstanceName(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
 
@@ -95,7 +95,7 @@ public class MongoDBRdfConfigurationTest {
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getRyaInstance(), "prefix_");
+        assertEquals(conf.getRyaInstanceName(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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
index a3db535..6dbd19b 100644
--- 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
@@ -62,9 +62,10 @@ public class MongoTestBase {
     }
 
     /**
-     * TODO doc
+     * Override this method if you would like to augment the configuration object that
+     * will be used to initialize indexers and create the mongo client prior to running a test.
      *
-     * @param conf
+     * @param conf - The configuration object that may be updated. (not null)
      */
     protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         // By default, do nothing.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
index 58ec29a..6c4e05b 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/FilterFunctionOptimizer.java
@@ -97,16 +97,16 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     private synchronized void init() {
         if (!init) {
             if (ConfigUtils.getUseMongo(conf)) {
-            	StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
-            	for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
-        			if(indexer instanceof FreeTextIndexer) {
-        				freeTextIndexer = (FreeTextIndexer) indexer;
-        			} else if(indexer instanceof TemporalIndexer) {
-        				temporalIndexer = (TemporalIndexer) indexer;
-        			}
-            	}
+                StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
+                for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
+                    if(indexer instanceof FreeTextIndexer) {
+                        freeTextIndexer = (FreeTextIndexer) indexer;
+                    } else if(indexer instanceof TemporalIndexer) {
+                        temporalIndexer = (TemporalIndexer) indexer;
+                    }
+                }
             } else {
-                 freeTextIndexer = new AccumuloFreeTextIndexer();
+                freeTextIndexer = new AccumuloFreeTextIndexer();
                 freeTextIndexer.setConf(conf);
                 temporalIndexer = new AccumuloTemporalIndexer();
                 temporalIndexer.setConf(conf);
@@ -164,7 +164,7 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
 
     //find vars contained in filters
     private static class SearchVarVisitor extends QueryModelVisitorBase<RuntimeException> {
-        private final Collection<Var> searchProperties = new ArrayList<Var>();
+        private final Collection<Var> searchProperties = new ArrayList<>();
 
         @Override
         public void meet(final FunctionCall fn) {
@@ -179,8 +179,8 @@ public class FilterFunctionOptimizer implements QueryOptimizer, Configurable {
     //find StatementPatterns containing filter variables
     private static class MatchStatementVisitor extends QueryModelVisitorBase<RuntimeException> {
         private final Collection<Var> propertyVars;
-        private final Collection<Var> usedVars = new ArrayList<Var>();
-        private final List<StatementPattern> matchStatements = new ArrayList<StatementPattern>();
+        private final Collection<Var> usedVars = new ArrayList<>();
+        private final List<StatementPattern> matchStatements = new ArrayList<>();
 
         public MatchStatementVisitor(final Collection<Var> propertyVars) {
             this.propertyVars = propertyVars;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
index 2ca6761..7392318 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/BaseEntityIndexer.java
@@ -75,21 +75,21 @@ public abstract class BaseEntityIndexer implements EntityIndexer, MongoSecondary
     private final AtomicReference<TypeStorage> types = new AtomicReference<>();
 
     @Override
-	public void init() {
-    	try {
-    		entities.set(getEntityStorage());
-    	} catch (final EntityStorageException e) {
-    		log.error("Unable to set entity storage.");
-    	}
-    	types.set(getTypeStorage());
+    public void init() {
+        try {
+            entities.set(getEntityStorage());
+        } catch (final EntityStorageException e) {
+            log.error("Unable to set entity storage.");
+        }
+        types.set(getTypeStorage());
     }
-    
+
     @Override
     public void setConf(final Configuration conf) {
         requireNonNull(conf);
-        checkArgument(conf instanceof StatefulMongoDBRdfConfiguration, 
-        		"The configuration provided must be a StatefulMongoDBRdfConfiguration, found: " 
-        	    + conf.getClass().getSimpleName());
+        checkArgument(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The configuration provided must be a StatefulMongoDBRdfConfiguration, found: "
+                        + conf.getClass().getSimpleName());
         configuration.set((StatefulMongoDBRdfConfiguration) conf);
     }
 
@@ -109,7 +109,7 @@ public abstract class BaseEntityIndexer implements EntityIndexer, MongoSecondary
         requireNonNull(statements);
 
         final Map<RyaURI,List<RyaStatement>> groupedBySubject = statements.stream()
-            .collect(groupingBy(RyaStatement::getSubject));
+                .collect(groupingBy(RyaStatement::getSubject));
 
         for(final Entry<RyaURI, List<RyaStatement>> entry : groupedBySubject.entrySet()) {
             try {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
index ea28388..a850557 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
@@ -39,12 +39,12 @@ public class MongoEntityIndexer extends BaseEntityIndexer {
     @Override
     public EntityStorage getEntityStorage() throws EntityStorageException {
     	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
-        return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstance());
+        return new MongoEntityStorage(conf.getMongoClient(), conf.getRyaInstanceName());
     }
 
     @Override
     public TypeStorage getTypeStorage() {
     	final StatefulMongoDBRdfConfiguration conf = super.configuration.get();
-        return new MongoTypeStorage(conf.getMongoClient(), conf.getRyaInstance());
+        return new MongoTypeStorage(conf.getMongoClient(), conf.getRyaInstanceName());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 249bde5..a50d293 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
@@ -148,7 +148,7 @@ public class MongoDbSmartUri implements SmartUriStorage {
      */
     private void setupClient(final StatefulMongoDBRdfConfiguration conf) throws UnknownHostException, MongoException, EntityStorageException {
         mongoClient = conf.getMongoClient();
-        entityStorage = new MongoEntityStorage(mongoClient, conf.getRyaInstance());
+        entityStorage = new MongoEntityStorage(mongoClient, conf.getRyaInstanceName());
         isInit = true;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
index ffde9b2..b5adee3 100644
--- a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
+++ b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
@@ -95,11 +95,9 @@ public class RyaSailFactory {
             // Add the Indexer and Optimizer names to the configuration object that are configured to be used.
             ConfigUtils.setIndexers(mongoConfig);
 
-            // Initialize the indexer and optimizer objects that will be used within the Sail object.
-
             // Populate the configuration using previously stored Rya Details if this instance uses them.
             try {
-                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstance());
+                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstanceName());
                 RyaDetailsToConfiguration.addRyaDetailsToConfiguration(ryaDetailsRepo.getRyaInstanceDetails(), mongoConfig);
             } catch (final RyaDetailsRepositoryException e) {
                LOG.info("Instance does not have a rya details collection, skipping.");
@@ -239,62 +237,4 @@ public class RyaSailFactory {
             LOG.info("Instance does not have a rya details collection, skipping.");
         }
     }
-}
-
-
-
-///**
-//* TODO add docs.  names for reflection
-//* @param indexers
-//*/
-//public void setMongoIndexers(final Class<? extends MongoSecondaryIndex>... indexers) {
-// final List<String> strs = Lists.newArrayList();
-// for (final Class<?> ai : indexers){
-//     strs.add(ai.getName());
-// }
-//
-// setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
-//}
-
-///**
-//* TODO add docs. explain hack is used here. do reflection. eww.
-//* @return
-//*/
-//public List<MongoSecondaryIndex> getAdditionalIndexers() {
-// stateLock.lock();
-// try {
-//     if(indexers == null) {
-//         indexers = getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
-//     }
-//     return indexers;
-// } finally {
-//     stateLock.unlock();
-// }
-//}
-
-//// XXX Not sure what all of this stuff is for. I'm guessing Rya Sail state stuff.
-//public void setAdditionalIndexers(final Class<? extends MongoSecondaryIndex>... indexers) {
-//  final List<String> strs = Lists.newArrayList();
-//  for (final Class<?> ai : indexers){
-//      strs.add(ai.getName());
-//  }
-//
-//  setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
-//}
-//
-
-//conf.setStrings(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, indexList.toArray(new String[] {}));
-//conf.setStrings(RdfCloudTripleStoreConfiguration.CONF_OPTIMIZERS, optimizers.toArray(new String[] {}));
-
-//public List<MongoSecondaryIndex> getAdditionalIndexers() {
-//  return getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
-//}
-
-//public void setMongoClient(final MongoClient client) {
-//  requireNonNull(client);
-//  this.mongoClient = client;
-//}
-//
-//public MongoClient getMongoClient() {
-//  return mongoClient;
-//}
\ No newline at end of file
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 88b2ed0..8b62a57 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
@@ -45,238 +45,238 @@ import com.google.common.collect.Sets;
  */
 public class MongoEntityIndexerIT extends MongoTestBase {
 
-	private static final Type PERSON_TYPE =
-			new Type(new RyaURI("urn:person"),
-					ImmutableSet.<RyaURI>builder()
-					.add(new RyaURI("urn:name"))
-					.add(new RyaURI("urn:age"))
-					.add(new RyaURI("urn:eye"))
-					.build());
-
-	private static final Type EMPLOYEE_TYPE =
-			new Type(new RyaURI("urn:employee"),
-					ImmutableSet.<RyaURI>builder()
-					.add(new RyaURI("urn:name"))
-					.add(new RyaURI("urn:hoursPerWeek"))
-					.build());
-
-	@Test
-	public void addStatement_setsType() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load a type into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-
-			// Index a RyaStatement that will create an Entity with an explicit type.
-			final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person"));
-			indexer.storeStatement(statement);
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-			final Entity expected = Entity.builder()
-					.setSubject(new RyaURI("urn:SSN/111-11-1111"))
-					.setExplicitType(new RyaURI("urn:person"))
-					.build();
-
-			assertEquals(expected, entity);
-		}
-	}
-
-	@Test
-	public void addStatement_setsProperty() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load the types into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-			types.create(EMPLOYEE_TYPE);
-
-			// Index a RyaStatement that will create an Entity with two implicit types.
-			final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice"));
-			indexer.storeStatement(statement);
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-			final Entity expected = Entity.builder()
-					.setSubject(new RyaURI("urn:SSN/111-11-1111"))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.build();
-
-			assertEquals(expected, entity);
-		}
-	}
-
-	@Test
-	public void addStatement_manyUpdates() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load the types into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-			types.create(EMPLOYEE_TYPE);
-
-			// Index a bunch of RyaStatements.
-			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
-			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
-			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")));
-			indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")));
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-			final Entity expected = Entity.builder()
-					.setSubject(aliceSSN)
-					.setExplicitType(new RyaURI("urn:person"))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setVersion( entity.getVersion() )
-					.build();
-
-			assertEquals(expected, entity);
-		}
-	}
-
-	@Test
-	public void addStatements() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load the types into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-			types.create(EMPLOYEE_TYPE);
-
-			// Index a bunch of RyaStatements.
-			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-			final RyaURI bobSSN = new RyaURI("urn:SSN/222-22-2222");
-
-			indexer.storeStatements(Sets.newHashSet(
-					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")),
-
-					new RyaStatement(bobSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")),
-					new RyaStatement(bobSSN, new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")),
-					new RyaStatement(bobSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:employee"))));
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-
-			final Entity alice = entities.get(aliceSSN).get();
-			final Entity bob = entities.get(bobSSN).get();
-			final Set<Entity> storedEntities = Sets.newHashSet(alice, bob);
-
-			final Entity expectedAlice = Entity.builder()
-					.setSubject(aliceSSN)
-					.setExplicitType(new RyaURI("urn:person"))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setVersion( alice.getVersion() )
-					.build();
-			final Entity expectedBob = Entity.builder()
-					.setSubject(bobSSN)
-					.setExplicitType(new RyaURI("urn:employee"))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
-					.setVersion( bob.getVersion() )
-					.build();
-			final Set<Entity> expected = Sets.newHashSet(expectedAlice, expectedBob);
-
-			assertEquals(expected, storedEntities);
-		}
-	}
-
-	@Test
-	public void deleteStatement_deletesType() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load the type into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-			types.create(EMPLOYEE_TYPE);
-
-			// Index a bunch of RyaStatements.
-			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-
-			indexer.storeStatements(Sets.newHashSet(
-					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
-
-			// Remove the explicit type from Alice.
-			indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-			final Entity expected = Entity.builder()
-					.setSubject(aliceSSN)
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-					.setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
-					.setVersion( entity.getVersion() )
-					.build();
-
-			assertEquals(expected, entity);
-		}
-	}
-
-	@Test
-	public void deleteStatement_deletesProperty() throws Exception {
-		try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-			// Load the type into the TypeStorage.
-			final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstance());
-			types.create(PERSON_TYPE);
-			types.create(EMPLOYEE_TYPE);
-
-			// Index a bunch of RyaStatements.
-			final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
-
-			indexer.storeStatements(Sets.newHashSet(
-					new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
-					new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
-
-			// Remove the name property from Alice.
-			indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
-
-			// Fetch the Entity from storage and ensure it looks correct.
-			final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstance());
-			final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
-
-			final Entity expected = Entity.builder()
-					.setSubject(aliceSSN)
-					.setExplicitType(new RyaURI("urn:person"))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
-					.setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
-					.setVersion( entity.getVersion() )
-					.build();
-
-			assertEquals(expected, entity);
-		}
-	}
+    private static final Type PERSON_TYPE =
+            new Type(new RyaURI("urn:person"),
+                    ImmutableSet.<RyaURI>builder()
+                    .add(new RyaURI("urn:name"))
+                    .add(new RyaURI("urn:age"))
+                    .add(new RyaURI("urn:eye"))
+                    .build());
+
+    private static final Type EMPLOYEE_TYPE =
+            new Type(new RyaURI("urn:employee"),
+                    ImmutableSet.<RyaURI>builder()
+                    .add(new RyaURI("urn:name"))
+                    .add(new RyaURI("urn:hoursPerWeek"))
+                    .build());
+
+    @Test
+    public void addStatement_setsType() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load a type into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+
+            // Index a RyaStatement that will create an Entity with an explicit type.
+            final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person"));
+            indexer.storeStatement(statement);
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+            final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+            final Entity expected = Entity.builder()
+                    .setSubject(new RyaURI("urn:SSN/111-11-1111"))
+                    .setExplicitType(new RyaURI("urn:person"))
+                    .build();
+
+            assertEquals(expected, entity);
+        }
+    }
+
+    @Test
+    public void addStatement_setsProperty() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load the types into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+            types.create(EMPLOYEE_TYPE);
+
+            // Index a RyaStatement that will create an Entity with two implicit types.
+            final RyaStatement statement = new RyaStatement(new RyaURI("urn:SSN/111-11-1111"), new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice"));
+            indexer.storeStatement(statement);
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+            final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+            final Entity expected = Entity.builder()
+                    .setSubject(new RyaURI("urn:SSN/111-11-1111"))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .build();
+
+            assertEquals(expected, entity);
+        }
+    }
+
+    @Test
+    public void addStatement_manyUpdates() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load the types into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+            types.create(EMPLOYEE_TYPE);
+
+            // Index a bunch of RyaStatements.
+            final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+            indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
+            indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
+            indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")));
+            indexer.storeStatement(new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")));
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+            final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+            final Entity expected = Entity.builder()
+                    .setSubject(aliceSSN)
+                    .setExplicitType(new RyaURI("urn:person"))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setVersion( entity.getVersion() )
+                    .build();
+
+            assertEquals(expected, entity);
+        }
+    }
+
+    @Test
+    public void addStatements() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load the types into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+            types.create(EMPLOYEE_TYPE);
+
+            // Index a bunch of RyaStatements.
+            final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+            final RyaURI bobSSN = new RyaURI("urn:SSN/222-22-2222");
+
+            indexer.storeStatements(Sets.newHashSet(
+                    new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")),
+
+                    new RyaStatement(bobSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")),
+                    new RyaStatement(bobSSN, new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")),
+                    new RyaStatement(bobSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:employee"))));
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+
+            final Entity alice = entities.get(aliceSSN).get();
+            final Entity bob = entities.get(bobSSN).get();
+            final Set<Entity> storedEntities = Sets.newHashSet(alice, bob);
+
+            final Entity expectedAlice = Entity.builder()
+                    .setSubject(aliceSSN)
+                    .setExplicitType(new RyaURI("urn:person"))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setVersion( alice.getVersion() )
+                    .build();
+            final Entity expectedBob = Entity.builder()
+                    .setSubject(bobSSN)
+                    .setExplicitType(new RyaURI("urn:employee"))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:hoursPerWeek"), new RyaType(XMLSchema.INT, "40")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Bob")))
+                    .setVersion( bob.getVersion() )
+                    .build();
+            final Set<Entity> expected = Sets.newHashSet(expectedAlice, expectedBob);
+
+            assertEquals(expected, storedEntities);
+        }
+    }
+
+    @Test
+    public void deleteStatement_deletesType() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load the type into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+            types.create(EMPLOYEE_TYPE);
+
+            // Index a bunch of RyaStatements.
+            final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+
+            indexer.storeStatements(Sets.newHashSet(
+                    new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
+
+            // Remove the explicit type from Alice.
+            indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")));
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+            final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+            final Entity expected = Entity.builder()
+                    .setSubject(aliceSSN)
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+                    .setProperty(new RyaURI("urn:employee"), new Property(new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")))
+                    .setVersion( entity.getVersion() )
+                    .build();
+
+            assertEquals(expected, entity);
+        }
+    }
+
+    @Test
+    public void deleteStatement_deletesProperty() throws Exception {
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+            // Load the type into the TypeStorage.
+            final TypeStorage types = new MongoTypeStorage(getMongoClient(), conf.getRyaInstanceName());
+            types.create(PERSON_TYPE);
+            types.create(EMPLOYEE_TYPE);
+
+            // Index a bunch of RyaStatements.
+            final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
+
+            indexer.storeStatements(Sets.newHashSet(
+                    new RyaStatement(aliceSSN, new RyaURI( RDF.TYPE.toString() ), new RyaType(XMLSchema.ANYURI, "urn:person")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")),
+                    new RyaStatement(aliceSSN, new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue"))));
+
+            // Remove the name property from Alice.
+            indexer.deleteStatement(new RyaStatement(aliceSSN, new RyaURI("urn:name"), new RyaType(XMLSchema.STRING, "Alice")));
+
+            // Fetch the Entity from storage and ensure it looks correct.
+            final EntityStorage entities = new MongoEntityStorage(getMongoClient(), conf.getRyaInstanceName());
+            final Entity entity = entities.get(new RyaURI("urn:SSN/111-11-1111")).get();
+
+            final Entity expected = Entity.builder()
+                    .setSubject(aliceSSN)
+                    .setExplicitType(new RyaURI("urn:person"))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, "30")))
+                    .setProperty(new RyaURI("urn:person"), new Property(new RyaURI("urn:eye"), new RyaType(XMLSchema.STRING, "blue")))
+                    .setVersion( entity.getVersion() )
+                    .build();
+
+            assertEquals(expected, entity);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 c33a5bb..e476874 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
@@ -67,8 +67,8 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-        	indexer.setConf(conf);
-        	indexer.init();
+            indexer.setConf(conf);
+            indexer.init();
 
             setupTypes(indexer);
             addStatements(conn);
@@ -89,9 +89,9 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-        	indexer.setConf(conf);
-        	indexer.init();
-        	
+            indexer.setConf(conf);
+            indexer.init();
+
             setupTypes(indexer);
             addStatements(conn);
 
@@ -125,9 +125,9 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.begin();
 
         try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
-        	indexer.setConf(conf);
-        	indexer.init();
-        	
+            indexer.setConf(conf);
+            indexer.init();
+
             setupTypes(indexer);
             addStatements(conn);
             conn.commit();
@@ -136,7 +136,7 @@ public class MongoEntityIndexIT extends MongoTestBase {
                     "<urn:george> <" + RDF.TYPE + "> <urn:person> ."+
                     "<urn:george> <urn:name> ?name . " +
                     "<urn:george> <urn:eye> ?eye . " +
-                "}";
+                    "}";
 
             final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
             final Set<BindingSet> results = new HashSet<>();
@@ -163,31 +163,31 @@ public class MongoEntityIndexIT extends MongoTestBase {
         // Add some Types to the storage.
         final Type cat = new Type(new RyaURI("urn:cat"),
                 ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:numLegs"))
-                    .add(new RyaURI("urn:eye"))
-                    .add(new RyaURI("urn:species"))
-                    .build());
+                .add(new RyaURI("urn:numLegs"))
+                .add(new RyaURI("urn:eye"))
+                .add(new RyaURI("urn:species"))
+                .build());
 
         final Type dog = new Type(new RyaURI("urn:dog"),
                 ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:numLegs"))
-                    .add(new RyaURI("urn:eye"))
-                    .add(new RyaURI("urn:species"))
-                    .build());
+                .add(new RyaURI("urn:numLegs"))
+                .add(new RyaURI("urn:eye"))
+                .add(new RyaURI("urn:species"))
+                .build());
 
         final Type icecream = new Type(new RyaURI("urn:icecream"),
                 ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:brand"))
-                    .add(new RyaURI("urn:flavor"))
-                    .add(new RyaURI("urn:cost"))
-                    .build());
+                .add(new RyaURI("urn:brand"))
+                .add(new RyaURI("urn:flavor"))
+                .add(new RyaURI("urn:cost"))
+                .build());
 
         final Type person = new Type(new RyaURI("urn:person"),
                 ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:age"))
-                    .add(new RyaURI("urn:eye"))
-                    .build());
+                .add(new RyaURI("urn:name"))
+                .add(new RyaURI("urn:age"))
+                .add(new RyaURI("urn:eye"))
+                .build());
 
         typeStore.create(cat);
         typeStore.create(dog);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
index 1f083f2..708219c 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
@@ -72,7 +72,7 @@ public class MongoIndexingConfigurationTest {
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getRyaInstance(), "prefix_");
+        assertEquals(conf.getRyaInstanceName(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
         assertTrue(
@@ -107,7 +107,7 @@ public class MongoIndexingConfigurationTest {
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getRyaInstance(), "prefix_");
+        assertEquals(conf.getRyaInstanceName(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
         assertTrue(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 ec7d1ec..0f8497e 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
@@ -40,7 +40,6 @@ 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.MongoTestBase;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
@@ -80,13 +79,10 @@ import info.aduna.iteration.CloseableIteration;
  *
  */
 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";
     private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
-//    private static final String STAT_VALUEHASH = "valuehash";
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
 
     // Assign this in setUpBeforeClass, store them in each test.
@@ -162,25 +158,6 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
         }
     }
 
-    @Before
-    public void before() throws Exception {
-//        // This is from http://linkedevents.org/ontology
-//        // and http://motools.sourceforge.net/event/event.html
-//        conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
-//                + URI_PROPERTY_AT_TIME + ","
-//                + URI_PROPERTY_CIRCA + ","
-//                + URI_PROPERTY_EVENT_TIME);
-
-//        tIndexer
-//        tIndexer.setConf(conf);
-//        tIndexer.init();
-////        tIndexer.initIndexer(conf, super.getMongoClient());
-
-//        final String dbName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
-//        final DB db = super.getMongoClient().getDB(dbName);
-//        collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
-   }
-
     @Override
     protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         // This is from http://linkedevents.org/ontology


[09/17] incubator-rya git commit: RYA-414 Fixing broken tests, cleaning up documentate, cleaning up whitespace.

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 e62404c..c4db88e 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
@@ -51,207 +51,207 @@ import org.openrdf.sail.Sail;
 
 public class MongoStatementMetadataIT extends MongoTestBase {
 
-	private final String query1 = "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 ?a ?b ?c where {_:blankNode1 rdf:type owl:Annotation; ano:Source ?a; "
-			+ "ano:Property <http://worksAt>; ano:Target <http://BurgerShack>; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date. "
-			+ "_:blankNode2 rdf:type owl:Annotation; ano:Source ?a; "
-			+ "ano:Property <http://talksTo>; ano:Target ?b; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date }";
+    private final String query1 = "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 ?a ?b ?c where {_:blankNode1 rdf:type owl:Annotation; ano:Source ?a; "
+            + "ano:Property <http://worksAt>; ano:Target <http://BurgerShack>; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date. "
+            + "_:blankNode2 rdf:type owl:Annotation; ano:Source ?a; "
+            + "ano:Property <http://talksTo>; ano:Target ?b; <http://createdBy> ?c; <http://createdOn> \'2017-01-04\'^^xsd:date }";
 
-	@Override
-	protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
-		final Set<RyaURI> propertySet = new HashSet<>(
-				Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-		conf.setUseStatementMetadata(true);
-		conf.setStatementMetadataProperties(propertySet);
-	}
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        final Set<RyaURI> propertySet = new HashSet<>(
+                Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+        conf.setUseStatementMetadata(true);
+        conf.setStatementMetadataProperties(propertySet);
+    }
 
-	@Test
-	public void simpleQueryWithoutBindingSet() throws Exception {
-		Sail sail = RyaSailFactory.getInstance(conf);
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
+    @Test
+    public void simpleQueryWithoutBindingSet() throws Exception {
+        Sail sail = RyaSailFactory.getInstance(conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
 
-			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"));
+            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"));
 
-			final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement);
+            final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
 
-			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-			final QueryBindingSet bs = new QueryBindingSet();
-			bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-			bs.addBinding("y", new LiteralImpl("Joe"));
+            final QueryBindingSet bs = new QueryBindingSet();
+            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bs.addBinding("y", new LiteralImpl("Joe"));
 
-			final List<BindingSet> bsList = new ArrayList<>();
-			while (result.hasNext()) {
-				bsList.add(result.next());
-			}
+            final List<BindingSet> bsList = new ArrayList<>();
+            while (result.hasNext()) {
+                bsList.add(result.next());
+            }
 
-			assertEquals(1, bsList.size());
-			assertEquals(bs, bsList.get(0));
-			dao.delete(statement, conf);
-		} finally {
-			dao.destroy();
-			sail.shutDown();
-		}
-	}
+            assertEquals(1, bsList.size());
+            assertEquals(bs, bsList.get(0));
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+            sail.shutDown();
+        }
+    }
 
-	/**
-	 * 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 {
-		Sail sail = RyaSailFactory.getInstance(conf);
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"));
+    /**
+     * 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 {
+        Sail sail = RyaSailFactory.getInstance(conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"));
 
-			final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement);
+            final RyaStatement statement = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
 
-			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-			final List<BindingSet> bsList = new ArrayList<>();
-			while (result.hasNext()) {
-				bsList.add(result.next());
-			}
-			assertEquals(0, bsList.size());
-			dao.delete(statement, conf);
-		} finally {
-			dao.destroy();
-			sail.shutDown();
-		}
-	}
+            final List<BindingSet> bsList = new ArrayList<>();
+            while (result.hasNext()) {
+                bsList.add(result.next());
+            }
+            assertEquals(0, bsList.size());
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+            sail.shutDown();
+        }
+    }
 
-	@Test
-	public void simpleQueryWithBindingSet() throws Exception {
-		Sail sail = RyaSailFactory.getInstance(conf);
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"));
+    @Test
+    public void simpleQueryWithBindingSet() throws Exception {
+        Sail sail = RyaSailFactory.getInstance(conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"));
 
-			final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			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);
+            final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            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);
 
-			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
+            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query1).evaluate();
 
-			final Set<BindingSet> expected = new HashSet<>();
-			final QueryBindingSet expected1 = new QueryBindingSet();
-			expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
-			expected1.addBinding("y", new LiteralImpl("Joe"));
-			final QueryBindingSet expected2 = new QueryBindingSet();
-			expected2.addBinding("x", new LiteralImpl("HardwareStore"));
-			expected2.addBinding("y", new LiteralImpl("Joe"));
-			expected.add(expected1);
-			expected.add(expected2);
+            final Set<BindingSet> expected = new HashSet<>();
+            final QueryBindingSet expected1 = new QueryBindingSet();
+            expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
+            expected1.addBinding("y", new LiteralImpl("Joe"));
+            final QueryBindingSet expected2 = new QueryBindingSet();
+            expected2.addBinding("x", new LiteralImpl("HardwareStore"));
+            expected2.addBinding("y", new LiteralImpl("Joe"));
+            expected.add(expected1);
+            expected.add(expected2);
 
-			final Set<BindingSet> bsSet = new HashSet<>();
-			while (result.hasNext()) {
-				bsSet.add(result.next());
-			}
+            final Set<BindingSet> bsSet = new HashSet<>();
+            while (result.hasNext()) {
+                bsSet.add(result.next());
+            }
 
-			assertEquals(expected, bsSet);
+            assertEquals(expected, bsSet);
 
-			dao.delete(statement1, conf);
-			dao.delete(statement2, conf);
-		} finally {
-			dao.destroy();
-			sail.shutDown();
-		}
-	}
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+            sail.shutDown();
+        }
+    }
 
-	/**
-	 * Tests to see if correct result is passed back when a metadata statement
-	 * is joined with a StatementPattern statement (i.e. a common variable
-	 * appears in a StatementPattern statement and a metadata statement).
-	 * StatementPattern statements have either rdf:subject, rdf:predicate, or
-	 * rdf:object as the predicate while a metadata statement is any statement
-	 * in the reified query whose predicate is not rdf:type and not a
-	 * StatementPattern predicate.
-	 *
-	 * @throws MalformedQueryException
-	 * @throws QueryEvaluationException
-	 * @throws RyaDAOException
-	 */
-	@Test
-	public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
-		Sail sail = RyaSailFactory.getInstance(conf);
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"));
-			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"));
+    /**
+     * Tests to see if correct result is passed back when a metadata statement
+     * is joined with a StatementPattern statement (i.e. a common variable
+     * appears in a StatementPattern statement and a metadata statement).
+     * StatementPattern statements have either rdf:subject, rdf:predicate, or
+     * rdf:object as the predicate while a metadata statement is any statement
+     * in the reified query whose predicate is not rdf:type and not a
+     * StatementPattern predicate.
+     *
+     * @throws MalformedQueryException
+     * @throws QueryEvaluationException
+     * @throws RyaDAOException
+     */
+    @Test
+    public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
+        Sail sail = RyaSailFactory.getInstance(conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"));
+            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"));
 
-			final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
-					new RyaURI("http://BurgerShack"), new RyaURI("http://context"), "", metadata1);
-			final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
-					new RyaURI("http://Betty"), new RyaURI("http://context"), "", metadata1);
-			final RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
-					new RyaURI("http://Amanda"), new RyaURI("http://context"), "", metadata1);
-			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);
+            final RyaStatement statement1 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://worksAt"),
+                    new RyaURI("http://BurgerShack"), new RyaURI("http://context"), "", metadata1);
+            final RyaStatement statement2 = new RyaStatement(new RyaURI("http://Joe"), new RyaURI("http://talksTo"),
+                    new RyaURI("http://Betty"), new RyaURI("http://context"), "", metadata1);
+            final RyaStatement statement3 = new RyaStatement(new RyaURI("http://Fred"), new RyaURI("http://talksTo"),
+                    new RyaURI("http://Amanda"), new RyaURI("http://context"), "", metadata1);
+            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);
 
-			SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-			final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
+            SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+            final TupleQueryResult result = conn.prepareTupleQuery(QueryLanguage.SPARQL, query2).evaluate();
 
-			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);
+            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);
 
-			final Set<BindingSet> bsSet = new HashSet<>();
-			while (result.hasNext()) {
-				bsSet.add(result.next());
-			}
+            final Set<BindingSet> bsSet = new HashSet<>();
+            while (result.hasNext()) {
+                bsSet.add(result.next());
+            }
 
-			assertEquals(expected, bsSet);
+            assertEquals(expected, bsSet);
 
-			dao.delete(statement1, conf);
-			dao.delete(statement2, conf);
-			dao.delete(statement3, conf);
-			dao.delete(statement4, conf);
-		} finally {
-			dao.destroy();
-			sail.shutDown();
-		}
-	}
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+            dao.delete(statement3, conf);
+            dao.delete(statement4, conf);
+        } finally {
+            dao.destroy();
+            sail.shutDown();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 aa700b1..a45197c 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
@@ -48,321 +48,321 @@ import org.openrdf.query.parser.sparql.SPARQLParser;
 import info.aduna.iteration.CloseableIteration;
 
 public class MongoStatementMetadataNodeTest extends MongoTestBase {
-	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 {
-		final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-		conf.setUseStatementMetadata(true);
-		conf.setStatementMetadataProperties(propertySet);
-	}
-
-	@Test
-	public void simpleQueryWithoutBindingSet() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-
-			StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-			QueryBindingSet bs = new QueryBindingSet();
-			bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-			bs.addBinding("y", new LiteralImpl("Joe"));
-
-			List<BindingSet> bsList = new ArrayList<>();
-			while (iteration.hasNext()) {
-				bsList.add(iteration.next());
-			}
-
-			Assert.assertEquals(1, bsList.size());
-			Assert.assertEquals(bs, bsList.get(0));
-			dao.delete(statement, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
-
-	/**
-	 * 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.
-	 */
-	@Test
-	public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();	
-
-			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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-			List<BindingSet> bsList = new ArrayList<>();
-			while (iteration.hasNext()) {
-				bsList.add(iteration.next());
-			}
-			Assert.assertEquals(0, bsList.size());
-			dao.delete(statement, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
-
-	@Test
-	public void simpleQueryWithBindingSet() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			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);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-			QueryBindingSet bsConstraint = new QueryBindingSet();
-			bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-			bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
-
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-			QueryBindingSet expected = new QueryBindingSet();
-			expected.addBinding("x", new LiteralImpl("CoffeeShop"));
-			expected.addBinding("y", new LiteralImpl("Joe"));
-			expected.addBinding("z", new LiteralImpl("Virginia"));
-
-			List<BindingSet> bsList = new ArrayList<>();
-			while (iteration.hasNext()) {
-				bsList.add(iteration.next());
-			}
-
-			Assert.assertEquals(1, bsList.size());
-			Assert.assertEquals(expected, bsList.get(0));
-
-			dao.delete(statement1, conf);
-			dao.delete(statement2, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
-
-	/**
-	 * Tests to see if correct result is passed back when a metadata statement
-	 * is joined with a StatementPattern statement (i.e. a common variable
-	 * appears in a StatementPattern statement and a metadata statement).
-	 * StatementPattern statements have either rdf:subject, rdf:predicate, or
-	 * rdf:object as the predicate while a metadata statement is any statement
-	 * in the reified query whose predicate is not rdf:type and not a
-	 * StatementPattern predicate.
-	 */
-	@Test
-	public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			StatementMetadata metadata = new StatementMetadata();
-			metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
-					new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement1);
-			dao.add(statement2);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query2, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-			List<BindingSet> bsCollection = new ArrayList<>();
-			QueryBindingSet bsConstraint1 = new QueryBindingSet();
-			bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
-			bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-			QueryBindingSet bsConstraint2 = new QueryBindingSet();
-			bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
-			bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-			bsCollection.add(bsConstraint1);
-			bsCollection.add(bsConstraint2);
-
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-			QueryBindingSet expected = new QueryBindingSet();
-			expected.addBinding("y", new LiteralImpl("CoffeeShop"));
-			expected.addBinding("x", new URIImpl("http://Joe"));
-			expected.addBinding("z", new LiteralImpl("Virginia"));
-
-			List<BindingSet> bsList = new ArrayList<>();
-			while (iteration.hasNext()) {
-				bsList.add(iteration.next());
-			}
-
-			Assert.assertEquals(1, bsList.size());
-			Assert.assertEquals(expected, bsList.get(0));
-
-			dao.delete(statement1, conf);
-			dao.delete(statement2, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
-
-	/**
-	 * Tests if the StatementMetadataNode joins BindingSet correctly for
-	 * variables appearing in metadata statements. In this case, the metadata
-	 * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
-	 * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
-	 * object in the above metadata statement and its values are joined to the
-	 * constraint BindingSets in the example below.
-	 */
-	@Test
-	public void simpleQueryWithBindingSetJoinOnProperty() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			dao.add(statement1);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-			QueryBindingSet bsConstraint = new QueryBindingSet();
-			bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-			bsConstraint.addBinding("y", new LiteralImpl("Doug"));
-
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-			List<BindingSet> bsList = new ArrayList<>();
-			while (iteration.hasNext()) {
-				bsList.add(iteration.next());
-			}
-
-			Assert.assertEquals(0, bsList.size());
-			dao.delete(statement1, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
-
-	/**
-	 * Tests if StatementMetadataNode joins BindingSet values correctly for
-	 * variables appearing as the object in one of the StatementPattern
-	 * statements (in the case ?x appears as the Object in the statement
-	 * _:blankNode rdf:object ?x). StatementPattern statements have either
-	 * rdf:subject, rdf:predicate, or rdf:object as the predicate.
-	 */
-	@Test
-	public void simpleQueryWithBindingSetCollection() throws Exception {
-		MongoDBRyaDAO dao = new MongoDBRyaDAO();
-		try {
-			dao.setConf(conf);
-			dao.init();
-			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"),
-					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-			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);
-
-			SPARQLParser parser = new SPARQLParser();
-			ParsedQuery pq = parser.parseQuery(query, null);
-			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-			List<BindingSet> bsCollection = new ArrayList<>();
-			QueryBindingSet bsConstraint1 = new QueryBindingSet();
-			bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
-			bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-			QueryBindingSet bsConstraint2 = new QueryBindingSet();
-			bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
-			bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-
-			QueryBindingSet bsConstraint3 = new QueryBindingSet();
-			bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
-			bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
-			bsCollection.add(bsConstraint1);
-			bsCollection.add(bsConstraint2);
-			bsCollection.add(bsConstraint3);
-
-			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-			Set<BindingSet> expected = new HashSet<>();
-			QueryBindingSet expected1 = new QueryBindingSet();
-			expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
-			expected1.addBinding("y", new LiteralImpl("Joe"));
-			expected1.addBinding("z", new LiteralImpl("Virginia"));
-
-			QueryBindingSet expected2 = new QueryBindingSet();
-			expected2.addBinding("x", new LiteralImpl("HardwareStore"));
-			expected2.addBinding("y", new LiteralImpl("Joe"));
-			expected2.addBinding("z", new LiteralImpl("Maryland"));
-			expected.add(expected1);
-			expected.add(expected2);
-
-			Set<BindingSet> bsSet = new HashSet<>();
-			while (iteration.hasNext()) {
-				bsSet.add(iteration.next());
-			}
-
-			Assert.assertEquals(expected, bsSet);
-
-			dao.delete(statement1, conf);
-			dao.delete(statement2, conf);
-		} finally {
-			dao.destroy();
-		}
-	}
+    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 {
+        final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+        conf.setUseStatementMetadata(true);
+        conf.setStatementMetadataProperties(propertySet);
+    }
+
+    @Test
+    public void simpleQueryWithoutBindingSet() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+
+            StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+            QueryBindingSet bs = new QueryBindingSet();
+            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bs.addBinding("y", new LiteralImpl("Joe"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(bs, bsList.get(0));
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * 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.
+     */
+    @Test
+    public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();	
+
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+            Assert.assertEquals(0, bsList.size());
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    @Test
+    public void simpleQueryWithBindingSet() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            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);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            QueryBindingSet bsConstraint = new QueryBindingSet();
+            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+            QueryBindingSet expected = new QueryBindingSet();
+            expected.addBinding("x", new LiteralImpl("CoffeeShop"));
+            expected.addBinding("y", new LiteralImpl("Joe"));
+            expected.addBinding("z", new LiteralImpl("Virginia"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(expected, bsList.get(0));
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests to see if correct result is passed back when a metadata statement
+     * is joined with a StatementPattern statement (i.e. a common variable
+     * appears in a StatementPattern statement and a metadata statement).
+     * StatementPattern statements have either rdf:subject, rdf:predicate, or
+     * rdf:object as the predicate while a metadata statement is any statement
+     * in the reified query whose predicate is not rdf:type and not a
+     * StatementPattern predicate.
+     */
+    @Test
+    public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            StatementMetadata metadata = new StatementMetadata();
+            metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
+                    new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement1);
+            dao.add(statement2);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query2, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            List<BindingSet> bsCollection = new ArrayList<>();
+            QueryBindingSet bsConstraint1 = new QueryBindingSet();
+            bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
+            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet bsConstraint2 = new QueryBindingSet();
+            bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
+            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+            bsCollection.add(bsConstraint1);
+            bsCollection.add(bsConstraint2);
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+            QueryBindingSet expected = new QueryBindingSet();
+            expected.addBinding("y", new LiteralImpl("CoffeeShop"));
+            expected.addBinding("x", new URIImpl("http://Joe"));
+            expected.addBinding("z", new LiteralImpl("Virginia"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(expected, bsList.get(0));
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests if the StatementMetadataNode joins BindingSet correctly for
+     * variables appearing in metadata statements. In this case, the metadata
+     * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
+     * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
+     * object in the above metadata statement and its values are joined to the
+     * constraint BindingSets in the example below.
+     */
+    @Test
+    public void simpleQueryWithBindingSetJoinOnProperty() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement1);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            QueryBindingSet bsConstraint = new QueryBindingSet();
+            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint.addBinding("y", new LiteralImpl("Doug"));
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(0, bsList.size());
+            dao.delete(statement1, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests if StatementMetadataNode joins BindingSet values correctly for
+     * variables appearing as the object in one of the StatementPattern
+     * statements (in the case ?x appears as the Object in the statement
+     * _:blankNode rdf:object ?x). StatementPattern statements have either
+     * rdf:subject, rdf:predicate, or rdf:object as the predicate.
+     */
+    @Test
+    public void simpleQueryWithBindingSetCollection() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            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);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            List<BindingSet> bsCollection = new ArrayList<>();
+            QueryBindingSet bsConstraint1 = new QueryBindingSet();
+            bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet bsConstraint2 = new QueryBindingSet();
+            bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
+            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+
+            QueryBindingSet bsConstraint3 = new QueryBindingSet();
+            bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
+            bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
+            bsCollection.add(bsConstraint1);
+            bsCollection.add(bsConstraint2);
+            bsCollection.add(bsConstraint3);
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+            Set<BindingSet> expected = new HashSet<>();
+            QueryBindingSet expected1 = new QueryBindingSet();
+            expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
+            expected1.addBinding("y", new LiteralImpl("Joe"));
+            expected1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet expected2 = new QueryBindingSet();
+            expected2.addBinding("x", new LiteralImpl("HardwareStore"));
+            expected2.addBinding("y", new LiteralImpl("Joe"));
+            expected2.addBinding("z", new LiteralImpl("Maryland"));
+            expected.add(expected1);
+            expected.add(expected2);
+
+            Set<BindingSet> bsSet = new HashSet<>();
+            while (iteration.hasNext()) {
+                bsSet.add(iteration.next());
+            }
+
+            Assert.assertEquals(expected, bsSet);
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/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 66d94db..addbd18 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -57,9 +57,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.config.IMongoConfig;
+import de.flapdoodle.embed.mongo.config.Net;
 import info.aduna.iteration.Iterations;
 
 
@@ -295,12 +294,11 @@ public class MongoRyaDirectExample {
             .setUseMockMongo(USE_MOCK).setUseInference(USE_INFER).setAuths("U");
 
         if (USE_MOCK) {
-        	final EmbeddedMongoFactory factory = EmbeddedMongoFactory.newFactory();
-            final MongoClient c = factory.newMongoClient();
+            final EmbeddedMongoFactory factory = EmbeddedMongoFactory.newFactory();
             final IMongoConfig connectionConfig = factory.getMongoServerDetails();
-            //c.close();
-            builder.setMongoHost(connectionConfig.net().getServerAddress().getHostAddress())
-            	   .setMongoPort(connectionConfig.net().getPort() + "");
+            Net net = connectionConfig.net();
+            builder.setMongoHost(net.getServerAddress().getHostAddress())
+                   .setMongoPort(net.getPort() + "");
         } else {
             // User name and password must be filled in:
             builder = builder.setMongoUser(MONGO_USER)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
index 2f7dca7..536ea60 100644
--- a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
+++ b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
@@ -20,7 +20,6 @@ package org.apache.rya.export.client.merge;
 
 import static java.util.Objects.requireNonNull;
 
-import java.util.ArrayList;
 import java.util.Date;
 
 import org.apache.rya.accumulo.AccumuloRyaDAO;
@@ -123,7 +122,7 @@ public class StatementStoreFactory {
     private MongoRyaStatementStore getBaseMongoStore(final String hostname, final int port, final String ryaInstanceName) throws RyaDAOException {
         final MongoClient client = new MongoClient(hostname, port);
         final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        dao.setConf(new StatefulMongoDBRdfConfiguration(MergeConfigHadoopAdapter.getMongoConfiguration(configuration), client, new ArrayList<>()));
+        dao.setConf(new StatefulMongoDBRdfConfiguration(MergeConfigHadoopAdapter.getMongoConfiguration(configuration), client));
         dao.init();
         return new MongoRyaStatementStore(client, ryaInstanceName, dao);
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
index c982d21..9b6c0c0 100644
--- a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
+++ b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
@@ -67,7 +67,7 @@ public class StoreToStoreIT extends ITBase {
     private static TimestampPolicyMongoRyaStatementStore getParentMongo() throws Exception {
         final MongoClient mongo = getNewMongoResources(RYA_INSTANCE);
         final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo, new ArrayList<>()));
+        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo));
         dao.init();
         final MongoRyaStatementStore store = new MongoRyaStatementStore(mongo, RYA_INSTANCE, dao);
         final TimestampPolicyMongoRyaStatementStore timeStore = new TimestampPolicyMongoRyaStatementStore(store, currentDate, RYA_INSTANCE);
@@ -78,7 +78,7 @@ public class StoreToStoreIT extends ITBase {
     private static MongoRyaStatementStore getChildMongo() throws Exception {
         final MongoClient mongo = getNewMongoResources(RYA_INSTANCE);
         final MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo, new ArrayList<>()));
+        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo));
         dao.init();
         final MongoRyaStatementStore store = new MongoRyaStatementStore(mongo, RYA_INSTANCE, dao);
         clients.add(mongo);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
index eee7310..a425ce4 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
@@ -91,35 +91,35 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
         this.conf = conf;
         //reset the init.
         init = false;
-            init();
+        init();
     }
     /**
      * Load instances of the selected indexers.  This is tricky because some (geomesa vs geowave) have incompatible dependencies (geotools versions).
      */
     private synchronized void init() {
         if (!init) {
-			if (ConfigUtils.getUseMongo(conf)) {
-				// create a new MongoGeoIndexer() without having it at compile time.
-				StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
-            	for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
-        			if(indexer instanceof FreeTextIndexer) {
-        				freeTextIndexer = (FreeTextIndexer) indexer;
-        			} else if(indexer instanceof TemporalIndexer) {
-        				temporalIndexer = (TemporalIndexer) indexer;
-        			} else if(indexer instanceof GeoIndexer) {
-        				geoIndexer = (GeoIndexer) indexer;
-        			}
-            	}
-			} else {
-				GeoIndexerType geoIndexerType = OptionalConfigUtils.getGeoIndexerType(conf);
-				if (geoIndexerType == GeoIndexerType.UNSPECIFIED) {
-					geoIndexer = instantiate(GeoIndexerType.GEO_MESA.getGeoIndexerClassString(), GeoIndexer.class);
-				} else {
-					geoIndexer = instantiate(geoIndexerType.getGeoIndexerClassString(), GeoIndexer.class);
-				}
-				geoIndexer.setConf(conf);
-				freeTextIndexer = new AccumuloFreeTextIndexer();
-				freeTextIndexer.setConf(conf);
+            if (ConfigUtils.getUseMongo(conf)) {
+                // create a new MongoGeoIndexer() without having it at compile time.
+                StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
+                for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
+                    if(indexer instanceof FreeTextIndexer) {
+                        freeTextIndexer = (FreeTextIndexer) indexer;
+                    } else if(indexer instanceof TemporalIndexer) {
+                        temporalIndexer = (TemporalIndexer) indexer;
+                    } else if(indexer instanceof GeoIndexer) {
+                        geoIndexer = (GeoIndexer) indexer;
+                    }
+                }
+            } else {
+                GeoIndexerType geoIndexerType = OptionalConfigUtils.getGeoIndexerType(conf);
+                if (geoIndexerType == GeoIndexerType.UNSPECIFIED) {
+                    geoIndexer = instantiate(GeoIndexerType.GEO_MESA.getGeoIndexerClassString(), GeoIndexer.class);
+                } else {
+                    geoIndexer = instantiate(geoIndexerType.getGeoIndexerClassString(), GeoIndexer.class);
+                }
+                geoIndexer.setConf(conf);
+                freeTextIndexer = new AccumuloFreeTextIndexer();
+                freeTextIndexer.setConf(conf);
                 temporalIndexer = new AccumuloTemporalIndexer();
                 temporalIndexer.setConf(conf);
             }
@@ -127,10 +127,10 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
         }
     }
 
-    
+
     @Override
     public void optimize(final TupleExpr tupleExpr, final Dataset dataset, final BindingSet bindings) {
-     // find variables used in property and resource based searches:
+        // find variables used in property and resource based searches:
         final SearchVarVisitor searchVars = new SearchVarVisitor();
         tupleExpr.visit(searchVars);
         // rewrites for property searches:
@@ -148,8 +148,8 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
         try{
             return type.cast(Class.forName(className).newInstance());
         } catch(InstantiationException
-              | IllegalAccessException
-              | ClassNotFoundException e){
+                | IllegalAccessException
+                | ClassNotFoundException e){
             throw new IllegalStateException(e);
         }
     }
@@ -191,7 +191,7 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
 
     //find vars contained in filters
     private static class SearchVarVisitor extends QueryModelVisitorBase<RuntimeException> {
-        private final Collection<Var> searchProperties = new ArrayList<Var>();
+        private final Collection<Var> searchProperties = new ArrayList<>();
 
         @Override
         public void meet(final FunctionCall fn) {
@@ -206,8 +206,8 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
     //find StatementPatterns containing filter variables
     private static class MatchStatementVisitor extends QueryModelVisitorBase<RuntimeException> {
         private final Collection<Var> propertyVars;
-        private final Collection<Var> usedVars = new ArrayList<Var>();
-        private final List<StatementPattern> matchStatements = new ArrayList<StatementPattern>();
+        private final Collection<Var> usedVars = new ArrayList<>();
+        private final List<StatementPattern> matchStatements = new ArrayList<>();
 
         public MatchStatementVisitor(final Collection<Var> propertyVars) {
             this.propertyVars = propertyVars;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
index ae6af10..c53fea1 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
@@ -86,29 +86,28 @@ public class GeoRyaSailFactory {
         Objects.requireNonNull(ryaInstance, "RyaInstance or table prefix is missing from configuration."+RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
 
         if(ConfigUtils.getUseMongo(config)) {
-        	// Get a reference to a Mongo DB configuration object.
+            // Get a reference to a Mongo DB configuration object.
             final MongoDBRdfConfiguration mongoConfig = (config instanceof MongoDBRdfConfiguration) ?
                     (MongoDBRdfConfiguration)config : new MongoDBRdfConfiguration(config);
 
             // Create the MongoClient that will be used by the Sail object's components.
             final MongoClient client = createMongoClient(mongoConfig);
-            
+
             // Add the Indexer and Optimizer names to the configuration object that are configured to be used.
             OptionalConfigUtils.setIndexers(mongoConfig);
-            
-            // Initialize the indexer and optimizer objects that will be used within the Sail object.
-            final List<MongoSecondaryIndex> indexers = mongoConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
 
             // Populate the configuration using previously stored Rya Details if this instance uses them.
             try {
-                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstance());
+                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstanceName());
                 RyaDetailsToConfiguration.addRyaDetailsToConfiguration(ryaDetailsRepo.getRyaInstanceDetails(), mongoConfig);
             } catch (final RyaDetailsRepositoryException e) {
                 LOG.info("Instance does not have a rya details collection, skipping.");
             }
 
             // Set the configuration to the stateful configuration that is used to pass the constructed objects around.
-            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client, indexers);
+            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client);
+            final List<MongoSecondaryIndex> indexers = statefulConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+            statefulConfig.setIndexers(indexers);
             rdfConfig = statefulConfig;
 
             // Create the DAO that is able to interact with MongoDB.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/pom.xml b/extras/rya.geoindexing/geo.mongo/pom.xml
index 261f061..154fd4e 100644
--- a/extras/rya.geoindexing/geo.mongo/pom.xml
+++ b/extras/rya.geoindexing/geo.mongo/pom.xml
@@ -46,8 +46,8 @@
         <dependency>
             <groupId>org.apache.rya</groupId>
             <artifactId>mongodb.rya</artifactId>
-			<type>test-jar</type>
-			<scope>test</scope>            
+            <type>test-jar</type>
+            <scope>test</scope>
         </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
index 62a2181..ce6d653 100644
--- a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
+++ b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
@@ -92,9 +92,9 @@ public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMon
     @Override
     public void setConf(final Configuration conf) {
         requireNonNull(conf);
-    	checkArgument(conf instanceof StatefulMongoDBRdfConfiguration, 
-    			"The configuration must be of type StatefulMongoDBRdfConfiguration but you provided: " 
-    		    + conf.getClass().getSimpleName());
+        checkArgument(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The configuration must be of type StatefulMongoDBRdfConfiguration but you provided: "
+                        + conf.getClass().getSimpleName());
         super.conf = (StatefulMongoDBRdfConfiguration) conf;
     }
 
@@ -214,7 +214,7 @@ public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMon
             return events.get();
         }
 
-        events.set(new MongoEventStorage(conf.getMongoClient(), conf.getRyaInstance()));
+        events.set(new MongoEventStorage(conf.getMongoClient(), conf.getRyaInstanceName()));
         return events.get();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
index cff0996..9c28bf0 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
@@ -51,135 +51,130 @@ import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
 public class MongoGeoTemporalIndexIT extends MongoTestBase {
-	private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
-
-	private static final ValueFactory VF = ValueFactoryImpl.getInstance();
-	@Override
-	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
-		//        mongoClient = super.getMongoClient();
-		//        conf = new MongoDBRdfConfiguration();
-		//        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, MongoGeoTemporalIndexIT.class.getSimpleName() + "_" + COUNTER.getAndIncrement());
-		//        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya");
-		//        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya");
-		//        conf.setBoolean(ConfigUtils.USE_MONGO, true);
-		conf.setBoolean(OptionalConfigUtils.USE_GEOTEMPORAL, true);
-
-		//        conn = new SailRepository(sail).getConnection();
-		//        conn.begin();
-
-	}
-
-	@Test
-	public void ensureInEventStore_Test() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepository repo = new SailRepository(sail);
-		try(final MongoGeoTemporalIndexer indexer = new MongoGeoTemporalIndexer()) {
-			indexer.setConf(conf);
-			indexer.init();
-
-			addStatements(repo.getConnection());
-			final EventStorage events = indexer.getEventStorage();
-			final RyaURI subject = new RyaURI("urn:event1");
-			final Optional<Event> event = events.get(subject);
-			assertTrue(event.isPresent());
-		} finally {
-			sail.shutDown();
-		}
-	}
-
-	@Test
-	public void constantSubjQuery_Test() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-
-		try {
-			final String query =
-					"PREFIX time: <http://www.w3.org/2006/time#> \n"
-							+ "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-							+ "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-							+ "SELECT * "
-							+ "WHERE { "
-							+ "  <urn:event1> time:atTime ?time . "
-							+ "  <urn:event1> geo:asWKT ?point . "
-							+ "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-							+ "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-							+ "}";
-
-			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			final Set<BindingSet> results = new HashSet<>();
-			while(rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			final MapBindingSet expected = new MapBindingSet();
-			expected.addBinding("point", VF.createLiteral("POINT (0 0)"));
-			expected.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-			assertEquals(1, results.size());
-			assertEquals(expected, results.iterator().next());
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	@Test
-	public void variableSubjQuery_Test() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-
-		try {
-			final String query =
-					"PREFIX time: <http://www.w3.org/2006/time#> \n"
-							+ "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-							+ "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-							+ "SELECT * "
-							+ "WHERE { "
-							+ "  ?subj time:atTime ?time . "
-							+ "  ?subj geo:asWKT ?point . "
-							+ "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-							+ "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-							+ "}";
-
-			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			final List<BindingSet> results = new ArrayList<>();
-			while(rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			final MapBindingSet expected1 = new MapBindingSet();
-			expected1.addBinding("point", VF.createLiteral("POINT (0 0)"));
-			expected1.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-			final MapBindingSet expected2 = new MapBindingSet();
-			expected2.addBinding("point", VF.createLiteral("POINT (1 1)"));
-			expected2.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-			assertEquals(2, results.size());
-			assertEquals(expected1, results.get(0));
-			assertEquals(expected2, results.get(1));
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	private void addStatements(final SailRepositoryConnection conn) throws Exception {
-		URI subject = VF.createURI("urn:event1");
-		final URI predicate = VF.createURI(URI_PROPERTY_AT_TIME);
-		Value object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
-		conn.add(VF.createStatement(subject, predicate, object));
-
-		object = VF.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
-		conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
-
-		subject = VF.createURI("urn:event2");
-		object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
-		conn.add(VF.createStatement(subject, predicate, object));
-
-		object = VF.createLiteral("Point(1 1)", GeoConstants.XMLSCHEMA_OGC_WKT);
-		conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
-	}
+    private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
+
+    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Override
+    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.setBoolean(OptionalConfigUtils.USE_GEOTEMPORAL, true);
+    }
+
+    @Test
+    public void ensureInEventStore_Test() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepository repo = new SailRepository(sail);
+        try(final MongoGeoTemporalIndexer indexer = new MongoGeoTemporalIndexer()) {
+            indexer.setConf(conf);
+            indexer.init();
+
+            addStatements(repo.getConnection());
+            final EventStorage events = indexer.getEventStorage();
+            final RyaURI subject = new RyaURI("urn:event1");
+            final Optional<Event> event = events.get(subject);
+            assertTrue(event.isPresent());
+        } finally {
+            sail.shutDown();
+        }
+    }
+
+    @Test
+    public void constantSubjQuery_Test() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+
+        try {
+            addStatements(conn);
+
+            final String query =
+                    "PREFIX time: <http://www.w3.org/2006/time#> \n"
+                            + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+                            + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+                            + "SELECT * "
+                            + "WHERE { "
+                            + "  <urn:event1> time:atTime ?time . "
+                            + "  <urn:event1> geo:asWKT ?point . "
+                            + "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                            + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+                            + "}";
+
+            final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            final Set<BindingSet> results = new HashSet<>();
+            while(rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            final MapBindingSet expected = new MapBindingSet();
+            expected.addBinding("point", VF.createLiteral("POINT (0 0)"));
+            expected.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+            assertEquals(1, results.size());
+            assertEquals(expected, results.iterator().next());
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    @Test
+    public void variableSubjQuery_Test() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+
+        try {
+            addStatements(conn);
+
+            final String query =
+                    "PREFIX time: <http://www.w3.org/2006/time#> \n"
+                            + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+                            + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+                            + "SELECT * "
+                            + "WHERE { "
+                            + "  ?subj time:atTime ?time . "
+                            + "  ?subj geo:asWKT ?point . "
+                            + "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                            + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+                            + "}";
+
+            final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            final List<BindingSet> results = new ArrayList<>();
+            while(rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            final MapBindingSet expected1 = new MapBindingSet();
+            expected1.addBinding("point", VF.createLiteral("POINT (0 0)"));
+            expected1.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+            final MapBindingSet expected2 = new MapBindingSet();
+            expected2.addBinding("point", VF.createLiteral("POINT (1 1)"));
+            expected2.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+            assertEquals(2, results.size());
+            assertEquals(expected1, results.get(0));
+            assertEquals(expected2, results.get(1));
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    private void addStatements(final SailRepositoryConnection conn) throws Exception {
+        URI subject = VF.createURI("urn:event1");
+        final URI predicate = VF.createURI(URI_PROPERTY_AT_TIME);
+        Value object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
+        conn.add(VF.createStatement(subject, predicate, object));
+
+        object = VF.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+        conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
+
+        subject = VF.createURI("urn:event2");
+        object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
+        conn.add(VF.createStatement(subject, predicate, object));
+
+        object = VF.createLiteral("Point(1 1)", GeoConstants.XMLSCHEMA_OGC_WKT);
+        conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
+    }
 }


[02/17] incubator-rya git commit: RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
index e156f86..1ec8543 100644
--- a/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
+++ b/extras/indexing/src/main/java/org/apache/rya/sail/config/RyaSailFactory.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -20,12 +20,14 @@ package org.apache.rya.sail.config;
 
 import static java.util.Objects.requireNonNull;
 
-import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Objects;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.commons.configuration.ConfigurationRuntimeException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.AccumuloRyaDAO;
@@ -37,9 +39,10 @@ import org.apache.rya.api.layout.TablePrefixLayoutStrategy;
 import org.apache.rya.api.persist.RyaDAO;
 import org.apache.rya.api.persist.RyaDAOException;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
 import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
@@ -50,6 +53,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.mongodb.MongoClient;
+import com.mongodb.MongoCredential;
+import com.mongodb.MongoException;
+import com.mongodb.ServerAddress;
 
 public class RyaSailFactory {
     private static final Logger LOG = LoggerFactory.getLogger(RyaSailFactory.class);
@@ -79,16 +85,38 @@ public class RyaSailFactory {
         Objects.requireNonNull(ryaInstance, "RyaInstance or table prefix is missing from configuration."+RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
 
         if(ConfigUtils.getUseMongo(config)) {
-            final MongoDBRdfConfiguration mongoConfig = new MongoDBRdfConfiguration(config);
-            rdfConfig = mongoConfig;
-            final MongoClient client = MongoConnectorFactory.getMongoClient(config);
+            // Get a reference to a Mongo DB configuration object.
+            final MongoDBRdfConfiguration mongoConfig = (config instanceof MongoDBRdfConfiguration) ?
+                    (MongoDBRdfConfiguration)config : new MongoDBRdfConfiguration(config);
+
+            // Create the MongoClient that will be used by the Sail object's components.
+            final MongoClient client = createMongoClient(mongoConfig);
+
+            // Add the Indexer and Optimizer names to the configuration object that are configured to be used.
+            ConfigUtils.setIndexers(mongoConfig);
+
+            // Initialize the indexer and optimizer objects that will be used within the Sail object.
+            final List<MongoSecondaryIndex> indexers = mongoConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+            // TODO Optimizers the same way. They're getting the wrong configuration somehow.
+
+            // Populate the configuration using previously stored Rya Details if this instance uses them.
             try {
-                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getCollectionName());
+                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstance());
                 RyaDetailsToConfiguration.addRyaDetailsToConfiguration(ryaDetailsRepo.getRyaInstanceDetails(), mongoConfig);
             } catch (final RyaDetailsRepositoryException e) {
                LOG.info("Instance does not have a rya details collection, skipping.");
-           }
-            dao = getMongoDAO((MongoDBRdfConfiguration)rdfConfig, client);
+            }
+
+            // Set the configuration to the stateful configuration that is used to pass the constructed objects around.
+            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client, indexers);
+            rdfConfig = statefulConfig;
+
+            // Create the DAO that is able to interact with MongoDB.
+            final MongoDBRyaDAO mongoDao = new MongoDBRyaDAO();
+            mongoDao.setConf(statefulConfig);
+            mongoDao.init();
+            dao = mongoDao;
+
         } else {
             rdfConfig = new AccumuloRdfConfiguration(config);
             user = rdfConfig.get(ConfigUtils.CLOUDBASE_USER);
@@ -115,20 +143,39 @@ public class RyaSailFactory {
         return store;
     }
 
-    private static MongoDBRyaDAO getMongoDAO(final MongoDBRdfConfiguration config, final MongoClient client) throws RyaDAOException {
-        MongoDBRyaDAO dao = null;
-        ConfigUtils.setIndexers(config);
-        if(client != null) {
-            dao = new MongoDBRyaDAO(config, client);
+    /**
+     * Create a {@link MongoClient} that is connected to the configured database.
+     *
+     * @param mongoConf - Configures what will be connected to. (not null)
+     * @throws ConfigurationRuntimeException An invalid port was provided by {@code mongoConf}.
+     * @throws MongoException Couldn't connect to the MongoDB database.
+     */
+    private static MongoClient createMongoClient(final MongoDBRdfConfiguration mongoConf) throws ConfigurationRuntimeException, MongoException {
+        requireNonNull(mongoConf);
+        requireNonNull(mongoConf.getMongoHostname());
+        requireNonNull(mongoConf.getMongoPort());
+        requireNonNull(mongoConf.getMongoDBName());
+
+        // Connect to a running MongoDB server.
+        final int port;
+        try {
+            port = Integer.parseInt( mongoConf.getMongoPort() );
+        } catch(final NumberFormatException e) {
+            throw new ConfigurationRuntimeException("Port '" + mongoConf.getMongoPort() + "' must be an integer.");
+        }
+
+        final ServerAddress server = new ServerAddress(mongoConf.getMongoHostname(), port);
+
+        // Connect to a specific MongoDB Database if that information is provided.
+        final String username = mongoConf.getMongoUser();
+        final String database = mongoConf.getMongoDBName();
+        final String password = mongoConf.getMongoPassword();
+        if(username != null && password != null) {
+            final MongoCredential cred = MongoCredential.createCredential(username, database, password.toCharArray());
+            return new MongoClient(server, Arrays.asList(cred));
         } else {
-            try {
-                dao = new MongoDBRyaDAO(config);
-            } catch (NumberFormatException | UnknownHostException e) {
-                throw new RyaDAOException("Unable to connect to mongo at the configured location.", e);
-            }
+            return new MongoClient(server);
         }
-        dao.init();
-        return dao;
     }
 
     /**
@@ -137,7 +184,7 @@ public class RyaSailFactory {
      * tables might be created when using this method.  This method does not require the {@link AccumuloRyaInstanceDetailsRepository}
      * to exist.  This is for internal use, backwards compatibility and testing purposes only.  It is recommended that
      * {@link RyaSailFactory#getAccumuloDAOWithUpdatedConfig(AccumuloRdfConfiguration)} be used for new installations of Rya.
-     * 
+     *
      * @param config - user configuration
      * @return - AccumuloRyaDAO with Indexers configured according to user's specification
      * @throws AccumuloException
@@ -155,14 +202,14 @@ public class RyaSailFactory {
         dao.init();
         return dao;
     }
-    
+
     /**
      * Creates an AccumuloRyaDAO after updating the AccumuloRdfConfiguration so that it is consistent
      * with the configuration of the RyaInstance that the user is trying to connect to.  This ensures
-     * that user configuration aligns with Rya instance configuration and prevents the creation of 
+     * that user configuration aligns with Rya instance configuration and prevents the creation of
      * new index tables based on a user's query configuration.  This method requires the {@link AccumuloRyaInstanceDetailsRepository}
      * to exist.
-     * 
+     *
      * @param config - user's query configuration
      * @return - AccumuloRyaDAO with an updated configuration that is consistent with the Rya instance configuration
      * @throws AccumuloException
@@ -170,16 +217,16 @@ public class RyaSailFactory {
      * @throws RyaDAOException
      */
     public static AccumuloRyaDAO getAccumuloDAOWithUpdatedConfig(final AccumuloRdfConfiguration config) throws AccumuloException, AccumuloSecurityException, RyaDAOException {
-        
-        String ryaInstance = config.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
+
+        final String ryaInstance = config.get(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
         Objects.requireNonNull(ryaInstance, "RyaInstance or table prefix is missing from configuration."+RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
-        String user = config.get(AccumuloRdfConfiguration.CLOUDBASE_USER);
-        String pswd = config.get(AccumuloRdfConfiguration.CLOUDBASE_PASSWORD);
+        final String user = config.get(AccumuloRdfConfiguration.CLOUDBASE_USER);
+        final String pswd = config.get(AccumuloRdfConfiguration.CLOUDBASE_PASSWORD);
         Objects.requireNonNull(user, "Accumulo user name is missing from configuration."+AccumuloRdfConfiguration.CLOUDBASE_USER);
         Objects.requireNonNull(pswd, "Accumulo user password is missing from configuration."+AccumuloRdfConfiguration.CLOUDBASE_PASSWORD);
         config.setTableLayoutStrategy( new TablePrefixLayoutStrategy(ryaInstance) );
         updateAccumuloConfig(config, user, pswd, ryaInstance);
-        
+
         return getAccumuloDAO(config);
     }
 
@@ -192,4 +239,62 @@ public class RyaSailFactory {
             LOG.info("Instance does not have a rya details collection, skipping.");
         }
     }
-}
\ No newline at end of file
+}
+
+
+
+///**
+//* TODO add docs.  names for reflection
+//* @param indexers
+//*/
+//public void setMongoIndexers(final Class<? extends MongoSecondaryIndex>... indexers) {
+// final List<String> strs = Lists.newArrayList();
+// for (final Class<?> ai : indexers){
+//     strs.add(ai.getName());
+// }
+//
+// setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
+//}
+
+///**
+//* TODO add docs. explain hack is used here. do reflection. eww.
+//* @return
+//*/
+//public List<MongoSecondaryIndex> getAdditionalIndexers() {
+// stateLock.lock();
+// try {
+//     if(indexers == null) {
+//         indexers = getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+//     }
+//     return indexers;
+// } finally {
+//     stateLock.unlock();
+// }
+//}
+
+//// XXX Not sure what all of this stuff is for. I'm guessing Rya Sail state stuff.
+//public void setAdditionalIndexers(final Class<? extends MongoSecondaryIndex>... indexers) {
+//  final List<String> strs = Lists.newArrayList();
+//  for (final Class<?> ai : indexers){
+//      strs.add(ai.getName());
+//  }
+//
+//  setStrings(CONF_ADDITIONAL_INDEXERS, strs.toArray(new String[]{}));
+//}
+//
+
+//conf.setStrings(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, indexList.toArray(new String[] {}));
+//conf.setStrings(RdfCloudTripleStoreConfiguration.CONF_OPTIMIZERS, optimizers.toArray(new String[] {}));
+
+//public List<MongoSecondaryIndex> getAdditionalIndexers() {
+//  return getInstances(CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+//}
+
+//public void setMongoClient(final MongoClient client) {
+//  requireNonNull(client);
+//  this.mongoClient = client;
+//}
+//
+//public MongoClient getMongoClient() {
+//  return mongoClient;
+//}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 019760d..36e1445 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
@@ -32,10 +32,9 @@ 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.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoTestBase;
 import org.apache.rya.sail.config.RyaSailFactory;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -54,100 +53,130 @@ import com.google.common.collect.ImmutableSet;
 
 public class MongoEntityIndexIT extends MongoTestBase {
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();
-    private SailRepositoryConnection conn;
-    private MongoEntityIndexer indexer;
 
-    @Before
-    public void setUp() throws Exception {
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.setBoolean(ConfigUtils.USE_MONGO, true);
         conf.setBoolean(ConfigUtils.USE_ENTITY, true);
-
-        final Sail sail = RyaSailFactory.getInstance(conf);
-        conn = new SailRepository(sail).getConnection();
-        conn.begin();
-
-        indexer = new MongoEntityIndexer();
-        indexer.setConf(conf);
-        indexer.init();
     }
 
-    @After
-    public void tearDown() throws Exception {
-        if (conn != null) {
-            conn.clear();
-        }
-        if (indexer != null) {
-            indexer.close();
-        }
-    }
+//    private SailRepositoryConnection conn;
+//    private MongoEntityIndexer indexer;
+
+//    @Before
+//    public void setUp() throws Exception {
+//        conf.setBoolean(ConfigUtils.USE_MONGO, true);
+//        conf.setBoolean(ConfigUtils.USE_ENTITY, true);
+//
+//        final Sail sail = RyaSailFactory.getInstance(conf);
+//        conn = new SailRepository(sail).getConnection();
+//        conn.begin();
+//
+//        indexer = new MongoEntityIndexer();
+//        indexer.setConf(conf);
+//        indexer.init();
+//    }
+//
+//    @After
+//    public void tearDown() throws Exception {
+//        if (conn != null) {
+//            conn.clear();
+//        }
+//        if (indexer != null) {
+//            indexer.close();
+//        }
+//    }
 
     @Test
     public void ensureInEntityStore_Test() throws Exception {
-        setupTypes();
-        addStatements();
+        final Sail sail = RyaSailFactory.getInstance(conf);
+        SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        conn.begin();
 
-        final EntityStorage entities = indexer.getEntityStorage(conf);
-        final RyaURI subject = new RyaURI("urn:alice");
-        final Optional<Entity> alice = entities.get(subject);
-        assertTrue(alice.isPresent());
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            setupTypes(indexer);
+            addStatements(conn);
+
+            final EntityStorage entities = indexer.getEntityStorage(conf);
+            final RyaURI subject = new RyaURI("urn:alice");
+            final Optional<Entity> alice = entities.get(subject);
+            assertTrue(alice.isPresent());
+        } finally {
+            conn.close();
+        }
     }
 
     @Test
     public void sparqlQuery_Test() throws Exception {
-        setupTypes();
-        addStatements();
-        //conn.commit();
-
-        final String query = "SELECT * WHERE { " +
-                "<urn:strawberry> <" + RDF.TYPE + "> <urn:icecream> ."+
-                "<urn:strawberry> <urn:brand> ?brand . " +
-                "<urn:strawberry> <urn:flavor> ?flavor . " +
-            "}";
-
-        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        final Set<BindingSet> results = new HashSet<>();
-        while(rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("flavor", ValueFactoryImpl.getInstance().createLiteral("Strawberry"));
-        expected.addBinding("brand", ValueFactoryImpl.getInstance().createLiteral("Awesome Icecream"));
+        final Sail sail = RyaSailFactory.getInstance(conf);
+        SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        conn.begin();
 
-        assertEquals(1, results.size());
-        assertEquals(expected, results.iterator().next());
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            setupTypes(indexer);
+            addStatements(conn);
+
+            final String query = "SELECT * WHERE { " +
+                    "<urn:strawberry> <" + RDF.TYPE + "> <urn:icecream> ."+
+                    "<urn:strawberry> <urn:brand> ?brand . " +
+                    "<urn:strawberry> <urn:flavor> ?flavor . " +
+                    "}";
+
+            final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            final Set<BindingSet> results = new HashSet<>();
+            while(rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            final MapBindingSet expected = new MapBindingSet();
+            expected.addBinding("flavor", ValueFactoryImpl.getInstance().createLiteral("Strawberry"));
+            expected.addBinding("brand", ValueFactoryImpl.getInstance().createLiteral("Awesome Icecream"));
+
+            assertEquals(1, results.size());
+            assertEquals(expected, results.iterator().next());
+        } finally {
+            conn.close();
+        }
     }
 
     @Test
     public void partialQuery_Test() throws Exception {
-        setupTypes();
-        addStatements();
-        conn.commit();
-
-        final String query = "SELECT * WHERE { " +
-                "<urn:george> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:george> <urn:name> ?name . " +
-                "<urn:george> <urn:eye> ?eye . " +
-            "}";
-
-        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        final Set<BindingSet> results = new HashSet<>();
-        while(rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            System.out.println(bs);
-            results.add(bs);
+        final Sail sail = RyaSailFactory.getInstance(conf);
+        SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        conn.begin();
+
+        try(MongoEntityIndexer indexer = new MongoEntityIndexer()) {
+            setupTypes(indexer);
+            addStatements(conn);
+            conn.commit();
+
+            final String query = "SELECT * WHERE { " +
+                    "<urn:george> <" + RDF.TYPE + "> <urn:person> ."+
+                    "<urn:george> <urn:name> ?name . " +
+                    "<urn:george> <urn:eye> ?eye . " +
+                "}";
+
+            final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            final Set<BindingSet> results = new HashSet<>();
+            while(rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                System.out.println(bs);
+                results.add(bs);
+            }
+            final ValueFactory vf = ValueFactoryImpl.getInstance();
+            final MapBindingSet expected = new MapBindingSet();
+            //expected.addBinding("name", vf.createURI("http://www.w3.org/2001/SMLSchema#string", "George"));
+            expected.addBinding("name", vf.createLiteral("George"));
+            expected.addBinding("eye", vf.createLiteral("blue"));
+
+            assertEquals(1, results.size());
+            assertEquals(expected, results.iterator().next());
+        } finally {
+            conn.close();
         }
-        final ValueFactory vf = ValueFactoryImpl.getInstance();
-        final MapBindingSet expected = new MapBindingSet();
-        //expected.addBinding("name", vf.createURI("http://www.w3.org/2001/SMLSchema#string", "George"));
-        expected.addBinding("name", vf.createLiteral("George"));
-        expected.addBinding("eye", vf.createLiteral("blue"));
-
-        assertEquals(1, results.size());
-        assertEquals(expected, results.iterator().next());
     }
 
-    private void setupTypes() throws Exception {
+    private void setupTypes(MongoEntityIndexer indexer) throws Exception {
         final TypeStorage typeStore = indexer.getTypeStorage(conf);
         // Add some Types to the storage.
         final Type cat = new Type(new RyaURI("urn:cat"),
@@ -184,7 +213,7 @@ public class MongoEntityIndexIT extends MongoTestBase {
         typeStore.create(person);
     }
 
-    private void addStatements() throws Exception {
+    private void addStatements(SailRepositoryConnection conn) throws Exception {
         //alice
         URI subject = VF.createURI("urn:alice");
         URI predicate = VF.createURI("urn:name");
@@ -304,4 +333,4 @@ public class MongoEntityIndexIT extends MongoTestBase {
         conn.add(VF.createStatement(subject, predicate, object));
         conn.add(VF.createStatement(subject, RDF.TYPE, VF.createURI("urn:icecream")));
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 f111fd1..71840ea 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
@@ -1,5 +1,4 @@
-package org.apache.rya.indexing.mongo;
-/*
+/**
  * 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
@@ -17,6 +16,10 @@ package org.apache.rya.indexing.mongo;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.indexing.mongo;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.util.HashSet;
 import java.util.Set;
@@ -30,7 +33,6 @@ 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.MongoTestBase;
-import org.junit.Assert;
 import org.junit.Test;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
@@ -44,13 +46,17 @@ import com.google.common.collect.Sets;
 
 import info.aduna.iteration.CloseableIteration;
 
+/**
+ * Integration tests the methods of {@link MongoFreeTextIndexer}.
+ */
 public class MongoFreeTextIndexerTest extends MongoTestBase {
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
 
     @Test
     public void testSearch() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+            f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -64,17 +70,18 @@ public class MongoFreeTextIndexerTest extends MongoTestBase {
             f.storeStatement(RdfToRyaConversions.convertStatement(statement));
             f.flush();
 
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryText("asdf", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("asdf", EMPTY_CONSTRAINTS)));
 
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat new", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat new", EMPTY_CONSTRAINTS)));
         }
     }
 
     @Test
     public void testDelete() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+            f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -100,15 +107,15 @@ public class MongoFreeTextIndexerTest extends MongoTestBase {
 
 
             f.deleteStatement(RdfToRyaConversions.convertStatement(statement1));
-            Assert.assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
 
             // Check that "new" didn't get deleted from the term table after "this is a new hat"
             // was deleted since "new" is still in "Do you like my new hat?"
-            Assert.assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
 
             f.deleteStatement(RdfToRyaConversions.convertStatement(statement2));
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryText("this is a new hat", EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("this is a new hat", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
         }
     }
 
@@ -117,7 +124,8 @@ public class MongoFreeTextIndexerTest extends MongoTestBase {
         conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, "pred:1,pred:2");
 
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+            f.setConf(conf);
+            f.init();
 
             // 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")));
@@ -137,20 +145,21 @@ public class MongoFreeTextIndexerTest extends MongoTestBase {
 
             f.flush();
 
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryText("invalid", EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryText("in:validURI", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("invalid", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("in:validURI", EMPTY_CONSTRAINTS)));
 
             final Set<Statement> actual = getSet(f.queryText("valid", EMPTY_CONSTRAINTS));
-            Assert.assertEquals(2, actual.size());
-            Assert.assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s3)));
-            Assert.assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s4)));
+            assertEquals(2, actual.size());
+            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s3)));
+            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s4)));
         }
     }
 
     @Test
     public void testContextSearch() throws Exception {
         try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+            f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final URI subject = new URIImpl("foo:subj");
@@ -162,19 +171,18 @@ public class MongoFreeTextIndexerTest extends MongoTestBase {
             f.storeStatement(RdfToRyaConversions.convertStatement(statement));
             f.flush();
 
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", new StatementConstraints().setContext(context))));
-            Assert.assertEquals(Sets.newHashSet(),
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", new StatementConstraints().setContext(context))));
+            assertEquals(Sets.newHashSet(),
                     getSet(f.queryText("hat", new StatementConstraints().setContext(vf.createURI("foo:context2")))));
         }
     }
 
-
     private static <X> Set<X> getSet(final CloseableIteration<X, ?> iter) throws Exception {
-        final Set<X> set = new HashSet<X>();
+        final Set<X> set = new HashSet<>();
         while (iter.hasNext()) {
             set.add(iter.next());
         }
         return set;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
index cbd18b2..1f083f2 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoIndexingConfigurationTest.java
@@ -68,11 +68,11 @@ public class MongoIndexingConfigurationTest {
         assertEquals(conf.getCv(), visibility);
         assertEquals(conf.isInfer(), useInference);
         assertEquals(conf.isDisplayQueryPlan(), displayPlan);
-        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getMongoHostname(), "host");
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.getRyaInstance(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
         assertTrue(
@@ -103,11 +103,11 @@ public class MongoIndexingConfigurationTest {
         assertEquals(conf.getCv(), visibility);
         assertEquals(conf.isInfer(), useInference);
         assertEquals(conf.isDisplayQueryPlan(), displayPlan);
-        assertEquals(conf.getMongoInstance(), "host");
+        assertEquals(conf.getMongoHostname(), "host");
         assertEquals(conf.getBoolean(".useMockInstance", false), useMock);
         assertEquals(conf.getMongoPort(), "1000");
         assertEquals(conf.getMongoDBName(), "dbname");
-        assertEquals(conf.getCollectionName(), "prefix_");
+        assertEquals(conf.getRyaInstance(), "prefix_");
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER), user);
         assertEquals(conf.get(MongoDBRdfConfiguration.MONGO_USER_PASSWORD), password);
         assertTrue(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 215e64c..ec7d1ec 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
@@ -80,13 +80,13 @@ import info.aduna.iteration.CloseableIteration;
  *
  */
 public final class MongoTemporalIndexerTest extends MongoTestBase {
-    private MongoTemporalIndexer tIndexer;
-    private DBCollection collection;
+//    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";
     private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
-    private static final String STAT_VALUEHASH = "valuehash";
+//    private static final String STAT_VALUEHASH = "valuehash";
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
 
     // Assign this in setUpBeforeClass, store them in each test.
@@ -164,85 +164,110 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
 
     @Before
     public void before() throws Exception {
+//        // This is from http://linkedevents.org/ontology
+//        // and http://motools.sourceforge.net/event/event.html
+//        conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
+//                + URI_PROPERTY_AT_TIME + ","
+//                + URI_PROPERTY_CIRCA + ","
+//                + URI_PROPERTY_EVENT_TIME);
+
+//        tIndexer
+//        tIndexer.setConf(conf);
+//        tIndexer.init();
+////        tIndexer.initIndexer(conf, super.getMongoClient());
+
+//        final String dbName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
+//        final DB db = super.getMongoClient().getDB(dbName);
+//        collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
+   }
+
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         // This is from http://linkedevents.org/ontology
         // and http://motools.sourceforge.net/event/event.html
         conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
                 + URI_PROPERTY_AT_TIME + ","
                 + URI_PROPERTY_CIRCA + ","
                 + URI_PROPERTY_EVENT_TIME);
-
-        tIndexer = new MongoTemporalIndexer();
-        tIndexer.initIndexer(conf, super.getMongoClient());
-
-        final String dbName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
-        final DB db = super.getMongoClient().getDB(dbName);
-        collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
-   }
+    }
 
     /**
      * Test method for {@link MongoTemporalIndexer#storeStatement(convertStatement(org.openrdf.model.Statement)}
      */
     @Test
     public void testStoreStatement() throws IOException {
-        final ValueFactory vf = new ValueFactoryImpl();
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
 
-        final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-        final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
-
-        // Should not be stored because they are not in the predicate list
-        final String validDateStringWithThirteens = "1313-12-13T13:13:13Z";
-        tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), RDFS.LABEL, vf.createLiteral(validDateStringWithThirteens))));
-
-        final String invalidDateString = "ThisIsAnInvalidDate";
-        tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), pred1_atTime, vf.createLiteral(invalidDateString))));
-
-        // These are different datetimes instant but from different time zones.
-        // This is an arbitrary zone, BRST=Brazil, better if not local.
-        // same as "2015-01-01T01:59:59Z"
-        final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
-        // next year, same as "2017-01-01T01:59:59Z"
-        final String testDate2016InET = "2016-12-31T20:59:59-05:00";
-
-        // These should be stored because they are in the predicate list.
-        // BUT they will get converted to the same exact datetime in UTC.
-        final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
-        final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
-        tIndexer.storeStatement(convertStatement(s3));
-        tIndexer.storeStatement(convertStatement(s4));
-
-        // This should not be stored because the object is not a literal
-        tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1_atTime, vf.createURI("in:valid"))));
-
-        printTables("junit testing: Temporal entities stored in testStoreStatement");
-        assertEquals(2, tIndexer.getCollection().find().count());
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
+
+            // Should not be stored because they are not in the predicate list
+            final String validDateStringWithThirteens = "1313-12-13T13:13:13Z";
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), RDFS.LABEL, vf.createLiteral(validDateStringWithThirteens))));
+
+            final String invalidDateString = "ThisIsAnInvalidDate";
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), pred1_atTime, vf.createLiteral(invalidDateString))));
+
+            // These are different datetimes instant but from different time zones.
+            // This is an arbitrary zone, BRST=Brazil, better if not local.
+            // same as "2015-01-01T01:59:59Z"
+            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
+            // next year, same as "2017-01-01T01:59:59Z"
+            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
+
+            // These should be stored because they are in the predicate list.
+            // BUT they will get converted to the same exact datetime in UTC.
+            final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
+            final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
+            tIndexer.storeStatement(convertStatement(s3));
+            tIndexer.storeStatement(convertStatement(s4));
+
+            // This should not be stored because the object is not a literal
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1_atTime, vf.createURI("in:valid"))));
+
+            printTables(tIndexer, "junit testing: Temporal entities stored in testStoreStatement");
+            assertEquals(2, tIndexer.getCollection().find().count());
+        }
     }
 
     @Test
     public void testDelete() throws IOException, MongoException, TableNotFoundException, TableExistsException, NoSuchAlgorithmException {
-        final ValueFactory vf = new ValueFactoryImpl();
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
 
-        final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-        final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
 
-        final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
-        final String testDate2016InET = "2016-12-31T20:59:59-05:00";
+            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
+            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
 
-        // These should be stored because they are in the predicate list.
-        // BUT they will get converted to the same exact datetime in UTC.
-        final Statement s1 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
-        final Statement s2 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
-        tIndexer.storeStatement(convertStatement(s1));
-        tIndexer.storeStatement(convertStatement(s2));
+            // These should be stored because they are in the predicate list.
+            // BUT they will get converted to the same exact datetime in UTC.
+            final Statement s1 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
+            final Statement s2 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
+            tIndexer.storeStatement(convertStatement(s1));
+            tIndexer.storeStatement(convertStatement(s2));
 
+            final String dbName = conf.getMongoDBName();
+            final DB db = super.getMongoClient().getDB(dbName);
+            DBCollection collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
 
-        printTables("junit testing: Temporal entities stored in testDelete before delete");
-        assertEquals("Number of rows stored.", 2, collection.count()); // 4 index entries per statement
+            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete before delete");
+            assertEquals("Number of rows stored.", 2, collection.count()); // 4 index entries per statement
 
-        tIndexer.deleteStatement(convertStatement(s1));
-        tIndexer.deleteStatement(convertStatement(s2));
+            tIndexer.deleteStatement(convertStatement(s1));
+            tIndexer.deleteStatement(convertStatement(s2));
 
-        printTables("junit testing: Temporal entities stored in testDelete after delete");
-        assertEquals("Number of rows stored after delete.", 0, collection.count());
+            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete after delete");
+            assertEquals("Number of rows stored after delete.", 0, collection.count());
+        }
     }
 
     /**
@@ -254,31 +279,36 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryInstantAfterInstant() throws IOException, QueryEvaluationException, TableNotFoundException, MongoException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-        final int searchForSeconds = 4;
-        final int expectedResultCount = 9;
-        for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 9;
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count + 1]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
     /**
      * Test instant before a given instant.
@@ -286,32 +316,37 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryInstantBeforeInstant() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-        final int searchForSeconds = 4;
-        final int expectedResultCount = 4;
-        for (int s = 0; s <= searchForSeconds + 15; s++) { // <== logic here
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 4;
+            for (int s = 0; s <= searchForSeconds + 15; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
 
-        iter = tIndexer.queryInstantBeforeInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            iter = tIndexer.queryInstantBeforeInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
 
     /**
@@ -320,31 +355,36 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryInstantBeforeInterval() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-        final TemporalInterval searchForSeconds = tvB02_E31;
-        final int expectedResultCount = 2; // 00 and 01 seconds.
-        for (int s = 0; s <= 40; s++) { // <== logic here
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchForSeconds = tvB02_E31;
+            final int expectedResultCount = 2; // 00 and 01 seconds.
+            for (int s = 0; s <= 40; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantBeforeInterval(searchForSeconds, EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantBeforeInterval(searchForSeconds, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
 
     /**
@@ -353,32 +393,37 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryInstantAfterInterval() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-        final TemporalInterval searchAfterInterval = tvB02_E31; // from 2 to 31 seconds
-        final int endingSeconds = 31;
-        final int expectedResultCount = 9; // 32,33,...,40 seconds.
-        for (int s = 0; s <= endingSeconds + expectedResultCount; s++) { // <== logic here
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchAfterInterval = tvB02_E31; // from 2 to 31 seconds
+            final int endingSeconds = 31;
+            final int expectedResultCount = 9; // 32,33,...,40 seconds.
+            for (int s = 0; s <= endingSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantAfterInterval(searchAfterInterval, EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[count + endingSeconds + 1]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantAfterInterval(searchAfterInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count + endingSeconds + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
 
     /**
@@ -387,99 +432,116 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryInstantInsideInterval() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-        final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-        final int beginningSeconds = 2; // <== logic here, and next few lines.
-        final int endingSeconds = 31;
-        final int expectedResultCount = endingSeconds - beginningSeconds - 1; // 3,4,...,30 seconds.
-        for (int s = 0; s <= 40; s++) {
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int beginningSeconds = 2; // <== logic here, and next few lines.
+            final int endingSeconds = 31;
+            final int expectedResultCount = endingSeconds - beginningSeconds - 1; // 3,4,...,30 seconds.
+            for (int s = 0; s <= 40; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantInsideInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[count + beginningSeconds + 1]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantInsideInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count + beginningSeconds + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
+
     /**
      * Test instant is the Beginning of the given interval.
      * from the series: Instance {hasBeginning, hasEnd} Interval
      */
     @Test
     public void testQueryInstantHasBeginningInterval() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-        final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-        final int searchSeconds = 2; // <== logic here, and next few lines.
-        final int expectedResultCount = 1; // 2 seconds.
-        for (int s = 0; s <= 10; s++) {
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int searchSeconds = 2; // <== logic here, and next few lines.
+            final int expectedResultCount = 1; // 2 seconds.
+            for (int s = 0; s <= 10; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantHasBeginningInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantHasBeginningInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
+
     /**
      * Test instant is the end of the given interval.
      * from the series: Instance {hasBeginning, hasEnd} Interval
      */
     @Test
     public void testQueryInstantHasEndInterval()  throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-        final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-        final int searchSeconds = 31; // <== logic here, and next few lines.
-        final int expectedResultCount = 1; // 31 seconds.
-        for (int s = 0; s <= 40; s++) {
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int searchSeconds = 31; // <== logic here, and next few lines.
+            final int expectedResultCount = 1; // 31 seconds.
+            for (int s = 0; s <= 40; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryInstantHasEndInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-        int count = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
-            assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-            count++;
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantHasEndInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
         }
-        assertEquals("Should find count of rows.", expectedResultCount, count);
     }
 
     /**
@@ -492,21 +554,26 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryIntervalEquals() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-        tIndexer.storeStatement(convertStatement(seriesSpo[4])); // instance at 4 seconds
-
-
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryIntervalEquals(tvB02_E40, EMPTY_CONSTRAINTS);
-        // Should be found twice:
-        assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but actually returned empty results. spo_B02_E40=" + spo_B02_E40, iter.hasNext());
-        assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but does not match.", spo_B02_E40.equals(iter.next()));
-        assertFalse("queryIntervalEquals: Find no more than one, but actually has more.", iter.hasNext());
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            tIndexer.storeStatement(convertStatement(seriesSpo[4])); // instance at 4 seconds
+
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalEquals(tvB02_E40, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but actually returned empty results. spo_B02_E40=" + spo_B02_E40, iter.hasNext());
+            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but does not match.", spo_B02_E40.equals(iter.next()));
+            assertFalse("queryIntervalEquals: Find no more than one, but actually has more.", iter.hasNext());
+        }
     }
 
     /**
@@ -518,25 +585,30 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryIntervalBefore() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        tIndexer.storeStatement(convertStatement(spo_B00_E01));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        // instants should be ignored.
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-        tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
-        tIndexer.storeStatement(convertStatement(seriesSpo[2]));
-        tIndexer.storeStatement(convertStatement(seriesSpo[31]));
-
-
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryIntervalBefore(tvB02_E31, EMPTY_CONSTRAINTS);
-        // Should be found twice:
-        assertTrue("spo_B00_E01 should be found, but actually returned empty results. spo_B00_E01=" + spo_B00_E01, iter.hasNext());
-        assertTrue("spo_B00_E01 should be found, but found another.", spo_B00_E01.equals(iter.next()));
-        assertFalse("Find no more than one, but actually has more.", iter.hasNext());
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B00_E01));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            // instants should be ignored.
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
+            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
+            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
+
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalBefore(tvB02_E31, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("spo_B00_E01 should be found, but actually returned empty results. spo_B00_E01=" + spo_B00_E01, iter.hasNext());
+            assertTrue("spo_B00_E01 should be found, but found another.", spo_B00_E01.equals(iter.next()));
+            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
+        }
     }
 
     /**
@@ -548,29 +620,33 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryIntervalAfter() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        tIndexer.storeStatement(convertStatement(spo_B00_E01));
-        tIndexer.storeStatement(convertStatement(spo_B02_E29)); //<- after this one.
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B29_E30));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-        // instants should be ignored.
-        tIndexer.storeStatement(convertStatement(spo_B02));
-        tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
-        tIndexer.storeStatement(convertStatement(seriesSpo[2]));
-        tIndexer.storeStatement(convertStatement(seriesSpo[31]));
-
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        iter = tIndexer.queryIntervalAfter(tvB02_E29, EMPTY_CONSTRAINTS);
-        // Should be found twice:
-        assertTrue("spo_B30_E32 should be found, but actually returned empty results. spo_B30_E32=" + spo_B30_E32, iter.hasNext());
-        final Statement s = iter.next();
-        assertTrue("spo_B30_E32 should be found, but found another. spo_B30_E32="+spo_B30_E32+", but found="+s, spo_B30_E32.equals(s));
-        assertFalse("Find no more than one, but actually has more.", iter.hasNext());
-
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B00_E01));
+            tIndexer.storeStatement(convertStatement(spo_B02_E29)); //<- after this one.
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B29_E30));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            // instants should be ignored.
+            tIndexer.storeStatement(convertStatement(spo_B02));
+            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
+            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
+            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalAfter(tvB02_E29, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("spo_B30_E32 should be found, but actually returned empty results. spo_B30_E32=" + spo_B30_E32, iter.hasNext());
+            final Statement s = iter.next();
+            assertTrue("spo_B30_E32 should be found, but found another. spo_B30_E32="+spo_B30_E32+", but found="+s, spo_B30_E32.equals(s));
+            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
+        }
     }
 
     /**
@@ -578,65 +654,70 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      */
     @Test
     public void testQueryWithMultiplePredicates() throws IOException, QueryEvaluationException {
-        // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-        // these should not match as they are not instances.
-        tIndexer.storeStatement(convertStatement(spo_B03_E20));
-        tIndexer.storeStatement(convertStatement(spo_B02_E30));
-        tIndexer.storeStatement(convertStatement(spo_B02_E40));
-        tIndexer.storeStatement(convertStatement(spo_B02_E31));
-        tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-        // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-        final int searchForSeconds = 4;
-        final int expectedResultCount = 9;
-        for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-            tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-        }
-        final ValueFactory vf = new ValueFactoryImpl();
-        final URI pred3_CIRCA_ = vf.createURI(URI_PROPERTY_CIRCA);  // this one to ignore.
-        final URI pred2_eventTime = vf.createURI(URI_PROPERTY_EVENT_TIME);
-        final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-
-        // add the predicate = EventTime ; Store in an array for verification.
-        final Statement[] SeriesTs_EventTime = new Statement[expectedResultCount+1];
-        for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-            final Statement statement = new StatementImpl(vf.createURI("foo:EventTimeSubj0" + s), pred2_eventTime, vf.createLiteral(seriesTs[s].getAsReadable()));
-            tIndexer.storeStatement(convertStatement(statement));
-            if (s>searchForSeconds) {
-                SeriesTs_EventTime[s - searchForSeconds -1 ] = statement;
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 9;
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+            final ValueFactory vf = new ValueFactoryImpl();
+            final URI pred3_CIRCA_ = vf.createURI(URI_PROPERTY_CIRCA);  // this one to ignore.
+            final URI pred2_eventTime = vf.createURI(URI_PROPERTY_EVENT_TIME);
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+
+            // add the predicate = EventTime ; Store in an array for verification.
+            final Statement[] SeriesTs_EventTime = new Statement[expectedResultCount+1];
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                final Statement statement = new StatementImpl(vf.createURI("foo:EventTimeSubj0" + s), pred2_eventTime, vf.createLiteral(seriesTs[s].getAsReadable()));
+                tIndexer.storeStatement(convertStatement(statement));
+                if (s>searchForSeconds) {
+                    SeriesTs_EventTime[s - searchForSeconds -1 ] = statement;
+                }
+            }
+            // add the predicate = CIRCA ; to be ignored because it is not in the constraints.
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                final Statement statement = new StatementImpl(vf.createURI("foo:CircaEventSubj0" + s), pred3_CIRCA_, vf.createLiteral(seriesTs[s].getAsReadable()));
+                tIndexer.storeStatement(convertStatement(statement));
             }
-        }
-        // add the predicate = CIRCA ; to be ignored because it is not in the constraints.
-        for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-            final Statement statement = new StatementImpl(vf.createURI("foo:CircaEventSubj0" + s), pred3_CIRCA_, vf.createLiteral(seriesTs[s].getAsReadable()));
-            tIndexer.storeStatement(convertStatement(statement));
-        }
 
-        CloseableIteration<Statement, QueryEvaluationException> iter;
-        final StatementConstraints constraints = new StatementConstraints();
-        constraints.setPredicates(new HashSet<URI>(Arrays.asList( pred2_eventTime,  pred1_atTime )));
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            final StatementConstraints constraints = new StatementConstraints();
+            constraints.setPredicates(new HashSet<>(Arrays.asList( pred2_eventTime,  pred1_atTime )));
+
+            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], constraints); // EMPTY_CONSTRAINTS);//
+            int count_AtTime = 0;
+            int count_EventTime = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count_AtTime + 1]; // <== logic here
+                if (s.getPredicate().equals(pred1_atTime)) {
+                    assertTrue("Should match atTime: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                    count_AtTime++;
+                }
+                else if (s.getPredicate().equals(pred2_eventTime)) {
+                    assertTrue("Should match eventTime: " + SeriesTs_EventTime[count_EventTime] + " == " + s, SeriesTs_EventTime[count_EventTime].equals(s));
+                    count_EventTime++;
+                } else {
+                    assertTrue("This predicate should not be returned: "+s, false);
+                }
 
-        iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], constraints); // EMPTY_CONSTRAINTS);//
-        int count_AtTime = 0;
-        int count_EventTime = 0;
-        while (iter.hasNext()) {
-            final Statement s = iter.next();
-            final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count_AtTime + 1]; // <== logic here
-            if (s.getPredicate().equals(pred1_atTime)) {
-                assertTrue("Should match atTime: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count_AtTime++;
-            }
-            else if (s.getPredicate().equals(pred2_eventTime)) {
-                assertTrue("Should match eventTime: " + SeriesTs_EventTime[count_EventTime] + " == " + s, SeriesTs_EventTime[count_EventTime].equals(s));
-                count_EventTime++;
-            } else {
-                assertTrue("This predicate should not be returned: "+s, false);
             }
 
+            assertEquals("Should find count of atTime    rows.", expectedResultCount, count_AtTime);
+            assertEquals("Should find count of eventTime rows.", expectedResultCount, count_EventTime);
         }
-
-        assertEquals("Should find count of atTime    rows.", expectedResultCount, count_AtTime);
-        assertEquals("Should find count of eventTime rows.", expectedResultCount, count_EventTime);
     }
 
     /**
@@ -651,7 +732,7 @@ public final class MongoTemporalIndexerTest extends MongoTestBase {
      * @return Count of entries in the index table.
      * @throws IOException
      */
-    public void printTables(final String description) throws IOException {
+    public void printTables(MongoTemporalIndexer tIndexer, final String description) throws IOException {
         System.out.println("-- start printTables() -- " + description);
         System.out.println("Reading : " + tIndexer.getCollection().getFullName());
         final DBCursor cursor = tIndexer.getCollection().find();


[12/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 c4db88e..647e3cc 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
@@ -33,7 +33,7 @@ import org.apache.rya.api.domain.StatementMetadata;
 import org.apache.rya.api.persist.RyaDAOException;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.apache.rya.sail.config.RyaSailFactory;
 import org.junit.Test;
 import org.openrdf.model.impl.LiteralImpl;
@@ -49,7 +49,7 @@ import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
-public class MongoStatementMetadataIT extends MongoTestBase {
+public class MongoStatementMetadataIT extends MongoITBase {
 
     private final String query1 = "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 }";

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeIT.java
new file mode 100644
index 0000000..cab6c35
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeIT.java
@@ -0,0 +1,368 @@
+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
+ * 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.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaStatement;
+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.indexing.statement.metadata.matching.StatementMetadataNode;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.XMLSchema;
+import org.openrdf.query.BindingSet;
+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.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import info.aduna.iteration.CloseableIteration;
+
+public class MongoStatementMetadataNodeIT extends MongoITBase {
+    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 {
+        final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+        conf.setUseStatementMetadata(true);
+        conf.setStatementMetadataProperties(propertySet);
+    }
+
+    @Test
+    public void simpleQueryWithoutBindingSet() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+
+            StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+            QueryBindingSet bs = new QueryBindingSet();
+            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bs.addBinding("y", new LiteralImpl("Joe"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(bs, bsList.get(0));
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * 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.
+     */
+    @Test
+    public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();	
+
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+            Assert.assertEquals(0, bsList.size());
+            dao.delete(statement, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    @Test
+    public void simpleQueryWithBindingSet() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            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);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            QueryBindingSet bsConstraint = new QueryBindingSet();
+            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+            QueryBindingSet expected = new QueryBindingSet();
+            expected.addBinding("x", new LiteralImpl("CoffeeShop"));
+            expected.addBinding("y", new LiteralImpl("Joe"));
+            expected.addBinding("z", new LiteralImpl("Virginia"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(expected, bsList.get(0));
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests to see if correct result is passed back when a metadata statement
+     * is joined with a StatementPattern statement (i.e. a common variable
+     * appears in a StatementPattern statement and a metadata statement).
+     * StatementPattern statements have either rdf:subject, rdf:predicate, or
+     * rdf:object as the predicate while a metadata statement is any statement
+     * in the reified query whose predicate is not rdf:type and not a
+     * StatementPattern predicate.
+     */
+    @Test
+    public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            StatementMetadata metadata = new StatementMetadata();
+            metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
+                    new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement1);
+            dao.add(statement2);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query2, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            List<BindingSet> bsCollection = new ArrayList<>();
+            QueryBindingSet bsConstraint1 = new QueryBindingSet();
+            bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
+            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet bsConstraint2 = new QueryBindingSet();
+            bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
+            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+            bsCollection.add(bsConstraint1);
+            bsCollection.add(bsConstraint2);
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+            QueryBindingSet expected = new QueryBindingSet();
+            expected.addBinding("y", new LiteralImpl("CoffeeShop"));
+            expected.addBinding("x", new URIImpl("http://Joe"));
+            expected.addBinding("z", new LiteralImpl("Virginia"));
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(1, bsList.size());
+            Assert.assertEquals(expected, bsList.get(0));
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests if the StatementMetadataNode joins BindingSet correctly for
+     * variables appearing in metadata statements. In this case, the metadata
+     * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
+     * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
+     * object in the above metadata statement and its values are joined to the
+     * constraint BindingSets in the example below.
+     */
+    @Test
+    public void simpleQueryWithBindingSetJoinOnProperty() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            dao.add(statement1);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            QueryBindingSet bsConstraint = new QueryBindingSet();
+            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint.addBinding("y", new LiteralImpl("Doug"));
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+            List<BindingSet> bsList = new ArrayList<>();
+            while (iteration.hasNext()) {
+                bsList.add(iteration.next());
+            }
+
+            Assert.assertEquals(0, bsList.size());
+            dao.delete(statement1, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+
+    /**
+     * Tests if StatementMetadataNode joins BindingSet values correctly for
+     * variables appearing as the object in one of the StatementPattern
+     * statements (in the case ?x appears as the Object in the statement
+     * _:blankNode rdf:object ?x). StatementPattern statements have either
+     * rdf:subject, rdf:predicate, or rdf:object as the predicate.
+     */
+    @Test
+    public void simpleQueryWithBindingSetCollection() throws Exception {
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+            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"),
+                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+            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);
+
+            SPARQLParser parser = new SPARQLParser();
+            ParsedQuery pq = parser.parseQuery(query, null);
+            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+            List<BindingSet> bsCollection = new ArrayList<>();
+            QueryBindingSet bsConstraint1 = new QueryBindingSet();
+            bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
+            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet bsConstraint2 = new QueryBindingSet();
+            bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
+            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+
+            QueryBindingSet bsConstraint3 = new QueryBindingSet();
+            bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
+            bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
+            bsCollection.add(bsConstraint1);
+            bsCollection.add(bsConstraint2);
+            bsCollection.add(bsConstraint3);
+
+            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+            Set<BindingSet> expected = new HashSet<>();
+            QueryBindingSet expected1 = new QueryBindingSet();
+            expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
+            expected1.addBinding("y", new LiteralImpl("Joe"));
+            expected1.addBinding("z", new LiteralImpl("Virginia"));
+
+            QueryBindingSet expected2 = new QueryBindingSet();
+            expected2.addBinding("x", new LiteralImpl("HardwareStore"));
+            expected2.addBinding("y", new LiteralImpl("Joe"));
+            expected2.addBinding("z", new LiteralImpl("Maryland"));
+            expected.add(expected1);
+            expected.add(expected2);
+
+            Set<BindingSet> bsSet = new HashSet<>();
+            while (iteration.hasNext()) {
+                bsSet.add(iteration.next());
+            }
+
+            Assert.assertEquals(expected, bsSet);
+
+            dao.delete(statement1, conf);
+            dao.delete(statement2, conf);
+        } finally {
+            dao.destroy();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index a45197c..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/statement/metadata/MongoStatementMetadataNodeTest.java
+++ /dev/null
@@ -1,368 +0,0 @@
-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
- * 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.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.rya.api.domain.RyaStatement;
-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.indexing.statement.metadata.matching.StatementMetadataNode;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoDBRyaDAO;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.openrdf.model.impl.LiteralImpl;
-import org.openrdf.model.impl.URIImpl;
-import org.openrdf.model.vocabulary.XMLSchema;
-import org.openrdf.query.BindingSet;
-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.ParsedQuery;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import info.aduna.iteration.CloseableIteration;
-
-public class MongoStatementMetadataNodeTest extends MongoTestBase {
-    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 {
-        final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-        conf.setUseStatementMetadata(true);
-        conf.setStatementMetadataProperties(propertySet);
-    }
-
-    @Test
-    public void simpleQueryWithoutBindingSet() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-            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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            dao.add(statement);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-
-            StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-            QueryBindingSet bs = new QueryBindingSet();
-            bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-            bs.addBinding("y", new LiteralImpl("Joe"));
-
-            List<BindingSet> bsList = new ArrayList<>();
-            while (iteration.hasNext()) {
-                bsList.add(iteration.next());
-            }
-
-            Assert.assertEquals(1, bsList.size());
-            Assert.assertEquals(bs, bsList.get(0));
-            dao.delete(statement, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    /**
-     * 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.
-     */
-    @Test
-    public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();	
-
-            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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            dao.add(statement);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-            List<BindingSet> bsList = new ArrayList<>();
-            while (iteration.hasNext()) {
-                bsList.add(iteration.next());
-            }
-            Assert.assertEquals(0, bsList.size());
-            dao.delete(statement, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    @Test
-    public void simpleQueryWithBindingSet() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-            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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            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);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-            QueryBindingSet bsConstraint = new QueryBindingSet();
-            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-            bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
-
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-            QueryBindingSet expected = new QueryBindingSet();
-            expected.addBinding("x", new LiteralImpl("CoffeeShop"));
-            expected.addBinding("y", new LiteralImpl("Joe"));
-            expected.addBinding("z", new LiteralImpl("Virginia"));
-
-            List<BindingSet> bsList = new ArrayList<>();
-            while (iteration.hasNext()) {
-                bsList.add(iteration.next());
-            }
-
-            Assert.assertEquals(1, bsList.size());
-            Assert.assertEquals(expected, bsList.get(0));
-
-            dao.delete(statement1, conf);
-            dao.delete(statement2, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    /**
-     * Tests to see if correct result is passed back when a metadata statement
-     * is joined with a StatementPattern statement (i.e. a common variable
-     * appears in a StatementPattern statement and a metadata statement).
-     * StatementPattern statements have either rdf:subject, rdf:predicate, or
-     * rdf:object as the predicate while a metadata statement is any statement
-     * in the reified query whose predicate is not rdf:type and not a
-     * StatementPattern predicate.
-     */
-    @Test
-    public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-            StatementMetadata metadata = new StatementMetadata();
-            metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
-                    new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
-            dao.add(statement1);
-            dao.add(statement2);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query2, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-            List<BindingSet> bsCollection = new ArrayList<>();
-            QueryBindingSet bsConstraint1 = new QueryBindingSet();
-            bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
-            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-            QueryBindingSet bsConstraint2 = new QueryBindingSet();
-            bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
-            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-            bsCollection.add(bsConstraint1);
-            bsCollection.add(bsConstraint2);
-
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-            QueryBindingSet expected = new QueryBindingSet();
-            expected.addBinding("y", new LiteralImpl("CoffeeShop"));
-            expected.addBinding("x", new URIImpl("http://Joe"));
-            expected.addBinding("z", new LiteralImpl("Virginia"));
-
-            List<BindingSet> bsList = new ArrayList<>();
-            while (iteration.hasNext()) {
-                bsList.add(iteration.next());
-            }
-
-            Assert.assertEquals(1, bsList.size());
-            Assert.assertEquals(expected, bsList.get(0));
-
-            dao.delete(statement1, conf);
-            dao.delete(statement2, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    /**
-     * Tests if the StatementMetadataNode joins BindingSet correctly for
-     * variables appearing in metadata statements. In this case, the metadata
-     * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
-     * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
-     * object in the above metadata statement and its values are joined to the
-     * constraint BindingSets in the example below.
-     */
-    @Test
-    public void simpleQueryWithBindingSetJoinOnProperty() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-            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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            dao.add(statement1);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-            QueryBindingSet bsConstraint = new QueryBindingSet();
-            bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-            bsConstraint.addBinding("y", new LiteralImpl("Doug"));
-
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-            List<BindingSet> bsList = new ArrayList<>();
-            while (iteration.hasNext()) {
-                bsList.add(iteration.next());
-            }
-
-            Assert.assertEquals(0, bsList.size());
-            dao.delete(statement1, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-
-    /**
-     * Tests if StatementMetadataNode joins BindingSet values correctly for
-     * variables appearing as the object in one of the StatementPattern
-     * statements (in the case ?x appears as the Object in the statement
-     * _:blankNode rdf:object ?x). StatementPattern statements have either
-     * rdf:subject, rdf:predicate, or rdf:object as the predicate.
-     */
-    @Test
-    public void simpleQueryWithBindingSetCollection() throws Exception {
-        MongoDBRyaDAO dao = new MongoDBRyaDAO();
-        try {
-            dao.setConf(conf);
-            dao.init();
-            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"),
-                    new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-            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);
-
-            SPARQLParser parser = new SPARQLParser();
-            ParsedQuery pq = parser.parseQuery(query, null);
-            List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-            StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-            List<BindingSet> bsCollection = new ArrayList<>();
-            QueryBindingSet bsConstraint1 = new QueryBindingSet();
-            bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
-            bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-            QueryBindingSet bsConstraint2 = new QueryBindingSet();
-            bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
-            bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-
-            QueryBindingSet bsConstraint3 = new QueryBindingSet();
-            bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
-            bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
-            bsCollection.add(bsConstraint1);
-            bsCollection.add(bsConstraint2);
-            bsCollection.add(bsConstraint3);
-
-            CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-            Set<BindingSet> expected = new HashSet<>();
-            QueryBindingSet expected1 = new QueryBindingSet();
-            expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
-            expected1.addBinding("y", new LiteralImpl("Joe"));
-            expected1.addBinding("z", new LiteralImpl("Virginia"));
-
-            QueryBindingSet expected2 = new QueryBindingSet();
-            expected2.addBinding("x", new LiteralImpl("HardwareStore"));
-            expected2.addBinding("y", new LiteralImpl("Joe"));
-            expected2.addBinding("z", new LiteralImpl("Maryland"));
-            expected.add(expected1);
-            expected.add(expected2);
-
-            Set<BindingSet> bsSet = new HashSet<>();
-            while (iteration.hasNext()) {
-                bsSet.add(iteration.next());
-            }
-
-            Assert.assertEquals(expected, bsSet);
-
-            dao.delete(statement1, conf);
-            dao.delete(statement2, conf);
-        } finally {
-            dao.destroy();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
index cb34bd0..d74dca6 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
@@ -33,7 +33,7 @@ public interface GeoTemporalIndexer extends RyaSecondaryIndexer {
 	 * initialize after setting configuration.
 	 */
     @Override
-	public void init();  
+    public void init();
 
     /**
      * Creates the {@link EventStorage} that will be used by the indexer.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
index fa2f52b..9f60e2e 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.rya.indexing.geotemporal;
 
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getQueryNode;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 
@@ -26,7 +27,6 @@ import java.util.List;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.TemporalInstantRfc3339;
 import org.apache.rya.indexing.external.matching.QuerySegment;
-import org.apache.rya.indexing.geotemporal.GeoTemporalIndexSetProvider;
 import org.apache.rya.indexing.geotemporal.model.EventQueryNode;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
 import org.junit.Before;
@@ -36,7 +36,7 @@ import org.openrdf.model.Value;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
 
-public class GeoTemporalProviderTest extends GeoTemporalTestUtils {
+public class GeoTemporalProviderTest {
     private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
     private GeoTemporalIndexSetProvider provider;
     private EventStorage events;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
index 23399c8..51b2ba0 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
@@ -45,10 +45,15 @@ import com.vividsolutions.jts.geom.Polygon;
 import com.vividsolutions.jts.geom.PrecisionModel;
 import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
 
-public class GeoTemporalTestUtils {
+public final class GeoTemporalTestUtils {
     private static final GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
 
     /**
+     * Private constructor to prevent initialization.
+     */
+    private GeoTemporalTestUtils() { }
+
+    /**
      * Make an uniform instant with given seconds.
      */
     public static TemporalInstant makeInstant(final int secondsMakeMeUnique) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
index 9c28bf0..d629d04 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
@@ -36,7 +36,7 @@ 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.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -50,7 +50,7 @@ import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
-public class MongoGeoTemporalIndexIT extends MongoTestBase {
+public class MongoGeoTemporalIndexIT extends MongoITBase {
     private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
 
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNode2IT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNode2IT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNode2IT.java
new file mode 100644
index 0000000..9875091
--- /dev/null
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNode2IT.java
@@ -0,0 +1,364 @@
+/**
+ * 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.geotemporal.model;
+
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getFilters;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getSps;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.indexing.IndexingExpr;
+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.mongo.MongoEventStorage;
+import org.apache.rya.indexing.geotemporal.storage.EventStorage;
+import org.apache.rya.mongodb.MongoITBase;
+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.query.BindingSet;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.FunctionCall;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.ValueConstant;
+import org.openrdf.query.algebra.ValueExpr;
+import org.openrdf.query.algebra.Var;
+import org.openrdf.query.impl.MapBindingSet;
+
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.PrecisionModel;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Integration tests the methods of {@link EventQueryNode}.
+ */
+public class EventQueryNode2IT extends MongoITBase {
+    private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
+    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_differentSubjects() throws Exception {
+        final Var geoSubj = new Var("point");
+        final Var geoPred = new Var("-const-http://www.opengis.net/ont/geosparql#asWKT", ValueFactoryImpl.getInstance().createURI("http://www.opengis.net/ont/geosparql#asWKT"));
+        final Var geoObj = new Var("wkt");
+        final StatementPattern geoSP = new StatementPattern(geoSubj, geoPred, geoObj);
+
+        final Var timeSubj = new Var("time");
+        final Var timePred = new Var("-const-http://www.w3.org/2006/time#inXSDDateTime", ValueFactoryImpl.getInstance().createURI("-const-http://www.w3.org/2006/time#inXSDDateTime"));
+        final Var timeObj = new Var("time");
+        final StatementPattern timeSP = new StatementPattern(timeSubj, timePred, timeObj);
+        // This will fail.
+        new EventQueryNode.EventQueryNodeBuilder()
+            .setStorage(mock(EventStorage.class))
+            .setGeoPattern(geoSP)
+            .setTemporalPattern(timeSP)
+            .setGeoFilters(new ArrayList<IndexingExpr>())
+            .setTemporalFilters(new ArrayList<IndexingExpr>())
+            .setUsedFilters(new ArrayList<>())
+            .build();
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void constructor_variablePredicate() throws Exception {
+        // A pattern that has a variable for its predicate.
+        final Var geoSubj = new Var("point");
+        final Var geoPred = new Var("geo");
+        final Var geoObj = new Var("wkt");
+        final StatementPattern geoSP = new StatementPattern(geoSubj, geoPred, geoObj);
+
+        final Var timeSubj = new Var("time");
+        final Var timePred = new Var("-const-http://www.w3.org/2006/time#inXSDDateTime", ValueFactoryImpl.getInstance().createURI("-const-http://www.w3.org/2006/time#inXSDDateTime"));
+        final Var timeObj = new Var("time");
+        final StatementPattern timeSP = new StatementPattern(timeSubj, timePred, timeObj);
+        // This will fail.
+        new EventQueryNode.EventQueryNodeBuilder()
+        .setStorage(mock(EventStorage.class))
+        .setGeoPattern(geoSP)
+        .setTemporalPattern(timeSP)
+        .setGeoFilters(new ArrayList<IndexingExpr>())
+        .setTemporalFilters(new ArrayList<IndexingExpr>())
+        .setUsedFilters(new ArrayList<>())
+        .build();
+    }
+
+    @Test
+    public void evaluate_constantSubject() throws Exception {
+        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);
+        final Event event = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        subject = new RyaURI("urn:event-2222");
+        final Event otherEvent = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        storage.create(event);
+        storage.create(otherEvent);
+
+        final String query =
+                "PREFIX time: <http://www.w3.org/2006/time#> \n"
+              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+              + "SELECT ?event ?time ?point ?wkt "
+              + "WHERE { "
+                + "  <urn:event-1111> time:atTime ?time . "
+                + "  <urn:event-1111> geo:asWKT ?wkt . "
+                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                + "  FILTER(tempo:equals(?time, \"" + temp.toString() + "\")) "
+              + "}";
+
+        final EventQueryNode node = buildNode(storage, query);
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("wkt", VF.createLiteral("POINT (1 1)"));
+        expected.addBinding("time", VF.createLiteral(temp.toString()));
+        int count = 0;
+        assertTrue(rez.hasNext());
+        while(rez.hasNext()) {
+            assertEquals(expected, rez.next());
+            count++;
+        }
+        assertEquals(1, count);
+    }
+
+    @Test
+    public void evaluate_variableSubject() throws Exception {
+        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);
+        final Event event = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        subject = new RyaURI("urn:event-2222");
+        geo = GF.createPoint(new Coordinate(-1, -1));
+        final Event otherEvent = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        storage.create(event);
+        storage.create(otherEvent);
+
+        final String query =
+                "PREFIX time: <http://www.w3.org/2006/time#> \n"
+              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+              + "SELECT ?event ?time ?point ?wkt "
+              + "WHERE { "
+                + "  ?event time:atTime ?time . "
+                + "  ?event geo:asWKT ?wkt . "
+                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+              + "}";
+
+        final EventQueryNode node = buildNode(storage, query);
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
+        final MapBindingSet expected1 = new MapBindingSet();
+        expected1.addBinding("wkt", VF.createLiteral("POINT (1 1)"));
+        expected1.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
+        final MapBindingSet expected2 = new MapBindingSet();
+        expected2.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
+        expected2.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
+
+        final List<BindingSet> actual = new ArrayList<>();
+        while(rez.hasNext()) {
+            actual.add(rez.next());
+        }
+        assertEquals(expected1, actual.get(0));
+        assertEquals(expected2, actual.get(1));
+        assertEquals(2, actual.size());
+    }
+
+    @Test
+    public void evaluate_variableSubject_existingBindingset() throws Exception {
+        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);
+        final Event event = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        subject = new RyaURI("urn:event-2222");
+        geo = GF.createPoint(new Coordinate(-1, -1));
+        final Event otherEvent = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        storage.create(event);
+        storage.create(otherEvent);
+
+        final String query =
+                "PREFIX time: <http://www.w3.org/2006/time#> \n"
+              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+              + "SELECT ?event ?time ?point ?wkt "
+              + "WHERE { "
+                + "  ?event time:atTime ?time . "
+                + "  ?event geo:asWKT ?wkt . "
+                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+              + "}";
+
+        final EventQueryNode node = buildNode(storage, query);
+        final MapBindingSet existingBindings = new MapBindingSet();
+        existingBindings.addBinding("event", VF.createURI("urn:event-2222"));
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(existingBindings);
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
+        expected.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
+
+        final List<BindingSet> actual = new ArrayList<>();
+        while(rez.hasNext()) {
+            actual.add(rez.next());
+        }
+        assertEquals(1, actual.size());
+        assertEquals(expected, actual.get(0));
+    }
+
+    @Test
+    public void evaluate_variableSubject_existingBindingsetWrongFilters() throws Exception {
+        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);
+        final Event event = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        subject = new RyaURI("urn:event-2222");
+        geo = GF.createPoint(new Coordinate(-10, -10));
+        final Event otherEvent = Event.builder()
+            .setSubject(subject)
+            .setGeometry(geo)
+            .setTemporalInstant(temp)
+            .build();
+
+        storage.create(event);
+        storage.create(otherEvent);
+
+        final String query =
+                "PREFIX time: <http://www.w3.org/2006/time#> \n"
+              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+              + "SELECT ?event ?time ?point ?wkt "
+              + "WHERE { "
+                + "  ?event time:atTime ?time . "
+                + "  ?event geo:asWKT ?wkt . "
+                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+              + "}";
+
+        final EventQueryNode node = buildNode(storage, query);
+        final MapBindingSet existingBindings = new MapBindingSet();
+        existingBindings.addBinding("event", VF.createURI("urn:event-2222"));
+        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(existingBindings);
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
+        expected.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
+
+        assertFalse(rez.hasNext());
+    }
+
+    private EventQueryNode buildNode(final EventStorage store, final String query) throws Exception {
+        final List<IndexingExpr> geoFilters = new ArrayList<>();
+        final List<IndexingExpr> temporalFilters = new ArrayList<>();
+        final List<StatementPattern> sps = getSps(query);
+        final List<FunctionCall> filters = getFilters(query);
+        for(final FunctionCall filter : filters) {
+            final URI filterURI = new URIImpl(filter.getURI());
+            final Var objVar = IndexingFunctionRegistry.getResultVarFromFunctionCall(filterURI, filter.getArgs());
+            final IndexingExpr expr = new IndexingExpr(filterURI, sps.get(0), extractArguments(objVar.getName(), filter));
+            if(IndexingFunctionRegistry.getFunctionType(filterURI) == FUNCTION_TYPE.GEO) {
+                geoFilters.add(expr);
+            } else {
+                temporalFilters.add(expr);
+            }
+        }
+
+        final StatementPattern geoPattern = sps.get(1);
+        final StatementPattern temporalPattern = sps.get(0);
+
+        return new EventQueryNode.EventQueryNodeBuilder()
+            .setStorage(store)
+            .setGeoPattern(geoPattern)
+            .setTemporalPattern(temporalPattern)
+            .setGeoFilters(geoFilters)
+            .setTemporalFilters(temporalFilters)
+            .setUsedFilters(filters)
+            .build();
+    }
+
+    private Value[] extractArguments(final String matchName, final FunctionCall call) {
+        final Value args[] = new Value[call.getArgs().size() - 1];
+        int argI = 0;
+        for (int i = 0; i != call.getArgs().size(); ++i) {
+            final ValueExpr arg = call.getArgs().get(i);
+            if (argI == i && arg instanceof Var && matchName.equals(((Var)arg).getName())) {
+                continue;
+            }
+            if (arg instanceof ValueConstant) {
+                args[argI] = ((ValueConstant)arg).getValue();
+            } else if (arg instanceof Var && ((Var)arg).hasValue()) {
+                args[argI] = ((Var)arg).getValue();
+            } else {
+                throw new IllegalArgumentException("Query error: Found " + arg + ", expected a Literal, BNode or URI");
+            }
+            ++argI;
+        }
+        return args;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
deleted file mode 100644
index f0099c8..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
+++ /dev/null
@@ -1,364 +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.geotemporal.model;
-
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getFilters;
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getSps;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.indexing.IndexingExpr;
-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.mongo.MongoEventStorage;
-import org.apache.rya.indexing.geotemporal.storage.EventStorage;
-import org.apache.rya.mongodb.MongoTestBase;
-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.query.BindingSet;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.algebra.FunctionCall;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.ValueConstant;
-import org.openrdf.query.algebra.ValueExpr;
-import org.openrdf.query.algebra.Var;
-import org.openrdf.query.impl.MapBindingSet;
-
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.GeometryFactory;
-import com.vividsolutions.jts.geom.PrecisionModel;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Unit tests the methods of {@link EventQueryNode}.
- */
-public class EventQueryNodeTest extends MongoTestBase {
-    private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
-    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_differentSubjects() throws Exception {
-        final Var geoSubj = new Var("point");
-        final Var geoPred = new Var("-const-http://www.opengis.net/ont/geosparql#asWKT", ValueFactoryImpl.getInstance().createURI("http://www.opengis.net/ont/geosparql#asWKT"));
-        final Var geoObj = new Var("wkt");
-        final StatementPattern geoSP = new StatementPattern(geoSubj, geoPred, geoObj);
-
-        final Var timeSubj = new Var("time");
-        final Var timePred = new Var("-const-http://www.w3.org/2006/time#inXSDDateTime", ValueFactoryImpl.getInstance().createURI("-const-http://www.w3.org/2006/time#inXSDDateTime"));
-        final Var timeObj = new Var("time");
-        final StatementPattern timeSP = new StatementPattern(timeSubj, timePred, timeObj);
-        // This will fail.
-        new EventQueryNode.EventQueryNodeBuilder()
-            .setStorage(mock(EventStorage.class))
-            .setGeoPattern(geoSP)
-            .setTemporalPattern(timeSP)
-            .setGeoFilters(new ArrayList<IndexingExpr>())
-            .setTemporalFilters(new ArrayList<IndexingExpr>())
-            .setUsedFilters(new ArrayList<>())
-            .build();
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void constructor_variablePredicate() throws Exception {
-        // A pattern that has a variable for its predicate.
-        final Var geoSubj = new Var("point");
-        final Var geoPred = new Var("geo");
-        final Var geoObj = new Var("wkt");
-        final StatementPattern geoSP = new StatementPattern(geoSubj, geoPred, geoObj);
-
-        final Var timeSubj = new Var("time");
-        final Var timePred = new Var("-const-http://www.w3.org/2006/time#inXSDDateTime", ValueFactoryImpl.getInstance().createURI("-const-http://www.w3.org/2006/time#inXSDDateTime"));
-        final Var timeObj = new Var("time");
-        final StatementPattern timeSP = new StatementPattern(timeSubj, timePred, timeObj);
-        // This will fail.
-        new EventQueryNode.EventQueryNodeBuilder()
-        .setStorage(mock(EventStorage.class))
-        .setGeoPattern(geoSP)
-        .setTemporalPattern(timeSP)
-        .setGeoFilters(new ArrayList<IndexingExpr>())
-        .setTemporalFilters(new ArrayList<IndexingExpr>())
-        .setUsedFilters(new ArrayList<>())
-        .build();
-    }
-
-    @Test
-    public void evaluate_constantSubject() throws Exception {
-        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);
-        final Event event = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        subject = new RyaURI("urn:event-2222");
-        final Event otherEvent = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        storage.create(event);
-        storage.create(otherEvent);
-
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT ?event ?time ?point ?wkt "
-              + "WHERE { "
-                + "  <urn:event-1111> time:atTime ?time . "
-                + "  <urn:event-1111> geo:asWKT ?wkt . "
-                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"" + temp.toString() + "\")) "
-              + "}";
-
-        final EventQueryNode node = buildNode(storage, query);
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("wkt", VF.createLiteral("POINT (1 1)"));
-        expected.addBinding("time", VF.createLiteral(temp.toString()));
-        int count = 0;
-        assertTrue(rez.hasNext());
-        while(rez.hasNext()) {
-            assertEquals(expected, rez.next());
-            count++;
-        }
-        assertEquals(1, count);
-    }
-
-    @Test
-    public void evaluate_variableSubject() throws Exception {
-        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);
-        final Event event = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        subject = new RyaURI("urn:event-2222");
-        geo = GF.createPoint(new Coordinate(-1, -1));
-        final Event otherEvent = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        storage.create(event);
-        storage.create(otherEvent);
-
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT ?event ?time ?point ?wkt "
-              + "WHERE { "
-                + "  ?event time:atTime ?time . "
-                + "  ?event geo:asWKT ?wkt . "
-                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-              + "}";
-
-        final EventQueryNode node = buildNode(storage, query);
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(new MapBindingSet());
-        final MapBindingSet expected1 = new MapBindingSet();
-        expected1.addBinding("wkt", VF.createLiteral("POINT (1 1)"));
-        expected1.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
-        final MapBindingSet expected2 = new MapBindingSet();
-        expected2.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
-        expected2.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
-
-        final List<BindingSet> actual = new ArrayList<>();
-        while(rez.hasNext()) {
-            actual.add(rez.next());
-        }
-        assertEquals(expected1, actual.get(0));
-        assertEquals(expected2, actual.get(1));
-        assertEquals(2, actual.size());
-    }
-
-    @Test
-    public void evaluate_variableSubject_existingBindingset() throws Exception {
-        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);
-        final Event event = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        subject = new RyaURI("urn:event-2222");
-        geo = GF.createPoint(new Coordinate(-1, -1));
-        final Event otherEvent = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        storage.create(event);
-        storage.create(otherEvent);
-
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT ?event ?time ?point ?wkt "
-              + "WHERE { "
-                + "  ?event time:atTime ?time . "
-                + "  ?event geo:asWKT ?wkt . "
-                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-              + "}";
-
-        final EventQueryNode node = buildNode(storage, query);
-        final MapBindingSet existingBindings = new MapBindingSet();
-        existingBindings.addBinding("event", VF.createURI("urn:event-2222"));
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(existingBindings);
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
-        expected.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
-
-        final List<BindingSet> actual = new ArrayList<>();
-        while(rez.hasNext()) {
-            actual.add(rez.next());
-        }
-        assertEquals(1, actual.size());
-        assertEquals(expected, actual.get(0));
-    }
-
-    @Test
-    public void evaluate_variableSubject_existingBindingsetWrongFilters() throws Exception {
-        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);
-        final Event event = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        subject = new RyaURI("urn:event-2222");
-        geo = GF.createPoint(new Coordinate(-10, -10));
-        final Event otherEvent = Event.builder()
-            .setSubject(subject)
-            .setGeometry(geo)
-            .setTemporalInstant(temp)
-            .build();
-
-        storage.create(event);
-        storage.create(otherEvent);
-
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT ?event ?time ?point ?wkt "
-              + "WHERE { "
-                + "  ?event time:atTime ?time . "
-                + "  ?event geo:asWKT ?wkt . "
-                + "  FILTER(geof:sfWithin(?wkt, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-              + "}";
-
-        final EventQueryNode node = buildNode(storage, query);
-        final MapBindingSet existingBindings = new MapBindingSet();
-        existingBindings.addBinding("event", VF.createURI("urn:event-2222"));
-        final CloseableIteration<BindingSet, QueryEvaluationException> rez = node.evaluate(existingBindings);
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("wkt", VF.createLiteral("POINT (-1 -1)"));
-        expected.addBinding("time", VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString()));
-
-        assertFalse(rez.hasNext());
-    }
-
-    private EventQueryNode buildNode(final EventStorage store, final String query) throws Exception {
-        final List<IndexingExpr> geoFilters = new ArrayList<>();
-        final List<IndexingExpr> temporalFilters = new ArrayList<>();
-        final List<StatementPattern> sps = getSps(query);
-        final List<FunctionCall> filters = getFilters(query);
-        for(final FunctionCall filter : filters) {
-            final URI filterURI = new URIImpl(filter.getURI());
-            final Var objVar = IndexingFunctionRegistry.getResultVarFromFunctionCall(filterURI, filter.getArgs());
-            final IndexingExpr expr = new IndexingExpr(filterURI, sps.get(0), extractArguments(objVar.getName(), filter));
-            if(IndexingFunctionRegistry.getFunctionType(filterURI) == FUNCTION_TYPE.GEO) {
-                geoFilters.add(expr);
-            } else {
-                temporalFilters.add(expr);
-            }
-        }
-
-        final StatementPattern geoPattern = sps.get(1);
-        final StatementPattern temporalPattern = sps.get(0);
-
-        return new EventQueryNode.EventQueryNodeBuilder()
-            .setStorage(store)
-            .setGeoPattern(geoPattern)
-            .setTemporalPattern(temporalPattern)
-            .setGeoFilters(geoFilters)
-            .setTemporalFilters(temporalFilters)
-            .setUsedFilters(filters)
-            .build();
-    }
-
-    private Value[] extractArguments(final String matchName, final FunctionCall call) {
-        final Value args[] = new Value[call.getArgs().size() - 1];
-        int argI = 0;
-        for (int i = 0; i != call.getArgs().size(); ++i) {
-            final ValueExpr arg = call.getArgs().get(i);
-            if (argI == i && arg instanceof Var && matchName.equals(((Var)arg).getName())) {
-                continue;
-            }
-            if (arg instanceof ValueConstant) {
-                args[argI] = ((ValueConstant)arg).getValue();
-            } else if (arg instanceof Var && ((Var)arg).hasValue()) {
-                args[argI] = ((Var)arg).getValue();
-            } else {
-                throw new IllegalArgumentException("Query error: Found " + arg + ", expected a Literal, BNode or URI");
-            }
-            ++argI;
-        }
-        return args;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
index 392d381..125a2e4 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
@@ -18,6 +18,10 @@ l * Licensed to the Apache Software Foundation (ASF) under one
  */
 package org.apache.rya.indexing.geotemporal.mongo;
 
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.assertEqualMongo;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getFilters;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getSps;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -28,7 +32,6 @@ import org.apache.rya.indexing.IndexingFunctionRegistry;
 import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.GeoPolicy;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.TemporalPolicy;
-import org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -56,7 +59,7 @@ import com.mongodb.util.JSON;
  * @see TemporalPolicy Temporal Filter Functions
  * @see GeoPolicy Geo Filter Functions
  */
-public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestUtils {
+public class GeoTemporalMongoDBStorageStrategyTest {
     private GeoTemporalMongoDBStorageStrategy adapter;
     @Before
     public void setup() {


[11/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageIT.java
new file mode 100644
index 0000000..c7d8a81
--- /dev/null
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageIT.java
@@ -0,0 +1,198 @@
+/**
+ * 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.geotemporal.mongo;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Optional;
+
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.indexing.TemporalInstant;
+import org.apache.rya.indexing.TemporalInstantRfc3339;
+import org.apache.rya.indexing.geotemporal.model.Event;
+import org.apache.rya.indexing.geotemporal.storage.EventStorage;
+import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventAlreadyExistsException;
+import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventStorageException;
+import org.apache.rya.mongodb.MongoITBase;
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.GeometryFactory;
+import com.vividsolutions.jts.geom.PrecisionModel;
+
+/**
+ * Integration tests the methods of {@link MongoEventStorage}.
+ */
+public class MongoEventStorageIT extends MongoITBase {
+
+    private static final String RYA_INSTANCE_NAME = "testInstance";
+    private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
+
+    @Test
+    public void create_and_get() throws Exception {
+        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
+        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
+
+        // An Event that will be stored.
+        final Event event = Event.builder()
+                .setSubject(new RyaURI("urn:event/001"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        // Create it.
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        storage.create(event);
+
+        // Get it.
+        final Optional<Event> storedEvent = storage.get(new RyaURI("urn:event/001"));
+
+        // Verify the correct value was returned.
+        assertEquals(event, storedEvent.get());
+    }
+
+    @Test
+    public void can_not_create_with_same_subject() throws Exception {
+        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
+        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
+
+        // An Event that will be stored.
+        final Event event = Event.builder()
+                .setSubject(new RyaURI("urn:event/001"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        // Create it.
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        storage.create(event);
+
+        // Try to create it again. This will fail.
+        boolean failed = false;
+        try {
+            storage.create(event);
+        } catch(final EventAlreadyExistsException e) {
+            failed = true;
+        }
+        assertTrue(failed);
+    }
+
+    @Test
+    public void get_noneExisting() throws Exception {
+        // Get a Type that hasn't been created.
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        final Optional<Event> storedEvent = storage.get(new RyaURI("urn:event/000"));
+
+        // Verify nothing was returned.
+        assertFalse(storedEvent.isPresent());
+    }
+
+    @Test
+    public void delete() throws Exception {
+        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
+        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
+
+        // An Event that will be stored.
+        final Event event = Event.builder()
+                .setSubject(new RyaURI("urn:event/002"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        // Create it.
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        storage.create(event);
+
+        // Delete it.
+        final boolean deleted = storage.delete( new RyaURI("urn:event/002") );
+
+        // Verify a document was deleted.
+        assertTrue( deleted );
+    }
+
+    @Test
+    public void delete_nonExisting() throws Exception {
+        // Delete an Event that has not been created.
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        final boolean deleted = storage.delete( new RyaURI("urn:event/003") );
+
+        // Verify no document was deleted.
+        assertFalse( deleted );
+    }
+
+    @Test
+    public void update() throws Exception {
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
+        TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
+
+        // An Event that will be stored.
+        final Event event = Event.builder()
+                .setSubject(new RyaURI("urn:event/004"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        storage.create(event);
+
+        // Show Alice was stored.
+        Optional<Event> latest = storage.get(new RyaURI("urn:event/004"));
+        assertEquals(event, latest.get());
+
+        instant = new TemporalInstantRfc3339(DateTime.now());
+        // Change Alice's eye color to brown.
+        final Event updated = Event.builder(event)
+                .setTemporalInstant(instant)
+                .build();
+
+        storage.update(event, updated);
+
+        // Fetch the Alice object and ensure it has the new value.
+        latest = storage.get(new RyaURI("urn:event/004"));
+
+        assertEquals(updated, latest.get());
+    }
+
+    @Test(expected = EventStorageException.class)
+    public void update_differentSubjects() throws Exception {
+        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
+        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
+        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
+
+        // Two objects that do not have the same Subjects.
+        final Event old = Event.builder()
+                .setSubject(new RyaURI("urn:event/001"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        final Event updated = Event.builder()
+                .setSubject(new RyaURI("urn:event/002"))
+                .setGeometry(geo)
+                .setTemporalInstant(instant)
+                .build();
+
+        // The update will fail.
+        storage.update(old, updated);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
deleted file mode 100644
index 6bd0b6d..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
+++ /dev/null
@@ -1,198 +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.geotemporal.mongo;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Optional;
-
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.indexing.TemporalInstant;
-import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.geotemporal.model.Event;
-import org.apache.rya.indexing.geotemporal.storage.EventStorage;
-import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventAlreadyExistsException;
-import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventStorageException;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.joda.time.DateTime;
-import org.junit.Test;
-
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.GeometryFactory;
-import com.vividsolutions.jts.geom.PrecisionModel;
-
-/**
- * Integration tests the methods of {@link MongoEventStorage}.
- */
-public class MongoEventStorageTest extends MongoTestBase {
-
-    private static final String RYA_INSTANCE_NAME = "testInstance";
-    private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
-
-    @Test
-    public void create_and_get() throws Exception {
-        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
-        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
-
-        // An Event that will be stored.
-        final Event event = Event.builder()
-                .setSubject(new RyaURI("urn:event/001"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        // Create it.
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        storage.create(event);
-
-        // Get it.
-        final Optional<Event> storedEvent = storage.get(new RyaURI("urn:event/001"));
-
-        // Verify the correct value was returned.
-        assertEquals(event, storedEvent.get());
-    }
-
-    @Test
-    public void can_not_create_with_same_subject() throws Exception {
-        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
-        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
-
-        // An Event that will be stored.
-        final Event event = Event.builder()
-                .setSubject(new RyaURI("urn:event/001"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        // Create it.
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        storage.create(event);
-
-        // Try to create it again. This will fail.
-        boolean failed = false;
-        try {
-            storage.create(event);
-        } catch(final EventAlreadyExistsException e) {
-            failed = true;
-        }
-        assertTrue(failed);
-    }
-
-    @Test
-    public void get_noneExisting() throws Exception {
-        // Get a Type that hasn't been created.
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        final Optional<Event> storedEvent = storage.get(new RyaURI("urn:event/000"));
-
-        // Verify nothing was returned.
-        assertFalse(storedEvent.isPresent());
-    }
-
-    @Test
-    public void delete() throws Exception {
-        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
-        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
-
-        // An Event that will be stored.
-        final Event event = Event.builder()
-                .setSubject(new RyaURI("urn:event/002"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        // Create it.
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        storage.create(event);
-
-        // Delete it.
-        final boolean deleted = storage.delete( new RyaURI("urn:event/002") );
-
-        // Verify a document was deleted.
-        assertTrue( deleted );
-    }
-
-    @Test
-    public void delete_nonExisting() throws Exception {
-        // Delete an Event that has not been created.
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        final boolean deleted = storage.delete( new RyaURI("urn:event/003") );
-
-        // Verify no document was deleted.
-        assertFalse( deleted );
-    }
-
-    @Test
-    public void update() throws Exception {
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
-        TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
-
-        // An Event that will be stored.
-        final Event event = Event.builder()
-                .setSubject(new RyaURI("urn:event/004"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        storage.create(event);
-
-        // Show Alice was stored.
-        Optional<Event> latest = storage.get(new RyaURI("urn:event/004"));
-        assertEquals(event, latest.get());
-
-        instant = new TemporalInstantRfc3339(DateTime.now());
-        // Change Alice's eye color to brown.
-        final Event updated = Event.builder(event)
-                .setTemporalInstant(instant)
-                .build();
-
-        storage.update(event, updated);
-
-        // Fetch the Alice object and ensure it has the new value.
-        latest = storage.get(new RyaURI("urn:event/004"));
-
-        assertEquals(updated, latest.get());
-    }
-
-    @Test(expected = EventStorageException.class)
-    public void update_differentSubjects() throws Exception {
-        final EventStorage storage = new MongoEventStorage(super.getMongoClient(), RYA_INSTANCE_NAME);
-        final Geometry geo = GF.createPoint(new Coordinate(10, 10));
-        final TemporalInstant instant = new TemporalInstantRfc3339(DateTime.now());
-
-        // Two objects that do not have the same Subjects.
-        final Event old = Event.builder()
-                .setSubject(new RyaURI("urn:event/001"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        final Event updated = Event.builder()
-                .setSubject(new RyaURI("urn:event/002"))
-                .setGeometry(geo)
-                .setTemporalInstant(instant)
-                .build();
-
-        // The update will fail.
-        storage.update(old, updated);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
index 9bfe148..881289a 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
@@ -31,7 +31,7 @@ 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.storage.EventStorage;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -46,7 +46,7 @@ import com.vividsolutions.jts.geom.Geometry;
 /**
  * Integration tests the methods of {@link MongoGeoTemporalIndexer}.
  */
-public class MongoGeoTemporalIndexerIT extends MongoTestBase {
+public class MongoGeoTemporalIndexerIT extends MongoITBase {
     private MongoGeoTemporalIndexer indexer;
 
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
index 11c1d21..f38fc06 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
@@ -31,7 +31,7 @@ import org.apache.rya.indexing.GeoRyaSailFactory;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -56,7 +56,7 @@ import com.vividsolutions.jts.io.ParseException;
 import com.vividsolutions.jts.io.WKTReader;
 import com.vividsolutions.jts.io.WKTWriter;
 
-public class MongoGeoIndexerFilterIT extends MongoTestBase {
+public class MongoGeoIndexerFilterIT extends MongoITBase {
     private static final GeometryFactory GF = new GeometryFactory();
     private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
     private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerIT.java
new file mode 100644
index 0000000..40751ae
--- /dev/null
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerIT.java
@@ -0,0 +1,377 @@
+/**
+ * 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.resolver.RdfToRyaConversions.convertStatement;
+import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.indexing.StatementConstraints;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
+import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ContextStatementImpl;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.Sets;
+import com.vividsolutions.jts.geom.Coordinate;
+import com.vividsolutions.jts.geom.GeometryFactory;
+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;
+
+public class MongoGeoIndexerIT extends MongoITBase {
+    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
+    GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
+
+    @Override
+    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        conf.set(OptionalConfigUtils.USE_GEO, "true");
+    }
+
+    @Test
+    public void testRestrictPredicatesSearch() throws Exception {
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final Point point = gf.createPoint(new Coordinate(10, 10));
+            final Value pointValue = vf.createLiteral("Point(10 10)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final URI invalidPredicate = GeoConstants.GEO_AS_WKT;
+
+            // These should not be stored because they are not in the predicate list
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), invalidPredicate, pointValue)));
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), invalidPredicate, pointValue)));
+
+            final URI pred1 = vf.createURI("pred:1");
+            final URI pred2 = vf.createURI("pred:2");
+
+            // These should be stored because they are in the predicate list
+            final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1, pointValue);
+            final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2, pointValue);
+            f.storeStatement(convertStatement(s3));
+            f.storeStatement(convertStatement(s4));
+
+            // This should not be stored because the object is not valid wkt
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1, vf.createLiteral("soint(10 10)"))));
+
+            // This should not be stored because the object is not a literal
+            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj6"), pred1, vf.createURI("p:Point(10 10)"))));
+
+            f.flush();
+
+            final Set<Statement> actual = getSet(f.queryEquals(point, EMPTY_CONSTRAINTS));
+            assertEquals(2, actual.size());
+            assertTrue(actual.contains(s3));
+            assertTrue(actual.contains(s4));
+        }
+    }
+
+    @Test
+    public void testPrimeMeridianSearch() throws Exception {
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] ONE = { 1, 1, -1, 1, -1, -1, 1, -1, 1, 1 };
+            final double[] TWO = { 2, 2, -2, 2, -2, -2, 2, -2, 2, 2 };
+            final double[] THREE = { 3, 3, -3, 3, -3, -3, 3, -3, 3, 3 };
+
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
+            final LinearRing r2 = gf.createLinearRing(new PackedCoordinateSequence.Double(TWO, 2));
+            final LinearRing r3 = gf.createLinearRing(new PackedCoordinateSequence.Double(THREE, 2));
+
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            final Polygon p2 = gf.createPolygon(r2, new LinearRing[] {});
+            final Polygon p3 = gf.createPolygon(r3, new LinearRing[] {});
+
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p2, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p3, EMPTY_CONSTRAINTS)));
+
+            // Test a ring with a hole in it
+            final Polygon p3m2 = gf.createPolygon(r3, new LinearRing[] { r2 });
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p3m2, EMPTY_CONSTRAINTS)));
+
+            // test a ring outside the point
+            final double[] OUT = { 3, 3, 1, 3, 1, 1, 3, 1, 3, 3 };
+            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
+            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDcSearch() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+
+            // test a ring outside the point
+            final double[] OUT = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
+            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
+            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDeleteSearch() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            f.deleteStatement(convertStatement(statement));
+
+            // test a ring that the point would be inside of if not deleted
+            final double[] in = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(in, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+
+            // test a ring that the point would be outside of if not deleted
+            final double[] out = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
+            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(out, 2));
+            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+
+            // test a ring for the whole world and make sure the point is gone
+            // Geomesa is a little sensitive around lon 180, so we only go to 179
+            final double[] world = { -180, 90, 179, 90, 179, -90, -180, -90, -180, 90 };
+            final LinearRing rWorld = gf.createLinearRing(new PackedCoordinateSequence.Double(world, 2));
+            final Polygon pWorld = gf.createPolygon(rWorld, new LinearRing[] {});
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pWorld, EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithContext() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct context
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setContext(context))));
+
+            // query with wrong context
+            assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithSubject() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct subject
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(subject))));
+
+            // query with wrong subject
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithSubjectAndContext() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct context subject
+            assertEquals(Sets.newHashSet(statement),
+                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(context).setSubject(subject))));
+
+            // query with wrong context
+            assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
+
+            // query with wrong subject
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
+        }
+    }
+
+    @Test
+    public void testDcSearchWithPredicate() throws Exception {
+        // test a ring around dc
+        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource subject = vf.createURI("foo:subj");
+            final URI predicate = GeoConstants.GEO_AS_WKT;
+            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Resource context = vf.createURI("foo:context");
+
+            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
+            f.storeStatement(convertStatement(statement));
+            f.flush();
+
+            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            // query with correct Predicate
+            assertEquals(Sets.newHashSet(statement),
+                    getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(predicate)))));
+
+            // query with wrong predicate
+            assertEquals(Sets.newHashSet(),
+                    getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(vf.createURI("other:pred"))))));
+        }
+    }
+
+    // @Test
+    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.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final Resource context = vf.createURI("foo:context");
+
+            final Resource subjectEast = vf.createURI("foo:subj:east");
+            final URI predicateEast = GeoConstants.GEO_AS_WKT;
+            final Value objectEast = vf.createLiteral("Point(179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Statement statementEast = new ContextStatementImpl(subjectEast, predicateEast, objectEast, context);
+            f.storeStatement(convertStatement(statementEast));
+
+            final Resource subjectWest = vf.createURI("foo:subj:west");
+            final URI predicateWest = GeoConstants.GEO_AS_WKT;
+            final Value objectWest = vf.createLiteral("Point(-179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+            final Statement statementWest = new ContextStatementImpl(subjectWest, predicateWest, objectWest, context);
+            f.storeStatement(convertStatement(statementWest));
+
+            f.flush();
+
+            final double[] ONE = { 178.1, 1, -178, 1, -178, -1, 178.1, -1, 178.1, 1 };
+
+            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
+
+            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
+
+            assertEquals(Sets.newHashSet(statementEast, statementWest), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfIT.java
new file mode 100644
index 0000000..a544a78
--- /dev/null
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfIT.java
@@ -0,0 +1,262 @@
+package org.apache.rya.indexing.mongo;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.bbox;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.line;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.point;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.poly;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.api.resolver.RyaToRdfConversions;
+import org.apache.rya.indexing.GeoConstants;
+import org.apache.rya.indexing.StatementConstraints;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
+import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.openrdf.model.Resource;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.Value;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.vividsolutions.jts.geom.Geometry;
+import com.vividsolutions.jts.geom.LineString;
+import com.vividsolutions.jts.geom.Point;
+import com.vividsolutions.jts.geom.Polygon;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Tests all of the "simple functions" of the geoindexer.
+ */
+public class MongoGeoIndexerSfIT extends MongoITBase {
+    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
+
+    // Here is the landscape:
+    /**
+     * <pre>
+     *   +---+---+---+---+---+---+---+
+     *   |        F          |       |
+     *   +  A    +           +   C   +
+     *   |                   |       |
+     *   +---+---+    E      +---+---+
+     *   |       |   /       |
+     *   +   B   +  /+---+---+
+     *   |       | / |       |
+     *   +---+---+/--+---+---+
+     *           /   |     D |
+     *          /    +---+---+
+     * </pre>
+     **/
+
+    private static final Polygon A = poly(bbox(0, 1, 4, 5));
+    private static final Polygon B = poly(bbox(0, 1, 2, 3));
+    private static final Polygon C = poly(bbox(4, 3, 6, 5));
+    private static final Polygon D = poly(bbox(3, 0, 5, 2));
+
+    private static final Point F = point(2, 4);
+
+    private static final LineString E = line(2, 0, 3, 3);
+
+    private static final Map<Geometry, String> names = Maps.newHashMap();
+    static {
+        names.put(A, "A");
+        names.put(B, "B");
+        names.put(C, "C");
+        names.put(D, "D");
+        names.put(E, "E");
+        names.put(F, "F");
+    }
+
+    @Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        conf.set(OptionalConfigUtils.USE_GEO, "true");
+    }
+
+    private static RyaStatement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("uri:" + names.get(geo));
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final Value object = vf.createLiteral(geo.toString(), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+
+    }
+
+    public void compare(final CloseableIteration<Statement, ?> actual, final Geometry... expected) throws Exception {
+        final Set<Statement> expectedSet = Sets.newHashSet();
+        for (final Geometry geo : expected) {
+            expectedSet.add(RyaToRdfConversions.convertStatement(statement(geo)));
+        }
+
+        Assert.assertEquals(expectedSet, getSet(actual));
+    }
+
+    private static Geometry[] EMPTY_RESULTS = {};
+
+    @Test
+    public void testEquals() throws Exception {
+        try(final MongoGeoIndexer g = new MongoGeoIndexer()) {
+            g.setConf(conf);
+            g.init();
+
+            g.storeStatement(statement(A));
+            g.storeStatement(statement(B));
+            g.storeStatement(statement(C));
+            g.storeStatement(statement(D));
+            g.storeStatement(statement(F));
+            g.storeStatement(statement(E));
+            // point
+            compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
+            compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+            // line
+            compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
+            compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+            // poly
+            compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
+            compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+        }
+    }
+
+//    @Test
+//    public void testDisjoint() throws Exception {
+//        // point
+//        compare(g.queryDisjoint(F, EMPTY_CONSTRAINTS), B, C, D, E);
+//
+//        // line
+//        compare(g.queryDisjoint(E, EMPTY_CONSTRAINTS), B, C, D, F);
+//
+//        // poly
+//        compare(g.queryDisjoint(A, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//        compare(g.queryDisjoint(B, EMPTY_CONSTRAINTS), C, D, F, E);
+//    }
+
+//    @Test
+//    public void testIntersectsPoint() throws Exception {
+        // This seems like a bug
+        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), A, F);
+        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+//
+//    @Test
+//    public void testIntersectsLine() throws Exception {
+        // This seems like a bug
+        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), A, E);
+        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testIntersectsPoly() throws Exception {
+//        compare(g.queryIntersects(A, EMPTY_CONSTRAINTS), A, B, C, D, F, E);
+//    }
+
+//    @Test
+//    public void testTouchesPoint() throws Exception {
+//        compare(g.queryTouches(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+//
+//    @Test
+//    public void testTouchesLine() throws Exception {
+//        compare(g.queryTouches(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testTouchesPoly() throws Exception {
+//        compare(g.queryTouches(A, EMPTY_CONSTRAINTS), C);
+//    }
+
+//    @Test
+//    public void testCrossesPoint() throws Exception {
+//        compare(g.queryCrosses(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testCrossesLine() throws Exception {
+        // compare(g.queryCrosses(E, EMPTY_CONSTRAINTS), A);
+//    }
+
+//    @Test
+//    public void testCrossesPoly() throws Exception {
+//        compare(g.queryCrosses(A, EMPTY_CONSTRAINTS), E);
+//    }
+
+//    @Test
+//    public void testWithin() throws Exception {
+//        // point
+//  //      compare(g.queryWithin(F, EMPTY_CONSTRAINTS), F);
+//
+//        // line
+////        compare(g.queryWithin(E, EMPTY_CONSTRAINTS), E);
+//
+//        // poly
+//        compare(g.queryWithin(A, EMPTY_CONSTRAINTS), A, B, F);
+//    }
+
+//    @Test
+//    public void testContainsPoint() throws Exception {
+//        compare(g.queryContains(F, EMPTY_CONSTRAINTS), A, F);
+//    }
+
+//    @Test
+//    public void testContainsLine() throws Exception {
+        // compare(g.queryContains(E, EMPTY_CONSTRAINTS), E);
+//    }
+
+//    @Test
+//    public void testContainsPoly() throws Exception {
+//        compare(g.queryContains(A, EMPTY_CONSTRAINTS), A);
+//        compare(g.queryContains(B, EMPTY_CONSTRAINTS), A, B);
+//    }
+//
+//    @Test
+//    public void testOverlapsPoint() throws Exception {
+        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), F);
+        // You cannot have overlapping points
+        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testOverlapsLine() throws Exception {
+        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), A, E);
+        // You cannot have overlapping lines
+        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+//    }
+
+//    @Test
+//    public void testOverlapsPoly() throws Exception {
+//        compare(g.queryOverlaps(A, EMPTY_CONSTRAINTS), D);
+//    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
deleted file mode 100644
index bc4d870..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
+++ /dev/null
@@ -1,262 +0,0 @@
-package org.apache.rya.indexing.mongo;
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.bbox;
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.line;
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.point;
-import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.poly;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.resolver.RdfToRyaConversions;
-import org.apache.rya.api.resolver.RyaToRdfConversions;
-import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
-import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Assert;
-import org.junit.Test;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.LineString;
-import com.vividsolutions.jts.geom.Point;
-import com.vividsolutions.jts.geom.Polygon;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Tests all of the "simple functions" of the geoindexer.
- */
-public class MongoGeoIndexerSfTest extends MongoTestBase {
-    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-
-    // Here is the landscape:
-    /**
-     * <pre>
-     *   +---+---+---+---+---+---+---+
-     *   |        F          |       |
-     *   +  A    +           +   C   +
-     *   |                   |       |
-     *   +---+---+    E      +---+---+
-     *   |       |   /       |
-     *   +   B   +  /+---+---+
-     *   |       | / |       |
-     *   +---+---+/--+---+---+
-     *           /   |     D |
-     *          /    +---+---+
-     * </pre>
-     **/
-
-    private static final Polygon A = poly(bbox(0, 1, 4, 5));
-    private static final Polygon B = poly(bbox(0, 1, 2, 3));
-    private static final Polygon C = poly(bbox(4, 3, 6, 5));
-    private static final Polygon D = poly(bbox(3, 0, 5, 2));
-
-    private static final Point F = point(2, 4);
-
-    private static final LineString E = line(2, 0, 3, 3);
-
-    private static final Map<Geometry, String> names = Maps.newHashMap();
-    static {
-        names.put(A, "A");
-        names.put(B, "B");
-        names.put(C, "C");
-        names.put(D, "D");
-        names.put(E, "E");
-        names.put(F, "F");
-    }
-
-    @Override
-	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
-        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
-        conf.set(OptionalConfigUtils.USE_GEO, "true");
-    }
-
-    private static RyaStatement statement(final Geometry geo) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("uri:" + names.get(geo));
-        final URI predicate = GeoConstants.GEO_AS_WKT;
-        final Value object = vf.createLiteral(geo.toString(), GeoConstants.XMLSCHEMA_OGC_WKT);
-        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
-
-    }
-
-    public void compare(final CloseableIteration<Statement, ?> actual, final Geometry... expected) throws Exception {
-        final Set<Statement> expectedSet = Sets.newHashSet();
-        for (final Geometry geo : expected) {
-            expectedSet.add(RyaToRdfConversions.convertStatement(statement(geo)));
-        }
-
-        Assert.assertEquals(expectedSet, getSet(actual));
-    }
-
-    private static Geometry[] EMPTY_RESULTS = {};
-
-    @Test
-    public void testEquals() throws Exception {
-        try(final MongoGeoIndexer g = new MongoGeoIndexer()) {
-            g.setConf(conf);
-            g.init();
-
-            g.storeStatement(statement(A));
-            g.storeStatement(statement(B));
-            g.storeStatement(statement(C));
-            g.storeStatement(statement(D));
-            g.storeStatement(statement(F));
-            g.storeStatement(statement(E));
-            // point
-            compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
-            compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-            // line
-            compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
-            compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-            // poly
-            compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
-            compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-        }
-    }
-
-//    @Test
-//    public void testDisjoint() throws Exception {
-//        // point
-//        compare(g.queryDisjoint(F, EMPTY_CONSTRAINTS), B, C, D, E);
-//
-//        // line
-//        compare(g.queryDisjoint(E, EMPTY_CONSTRAINTS), B, C, D, F);
-//
-//        // poly
-//        compare(g.queryDisjoint(A, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//        compare(g.queryDisjoint(B, EMPTY_CONSTRAINTS), C, D, F, E);
-//    }
-
-//    @Test
-//    public void testIntersectsPoint() throws Exception {
-        // This seems like a bug
-        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), A, F);
-        // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-//
-//    @Test
-//    public void testIntersectsLine() throws Exception {
-        // This seems like a bug
-        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), A, E);
-        // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-
-//    @Test
-//    public void testIntersectsPoly() throws Exception {
-//        compare(g.queryIntersects(A, EMPTY_CONSTRAINTS), A, B, C, D, F, E);
-//    }
-
-//    @Test
-//    public void testTouchesPoint() throws Exception {
-//        compare(g.queryTouches(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-//
-//    @Test
-//    public void testTouchesLine() throws Exception {
-//        compare(g.queryTouches(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-
-//    @Test
-//    public void testTouchesPoly() throws Exception {
-//        compare(g.queryTouches(A, EMPTY_CONSTRAINTS), C);
-//    }
-
-//    @Test
-//    public void testCrossesPoint() throws Exception {
-//        compare(g.queryCrosses(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-
-//    @Test
-//    public void testCrossesLine() throws Exception {
-        // compare(g.queryCrosses(E, EMPTY_CONSTRAINTS), A);
-//    }
-
-//    @Test
-//    public void testCrossesPoly() throws Exception {
-//        compare(g.queryCrosses(A, EMPTY_CONSTRAINTS), E);
-//    }
-
-//    @Test
-//    public void testWithin() throws Exception {
-//        // point
-//  //      compare(g.queryWithin(F, EMPTY_CONSTRAINTS), F);
-//
-//        // line
-////        compare(g.queryWithin(E, EMPTY_CONSTRAINTS), E);
-//
-//        // poly
-//        compare(g.queryWithin(A, EMPTY_CONSTRAINTS), A, B, F);
-//    }
-
-//    @Test
-//    public void testContainsPoint() throws Exception {
-//        compare(g.queryContains(F, EMPTY_CONSTRAINTS), A, F);
-//    }
-
-//    @Test
-//    public void testContainsLine() throws Exception {
-        // compare(g.queryContains(E, EMPTY_CONSTRAINTS), E);
-//    }
-
-//    @Test
-//    public void testContainsPoly() throws Exception {
-//        compare(g.queryContains(A, EMPTY_CONSTRAINTS), A);
-//        compare(g.queryContains(B, EMPTY_CONSTRAINTS), A, B);
-//    }
-//
-//    @Test
-//    public void testOverlapsPoint() throws Exception {
-        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), F);
-        // You cannot have overlapping points
-        // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-
-//    @Test
-//    public void testOverlapsLine() throws Exception {
-        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), A, E);
-        // You cannot have overlapping lines
-        // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-//    }
-
-//    @Test
-//    public void testOverlapsPoly() throws Exception {
-//        compare(g.queryOverlaps(A, EMPTY_CONSTRAINTS), D);
-//    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
deleted file mode 100644
index 5953132..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
+++ /dev/null
@@ -1,377 +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.resolver.RdfToRyaConversions.convertStatement;
-import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.rya.indexing.GeoConstants;
-import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
-import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Test;
-import org.openrdf.model.Resource;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.Value;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ContextStatementImpl;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-
-import com.google.common.collect.Sets;
-import com.vividsolutions.jts.geom.Coordinate;
-import com.vividsolutions.jts.geom.GeometryFactory;
-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;
-
-public class MongoGeoIndexerTest extends MongoTestBase {
-    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-    GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
-
-    @Override
-    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
-        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
-        conf.set(OptionalConfigUtils.USE_GEO, "true");
-    }
-
-    @Test
-    public void testRestrictPredicatesSearch() throws Exception {
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            final Point point = gf.createPoint(new Coordinate(10, 10));
-            final Value pointValue = vf.createLiteral("Point(10 10)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final URI invalidPredicate = GeoConstants.GEO_AS_WKT;
-
-            // These should not be stored because they are not in the predicate list
-            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), invalidPredicate, pointValue)));
-            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), invalidPredicate, pointValue)));
-
-            final URI pred1 = vf.createURI("pred:1");
-            final URI pred2 = vf.createURI("pred:2");
-
-            // These should be stored because they are in the predicate list
-            final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1, pointValue);
-            final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2, pointValue);
-            f.storeStatement(convertStatement(s3));
-            f.storeStatement(convertStatement(s4));
-
-            // This should not be stored because the object is not valid wkt
-            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1, vf.createLiteral("soint(10 10)"))));
-
-            // This should not be stored because the object is not a literal
-            f.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj6"), pred1, vf.createURI("p:Point(10 10)"))));
-
-            f.flush();
-
-            final Set<Statement> actual = getSet(f.queryEquals(point, EMPTY_CONSTRAINTS));
-            assertEquals(2, actual.size());
-            assertTrue(actual.contains(s3));
-            assertTrue(actual.contains(s4));
-        }
-    }
-
-    @Test
-    public void testPrimeMeridianSearch() throws Exception {
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] ONE = { 1, 1, -1, 1, -1, -1, 1, -1, 1, 1 };
-            final double[] TWO = { 2, 2, -2, 2, -2, -2, 2, -2, 2, 2 };
-            final double[] THREE = { 3, 3, -3, 3, -3, -3, 3, -3, 3, 3 };
-
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
-            final LinearRing r2 = gf.createLinearRing(new PackedCoordinateSequence.Double(TWO, 2));
-            final LinearRing r3 = gf.createLinearRing(new PackedCoordinateSequence.Double(THREE, 2));
-
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-            final Polygon p2 = gf.createPolygon(r2, new LinearRing[] {});
-            final Polygon p3 = gf.createPolygon(r3, new LinearRing[] {});
-
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p2, EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p3, EMPTY_CONSTRAINTS)));
-
-            // Test a ring with a hole in it
-            final Polygon p3m2 = gf.createPolygon(r3, new LinearRing[] { r2 });
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p3m2, EMPTY_CONSTRAINTS)));
-
-            // test a ring outside the point
-            final double[] OUT = { 3, 3, 1, 3, 1, 1, 3, 1, 3, 3 };
-            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
-            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
-        }
-    }
-
-    @Test
-    public void testDcSearch() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
-
-            // test a ring outside the point
-            final double[] OUT = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
-            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
-            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
-        }
-    }
-
-    @Test
-    public void testDeleteSearch() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            f.deleteStatement(convertStatement(statement));
-
-            // test a ring that the point would be inside of if not deleted
-            final double[] in = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(in, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
-
-            // test a ring that the point would be outside of if not deleted
-            final double[] out = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
-            final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(out, 2));
-            final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
-
-            // test a ring for the whole world and make sure the point is gone
-            // Geomesa is a little sensitive around lon 180, so we only go to 179
-            final double[] world = { -180, 90, 179, 90, 179, -90, -180, -90, -180, 90 };
-            final LinearRing rWorld = gf.createLinearRing(new PackedCoordinateSequence.Double(world, 2));
-            final Polygon pWorld = gf.createPolygon(rWorld, new LinearRing[] {});
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pWorld, EMPTY_CONSTRAINTS)));
-        }
-    }
-
-    @Test
-    public void testDcSearchWithContext() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-
-            // query with correct context
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setContext(context))));
-
-            // query with wrong context
-            assertEquals(Sets.newHashSet(),
-                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
-        }
-    }
-
-    @Test
-    public void testDcSearchWithSubject() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-
-            // query with correct subject
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(subject))));
-
-            // query with wrong subject
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
-        }
-    }
-
-    @Test
-    public void testDcSearchWithSubjectAndContext() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-
-            // query with correct context subject
-            assertEquals(Sets.newHashSet(statement),
-                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(context).setSubject(subject))));
-
-            // query with wrong context
-            assertEquals(Sets.newHashSet(),
-                    getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
-
-            // query with wrong subject
-            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
-        }
-    }
-
-    @Test
-    public void testDcSearchWithPredicate() throws Exception {
-        // test a ring around dc
-        try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource subject = vf.createURI("foo:subj");
-            final URI predicate = GeoConstants.GEO_AS_WKT;
-            final Value object = vf.createLiteral("Point(-77.03524 38.889468)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Resource context = vf.createURI("foo:context");
-
-            final Statement statement = new ContextStatementImpl(subject, predicate, object, context);
-            f.storeStatement(convertStatement(statement));
-            f.flush();
-
-            final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-
-            // query with correct Predicate
-            assertEquals(Sets.newHashSet(statement),
-                    getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(predicate)))));
-
-            // query with wrong predicate
-            assertEquals(Sets.newHashSet(),
-                    getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(vf.createURI("other:pred"))))));
-        }
-    }
-
-    // @Test
-    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.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final Resource context = vf.createURI("foo:context");
-
-            final Resource subjectEast = vf.createURI("foo:subj:east");
-            final URI predicateEast = GeoConstants.GEO_AS_WKT;
-            final Value objectEast = vf.createLiteral("Point(179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Statement statementEast = new ContextStatementImpl(subjectEast, predicateEast, objectEast, context);
-            f.storeStatement(convertStatement(statementEast));
-
-            final Resource subjectWest = vf.createURI("foo:subj:west");
-            final URI predicateWest = GeoConstants.GEO_AS_WKT;
-            final Value objectWest = vf.createLiteral("Point(-179 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
-            final Statement statementWest = new ContextStatementImpl(subjectWest, predicateWest, objectWest, context);
-            f.storeStatement(convertStatement(statementWest));
-
-            f.flush();
-
-            final double[] ONE = { 178.1, 1, -178, 1, -178, -1, 178.1, -1, 178.1, 1 };
-
-            final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(ONE, 2));
-
-            final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-
-            assertEquals(Sets.newHashSet(statementEast, statementWest), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
index 1c2a339..6acc998 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
@@ -30,7 +30,7 @@ import org.apache.rya.indexing.TemporalInstantRfc3339;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -53,7 +53,7 @@ import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
 import com.vividsolutions.jts.io.WKTWriter;
 
-public class MongoIndexerDeleteIT extends MongoTestBase {
+public class MongoIndexerDeleteIT extends MongoITBase {
     @Override
     public void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, new String[] {RDFS.LABEL.stringValue()});

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
index e401589..341cb15 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
@@ -1,7 +1,3 @@
-package org.apache.rya.rdftriplestore;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -20,8 +16,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.rdftriplestore;
 
-
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.persist.RdfEvalStatsDAO;


[14/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorIT.java
new file mode 100644
index 0000000..85d27e3
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/smarturi/duplication/DuplicateDataDetectorIT.java
@@ -0,0 +1,2053 @@
+/*
+ * 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.smarturi.duplication;
+
+import static java.util.Objects.requireNonNull;
+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.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang.builder.ReflectionToStringBuilder;
+import org.apache.rya.api.domain.RyaSchema;
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.domain.RyaTypeUtils;
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
+import org.apache.rya.indexing.entity.model.Entity;
+import org.apache.rya.indexing.entity.model.Entity.Builder;
+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.EntityStorage.EntityStorageException;
+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.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.MongoITBase;
+import org.joda.time.DateTime;
+import org.junit.Test;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+/**
+ * Tests the methods of {@link DuplicateDataDetector}.
+ */
+public class DuplicateDataDetectorIT extends MongoITBase {
+    private static final String RYA_INSTANCE_NAME = "testInstance";
+
+    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");
+
+    private static final Date NOW = new Date();
+
+    /**
+     * 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(NOW.getTime()).minusYears(40))))
+            .setProperty(PERSON_TYPE_URI, new Property(HAS_EXPIRATION_DATE, dateRyaType(NOW)))
+            .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 createErrorMessage(final Object originalValue, final Object testValue, final boolean expected, final boolean actual, final Tolerance tolerance) {
+        final String message = "The test value \"" + testValue + "\" was " + (expected ? "" : "NOT ") + "supposed to be almost equals to \"" + originalValue + "\" when the tolerance was \"" + tolerance.toString() + "\" but " + (actual ? "was" : "wasn't") + ".";
+        return message;
+    }
+
+    @Test
+    public void testCompareEntities() throws SmartUriException, ConfigurationException {
+        final Entity entity1 = createBobEntity();
+        final Entity entity2 = new Builder(entity1).build();
+
+        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
+        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
+        assertTrue(areDuplicates);
+    }
+
+    @Test
+    public void testCompareSmartUris() throws SmartUriException, ConfigurationException {
+        final Entity entity1 = createBobEntity();
+        final Entity entity2 = new Builder(entity1).build();
+
+        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
+        final boolean areDuplicates = duplicateDataDetector.compareSmartUris(entity1.getSmartUri(), entity2.getSmartUri());
+        assertTrue(areDuplicates);
+    }
+
+    @Test
+    public void testEntitySubjectsDifferent() throws SmartUriException, ConfigurationException {
+        final Entity entity1 = createBobEntity();
+        final Builder builder = new Builder(entity1);
+        builder.setSubject(createRyaUri("Susan"));
+        final Entity entity2 = builder.build();
+
+        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
+        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
+        assertTrue(areDuplicates);
+    }
+
+    @Test
+    public void testEntityMissingType() throws SmartUriException, ConfigurationException {
+        final Entity entity1 = createBobEntity();
+        final Builder builder = new Builder(entity1);
+        builder.setExplicitType(new RyaURI("urn:example/manager"));
+        final Entity entity2 = builder.build();
+
+        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
+        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
+        assertFalse(areDuplicates);
+    }
+
+    @Test
+    public void testEntityMissingProperty() throws SmartUriException, ConfigurationException {
+        final Entity entity1 = createBobEntity();
+        final Builder builder = new Builder(entity1);
+        builder.unsetProperty(PERSON_TYPE_URI, HAS_SSN);
+        final Entity entity2 = builder.build();
+
+        final DuplicateDataDetector duplicateDataDetector = new DuplicateDataDetector();
+        final boolean areDuplicates = duplicateDataDetector.compareEntities(entity1, entity2);
+        assertFalse(areDuplicates);
+    }
+
+    @Test
+    public void testReadConfigFile() throws SmartUriException, ConfigurationException {
+        final DuplicateDataConfig duplicateDataConfig = new DuplicateDataConfig();
+
+        assertNotNull(duplicateDataConfig.getBooleanTolerance());
+        assertNotNull(duplicateDataConfig.getByteTolerance());
+        assertNotNull(duplicateDataConfig.getDateTolerance());
+        assertNotNull(duplicateDataConfig.getDoubleTolerance());
+        assertNotNull(duplicateDataConfig.getFloatTolerance());
+        assertNotNull(duplicateDataConfig.getIntegerTolerance());
+        assertNotNull(duplicateDataConfig.getLongTolerance());
+        assertNotNull(duplicateDataConfig.getShortTolerance());
+        assertNotNull(duplicateDataConfig.getStringTolerance());
+        assertNotNull(duplicateDataConfig.getUriTolerance());
+
+        assertNotNull(duplicateDataConfig.getEquivalentTermsMap());
+        assertNotNull(duplicateDataConfig.isDetectionEnabled());
+    }
+
+    @Test
+    public void testBooleanProperty() throws SmartUriException {
+        System.out.println("Boolean Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(false, tolerance, false));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(false, tolerance, true));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(false, tolerance, true));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(false, tolerance, false));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(false, tolerance, true));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(false, tolerance, true));
+        builder.add(new TestInput(true, tolerance, true)); // Equals value
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_GLASSES);
+    }
+
+    @Test
+    public void testByteProperty() throws SmartUriException {
+        System.out.println("Byte Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((byte) 0xff, tolerance, false));
+        builder.add(new TestInput((byte) 0x00, tolerance, false));
+        builder.add(new TestInput((byte) 0x01, tolerance, false));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, false));
+        builder.add(new TestInput((byte) 0x04, tolerance, false));
+        builder.add(new TestInput((byte) 0x05, tolerance, false));
+        builder.add(new TestInput((byte) 0x10, tolerance, false));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((byte) 0xff, tolerance, false));
+        builder.add(new TestInput((byte) 0x00, tolerance, false));
+        builder.add(new TestInput((byte) 0x01, tolerance, true));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, true));
+        builder.add(new TestInput((byte) 0x04, tolerance, false));
+        builder.add(new TestInput((byte) 0x05, tolerance, false));
+        builder.add(new TestInput((byte) 0x10, tolerance, false));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((byte) 0xff, tolerance, false));
+        builder.add(new TestInput((byte) 0x00, tolerance, true));
+        builder.add(new TestInput((byte) 0x01, tolerance, true));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, true));
+        builder.add(new TestInput((byte) 0x04, tolerance, true));
+        builder.add(new TestInput((byte) 0x05, tolerance, false));
+        builder.add(new TestInput((byte) 0x10, tolerance, false));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((byte) 0xff, tolerance, false));
+        builder.add(new TestInput((byte) 0x00, tolerance, false));
+        builder.add(new TestInput((byte) 0x01, tolerance, false));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, false));
+        builder.add(new TestInput((byte) 0x04, tolerance, false));
+        builder.add(new TestInput((byte) 0x05, tolerance, false));
+        builder.add(new TestInput((byte) 0x10, tolerance, false));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
+        // Tolerance 50.0%
+        tolerance = new Tolerance(0.50, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((byte) 0xff, tolerance, false));
+        builder.add(new TestInput((byte) 0x00, tolerance, false));
+        builder.add(new TestInput((byte) 0x01, tolerance, true));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, true));
+        builder.add(new TestInput((byte) 0x04, tolerance, false));
+        builder.add(new TestInput((byte) 0x05, tolerance, false));
+        builder.add(new TestInput((byte) 0x10, tolerance, false));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Byte.MIN_VALUE, tolerance, true));
+        builder.add(new TestInput((byte) 0xff, tolerance, true));
+        builder.add(new TestInput((byte) 0x00, tolerance, true));
+        builder.add(new TestInput((byte) 0x01, tolerance, true));
+        builder.add(new TestInput((byte) 0x02, tolerance, true)); // Equals value
+        builder.add(new TestInput((byte) 0x03, tolerance, true));
+        builder.add(new TestInput((byte) 0x04, tolerance, true));
+        builder.add(new TestInput((byte) 0x05, tolerance, true));
+        builder.add(new TestInput((byte) 0x10, tolerance, true));
+        builder.add(new TestInput(Byte.MAX_VALUE, tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_NUMBER_OF_CHILDREN);
+    }
+
+    @Test
+    public void testDateProperty() throws SmartUriException {
+        System.out.println("Date Property Test");
+        final long ONE_YEAR_IN_MILLIS = 1000L * 60L * 60L * 24L * 365L;
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new Date(0L), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, false));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new Date(0L), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new Date(0L), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new Date(0L), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, false));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new Date(0L), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.985)), tolerance, false));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.989)), tolerance, false));
+        // It's tricky near the exact threshold since it might create a fraction
+        // of a time which is rounded. Check if it's fraction and test it with
+        // the floor and ceiling values if it is. Otherwise, use the exact value
+        // if it is not a fraction.
+        final double lowerThresholdFloor = Math.floor(NOW.getTime() * 0.99);
+        final double lowerThresholdCeiling = Math.ceil(NOW.getTime() * 0.99);
+        // If the floor equals the ceiling then it's not a fraction.
+        if (lowerThresholdFloor != lowerThresholdCeiling) {
+           builder.add(new TestInput(new Date((long) lowerThresholdFloor), tolerance, false));
+        }
+        builder.add(new TestInput(new Date((long) lowerThresholdCeiling), tolerance, true));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.991)), tolerance, true));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 0.995)), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, true));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.005)), tolerance, true));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.009)), tolerance, true));
+        // It's tricky near the exact threshold since it might create a fraction
+        // of a time which is rounded. Check if it's fraction and test it with
+        // the floor and ceiling values if it is. Otherwise, use the exact value
+        // if it is not a fraction.
+        final double upperThresholdFloor = Math.floor(NOW.getTime() * 1.01);
+        final double upperThresholdCeiling = Math.ceil(NOW.getTime() * 1.01);
+        builder.add(new TestInput(new Date((long) upperThresholdFloor), tolerance, true));
+        // If the floor equals the ceiling then it's not a fraction.
+        if (upperThresholdFloor != upperThresholdCeiling) {
+           builder.add(new TestInput(new Date((long) upperThresholdCeiling), tolerance, false));
+        }
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.011)), tolerance, false));
+        builder.add(new TestInput(new Date((long) (NOW.getTime() * 1.015)), tolerance, false));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, false));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new Date(0L), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - ONE_YEAR_IN_MILLIS), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 10000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1001), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 999), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 3), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() - 1), tolerance, true));
+        builder.add(new TestInput(NOW, tolerance, true)); // Equals value
+        builder.add(new TestInput(new Date(NOW.getTime() + 1), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 3), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 999), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 1001), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 2000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + 10000), tolerance, true));
+        builder.add(new TestInput(new Date(NOW.getTime() + ONE_YEAR_IN_MILLIS), tolerance, true));
+        builder.add(new TestInput(new Date(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_EXPIRATION_DATE);
+    }
+
+    @Test
+    public void testDateTimeProperty() throws SmartUriException {
+        System.out.println("DateTime Property Test");
+        final DateTime dob = new DateTime(NOW).minusYears(40);
+        final long ONE_YEAR_IN_MILLIS = 1000L * 60L * 60L * 24L * 365L;
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, false));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
+        builder.add(new TestInput(dob.minus(10000), tolerance, false));
+        builder.add(new TestInput(dob.minus(2000), tolerance, false));
+        builder.add(new TestInput(dob.minus(1001), tolerance, false));
+        builder.add(new TestInput(dob.minus(1000), tolerance, false));
+        builder.add(new TestInput(dob.minus(999), tolerance, false));
+        builder.add(new TestInput(dob.minus(3), tolerance, false));
+        builder.add(new TestInput(dob.minus(2), tolerance, false));
+        builder.add(new TestInput(dob.minus(1), tolerance, false));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, false));
+        builder.add(new TestInput(dob.plus(2), tolerance, false));
+        builder.add(new TestInput(dob.plus(3), tolerance, false));
+        builder.add(new TestInput(dob.plus(999), tolerance, false));
+        builder.add(new TestInput(dob.plus(1000), tolerance, false));
+        builder.add(new TestInput(dob.plus(1001), tolerance, false));
+        builder.add(new TestInput(dob.plus(2000), tolerance, false));
+        builder.add(new TestInput(dob.plus(10000), tolerance, false));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, false));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
+        builder.add(new TestInput(dob.minus(10000), tolerance, false));
+        builder.add(new TestInput(dob.minus(2000), tolerance, false));
+        builder.add(new TestInput(dob.minus(1001), tolerance, false));
+        builder.add(new TestInput(dob.minus(1000), tolerance, false));
+        builder.add(new TestInput(dob.minus(999), tolerance, false));
+        builder.add(new TestInput(dob.minus(3), tolerance, false));
+        builder.add(new TestInput(dob.minus(2), tolerance, false));
+        builder.add(new TestInput(dob.minus(1), tolerance, true));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, true));
+        builder.add(new TestInput(dob.plus(2), tolerance, false));
+        builder.add(new TestInput(dob.plus(3), tolerance, false));
+        builder.add(new TestInput(dob.plus(999), tolerance, false));
+        builder.add(new TestInput(dob.plus(1000), tolerance, false));
+        builder.add(new TestInput(dob.plus(1001), tolerance, false));
+        builder.add(new TestInput(dob.plus(2000), tolerance, false));
+        builder.add(new TestInput(dob.plus(10000), tolerance, false));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, false));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
+        builder.add(new TestInput(dob.minus(10000), tolerance, false));
+        builder.add(new TestInput(dob.minus(2000), tolerance, false));
+        builder.add(new TestInput(dob.minus(1001), tolerance, false));
+        builder.add(new TestInput(dob.minus(1000), tolerance, false));
+        builder.add(new TestInput(dob.minus(999), tolerance, false));
+        builder.add(new TestInput(dob.minus(3), tolerance, false));
+        builder.add(new TestInput(dob.minus(2), tolerance, true));
+        builder.add(new TestInput(dob.minus(1), tolerance, true));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, true));
+        builder.add(new TestInput(dob.plus(2), tolerance, true));
+        builder.add(new TestInput(dob.plus(3), tolerance, false));
+        builder.add(new TestInput(dob.plus(999), tolerance, false));
+        builder.add(new TestInput(dob.plus(1000), tolerance, false));
+        builder.add(new TestInput(dob.plus(1001), tolerance, false));
+        builder.add(new TestInput(dob.plus(2000), tolerance, false));
+        builder.add(new TestInput(dob.plus(10000), tolerance, false));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, false));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
+        builder.add(new TestInput(dob.minus(10000), tolerance, false));
+        builder.add(new TestInput(dob.minus(2000), tolerance, false));
+        builder.add(new TestInput(dob.minus(1001), tolerance, false));
+        builder.add(new TestInput(dob.minus(1000), tolerance, false));
+        builder.add(new TestInput(dob.minus(999), tolerance, false));
+        builder.add(new TestInput(dob.minus(3), tolerance, false));
+        builder.add(new TestInput(dob.minus(2), tolerance, false));
+        builder.add(new TestInput(dob.minus(1), tolerance, false));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, false));
+        builder.add(new TestInput(dob.plus(2), tolerance, false));
+        builder.add(new TestInput(dob.plus(3), tolerance, false));
+        builder.add(new TestInput(dob.plus(999), tolerance, false));
+        builder.add(new TestInput(dob.plus(1000), tolerance, false));
+        builder.add(new TestInput(dob.plus(1001), tolerance, false));
+        builder.add(new TestInput(dob.plus(2000), tolerance, false));
+        builder.add(new TestInput(dob.plus(10000), tolerance, false));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, false));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.985)), tolerance, false));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.989)), tolerance, false));
+        // It's tricky near the exact threshold since it might create a fraction
+        // of a time which is rounded. Check if it's fraction and test it with
+        // the floor and ceiling values if it is. Otherwise, use the exact value
+        // if it is not a fraction.
+        final double lowerThresholdFloor = Math.floor(dob.getMillis() * 0.99);
+        final double lowerThresholdCeiling = Math.ceil(dob.getMillis() * 0.99);
+        // If the floor equals the ceiling then it's not a fraction.
+        if (lowerThresholdFloor != lowerThresholdCeiling) {
+           builder.add(new TestInput(new DateTime((long) lowerThresholdFloor), tolerance, false));
+        }
+        builder.add(new TestInput(new DateTime((long) lowerThresholdCeiling), tolerance, true));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.991)), tolerance, true));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 0.995)), tolerance, true));
+        builder.add(new TestInput(dob.minus(10000), tolerance, true));
+        builder.add(new TestInput(dob.minus(2000), tolerance, true));
+        builder.add(new TestInput(dob.minus(1001), tolerance, true));
+        builder.add(new TestInput(dob.minus(1000), tolerance, true));
+        builder.add(new TestInput(dob.minus(999), tolerance, true));
+        builder.add(new TestInput(dob.minus(3), tolerance, true));
+        builder.add(new TestInput(dob.minus(2), tolerance, true));
+        builder.add(new TestInput(dob.minus(1), tolerance, true));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, true));
+        builder.add(new TestInput(dob.plus(2), tolerance, true));
+        builder.add(new TestInput(dob.plus(3), tolerance, true));
+        builder.add(new TestInput(dob.plus(999), tolerance, true));
+        builder.add(new TestInput(dob.plus(1000), tolerance, true));
+        builder.add(new TestInput(dob.plus(1001), tolerance, true));
+        builder.add(new TestInput(dob.plus(2000), tolerance, true));
+        builder.add(new TestInput(dob.plus(10000), tolerance, true));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.005)), tolerance, true));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.009)), tolerance, true));
+        // It's tricky near the exact threshold since it might create a fraction
+        // of a time which is rounded. Check if it's fraction and test it with
+        // the floor and ceiling values if it is. Otherwise, use the exact value
+        // if it is not a fraction.
+        final double upperThresholdFloor = Math.floor(dob.getMillis() * 1.01);
+        final double upperThresholdCeiling = Math.ceil(dob.getMillis() * 1.01);
+        builder.add(new TestInput(new DateTime((long) upperThresholdFloor), tolerance, true));
+        // If the floor equals the ceiling then it's not a fraction.
+        if (upperThresholdFloor != upperThresholdCeiling) {
+           builder.add(new TestInput(new DateTime((long) upperThresholdCeiling), tolerance, false));
+        }
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.011)), tolerance, false));
+        builder.add(new TestInput(new DateTime((long) (dob.getMillis() * 1.015)), tolerance, false));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, false));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(new DateTime(0L), tolerance, true));
+        builder.add(new TestInput(dob.minusYears(1), tolerance, true));
+        builder.add(new TestInput(dob.minus(10000), tolerance, true));
+        builder.add(new TestInput(dob.minus(2000), tolerance, true));
+        builder.add(new TestInput(dob.minus(1001), tolerance, true));
+        builder.add(new TestInput(dob.minus(1000), tolerance, true));
+        builder.add(new TestInput(dob.minus(999), tolerance, true));
+        builder.add(new TestInput(dob.minus(3), tolerance, true));
+        builder.add(new TestInput(dob.minus(2), tolerance, true));
+        builder.add(new TestInput(dob.minus(1), tolerance, true));
+        builder.add(new TestInput(dob, tolerance, true)); // Equals value
+        builder.add(new TestInput(dob.plus(1), tolerance, true));
+        builder.add(new TestInput(dob.plus(2), tolerance, true));
+        builder.add(new TestInput(dob.plus(3), tolerance, true));
+        builder.add(new TestInput(dob.plus(999), tolerance, true));
+        builder.add(new TestInput(dob.plus(1000), tolerance, true));
+        builder.add(new TestInput(dob.plus(1001), tolerance, true));
+        builder.add(new TestInput(dob.plus(2000), tolerance, true));
+        builder.add(new TestInput(dob.plus(10000), tolerance, true));
+        builder.add(new TestInput(dob.plusYears(1), tolerance, true));
+        builder.add(new TestInput(new DateTime(Long.MAX_VALUE - ONE_YEAR_IN_MILLIS), tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_DATE_OF_BIRTH);
+    }
+
+    @Test
+    public void testDoubleProperty() throws SmartUriException {
+        System.out.println("Double Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0, tolerance, false));
+        builder.add(new TestInput(0.0, tolerance, false));
+        builder.add(new TestInput(0.01, tolerance, false));
+        builder.add(new TestInput(0.02, tolerance, false));
+        builder.add(new TestInput(0.1, tolerance, false));
+        builder.add(new TestInput(0.2, tolerance, false));
+        builder.add(new TestInput(1.0, tolerance, false));
+        builder.add(new TestInput(71, tolerance, false));
+        builder.add(new TestInput(72, tolerance, false));
+        builder.add(new TestInput(72.4, tolerance, false));
+        builder.add(new TestInput(72.47, tolerance, false));
+        builder.add(new TestInput(72.48, tolerance, false));
+        builder.add(new TestInput(72.49, tolerance, false));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, false));
+        builder.add(new TestInput(72.52, tolerance, false));
+        builder.add(new TestInput(72.53, tolerance, false));
+        builder.add(new TestInput(72.6, tolerance, false));
+        builder.add(new TestInput(73, tolerance, false));
+        builder.add(new TestInput(74, tolerance, false));
+        builder.add(new TestInput(100, tolerance, false));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
+        // Tolerance 0.01
+        tolerance = new Tolerance(0.01, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0, tolerance, false));
+        builder.add(new TestInput(0.0, tolerance, false));
+        builder.add(new TestInput(0.01, tolerance, false));
+        builder.add(new TestInput(0.02, tolerance, false));
+        builder.add(new TestInput(0.1, tolerance, false));
+        builder.add(new TestInput(0.2, tolerance, false));
+        builder.add(new TestInput(1.0, tolerance, false));
+        builder.add(new TestInput(71, tolerance, false));
+        builder.add(new TestInput(72, tolerance, false));
+        builder.add(new TestInput(72.4, tolerance, false));
+        builder.add(new TestInput(72.47, tolerance, false));
+        builder.add(new TestInput(72.48, tolerance, false));
+        builder.add(new TestInput(72.49, tolerance, true));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, true));
+        builder.add(new TestInput(72.52, tolerance, false));
+        builder.add(new TestInput(72.53, tolerance, false));
+        builder.add(new TestInput(72.6, tolerance, false));
+        builder.add(new TestInput(73, tolerance, false));
+        builder.add(new TestInput(74, tolerance, false));
+        builder.add(new TestInput(100, tolerance, false));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
+        // Tolerance 0.02
+        tolerance = new Tolerance(0.02, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0, tolerance, false));
+        builder.add(new TestInput(0.0, tolerance, false));
+        builder.add(new TestInput(0.01, tolerance, false));
+        builder.add(new TestInput(0.02, tolerance, false));
+        builder.add(new TestInput(0.1, tolerance, false));
+        builder.add(new TestInput(0.2, tolerance, false));
+        builder.add(new TestInput(1.0, tolerance, false));
+        builder.add(new TestInput(71, tolerance, false));
+        builder.add(new TestInput(72, tolerance, false));
+        builder.add(new TestInput(72.4, tolerance, false));
+        builder.add(new TestInput(72.47, tolerance, false));
+        builder.add(new TestInput(72.48, tolerance, true));
+        builder.add(new TestInput(72.49, tolerance, true));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, true));
+        builder.add(new TestInput(72.52, tolerance, true));
+        builder.add(new TestInput(72.53, tolerance, false));
+        builder.add(new TestInput(72.6, tolerance, false));
+        builder.add(new TestInput(73, tolerance, false));
+        builder.add(new TestInput(74, tolerance, false));
+        builder.add(new TestInput(100, tolerance, false));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0%
+        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0, tolerance, false));
+        builder.add(new TestInput(0.0, tolerance, false));
+        builder.add(new TestInput(0.01, tolerance, false));
+        builder.add(new TestInput(0.02, tolerance, false));
+        builder.add(new TestInput(0.1, tolerance, false));
+        builder.add(new TestInput(0.2, tolerance, false));
+        builder.add(new TestInput(1.0, tolerance, false));
+        builder.add(new TestInput(71, tolerance, false));
+        builder.add(new TestInput(71.774, tolerance, false));
+        builder.add(new TestInput(71.775, tolerance, false));
+        builder.add(new TestInput(71.776, tolerance, false));
+        builder.add(new TestInput(72, tolerance, false));
+        builder.add(new TestInput(72.4, tolerance, false));
+        builder.add(new TestInput(72.47, tolerance, false));
+        builder.add(new TestInput(72.48, tolerance, false));
+        builder.add(new TestInput(72.49, tolerance, false));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, false));
+        builder.add(new TestInput(72.52, tolerance, false));
+        builder.add(new TestInput(72.53, tolerance, false));
+        builder.add(new TestInput(72.6, tolerance, false));
+        builder.add(new TestInput(73, tolerance, false));
+        builder.add(new TestInput(73.224, tolerance, false));
+        builder.add(new TestInput(73.225, tolerance, false));
+        builder.add(new TestInput(73.226, tolerance, false));
+        builder.add(new TestInput(73, tolerance, false));
+        builder.add(new TestInput(74, tolerance, false));
+        builder.add(new TestInput(100, tolerance, false));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
+        // Tolerance 1%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0, tolerance, false));
+        builder.add(new TestInput(0.0, tolerance, false));
+        builder.add(new TestInput(0.01, tolerance, false));
+        builder.add(new TestInput(0.02, tolerance, false));
+        builder.add(new TestInput(0.1, tolerance, false));
+        builder.add(new TestInput(0.2, tolerance, false));
+        builder.add(new TestInput(1.0, tolerance, false));
+        builder.add(new TestInput(71, tolerance, false));
+        builder.add(new TestInput(71.774, tolerance, false));
+        builder.add(new TestInput(71.775, tolerance, true));
+        builder.add(new TestInput(71.776, tolerance, true));
+        builder.add(new TestInput(72, tolerance, true));
+        builder.add(new TestInput(72.4, tolerance, true));
+        builder.add(new TestInput(72.47, tolerance, true));
+        builder.add(new TestInput(72.48, tolerance, true));
+        builder.add(new TestInput(72.49, tolerance, true));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, true));
+        builder.add(new TestInput(72.52, tolerance, true));
+        builder.add(new TestInput(72.53, tolerance, true));
+        builder.add(new TestInput(72.6, tolerance, true));
+        builder.add(new TestInput(73, tolerance, true));
+        builder.add(new TestInput(73.224, tolerance, true));
+        builder.add(new TestInput(73.225, tolerance, true));
+        builder.add(new TestInput(73.226, tolerance, false));
+        builder.add(new TestInput(74, tolerance, false));
+        builder.add(new TestInput(100, tolerance, false));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, false));
+        // Tolerance 100%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Double.MIN_VALUE, tolerance, true));
+        builder.add(new TestInput(-1.0, tolerance, true));
+        builder.add(new TestInput(0.0, tolerance, true));
+        builder.add(new TestInput(0.01, tolerance, true));
+        builder.add(new TestInput(0.02, tolerance, true));
+        builder.add(new TestInput(0.1, tolerance, true));
+        builder.add(new TestInput(0.2, tolerance, true));
+        builder.add(new TestInput(1.0, tolerance, true));
+        builder.add(new TestInput(71, tolerance, true));
+        builder.add(new TestInput(71.774, tolerance, true));
+        builder.add(new TestInput(71.775, tolerance, true));
+        builder.add(new TestInput(71.776, tolerance, true));
+        builder.add(new TestInput(72, tolerance, true));
+        builder.add(new TestInput(72.4, tolerance, true));
+        builder.add(new TestInput(72.47, tolerance, true));
+        builder.add(new TestInput(72.48, tolerance, true));
+        builder.add(new TestInput(72.49, tolerance, true));
+        builder.add(new TestInput(72.5, tolerance, true)); // Equals value
+        builder.add(new TestInput(72.51, tolerance, true));
+        builder.add(new TestInput(72.52, tolerance, true));
+        builder.add(new TestInput(72.53, tolerance, true));
+        builder.add(new TestInput(72.6, tolerance, true));
+        builder.add(new TestInput(73, tolerance, true));
+        builder.add(new TestInput(73.224, tolerance, true));
+        builder.add(new TestInput(73.225, tolerance, true));
+        builder.add(new TestInput(73.226, tolerance, true));
+        builder.add(new TestInput(74, tolerance, true));
+        builder.add(new TestInput(100, tolerance, true));
+        builder.add(new TestInput(Double.MAX_VALUE, tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_HEIGHT);
+    }
+
+    @Test
+    public void testFloatProperty() throws SmartUriException {
+        System.out.println("Float Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0f, tolerance, false));
+        builder.add(new TestInput(0.0f, tolerance, false));
+        builder.add(new TestInput(0.01f, tolerance, false));
+        builder.add(new TestInput(0.02f, tolerance, false));
+        builder.add(new TestInput(0.1f, tolerance, false));
+        builder.add(new TestInput(0.2f, tolerance, false));
+        builder.add(new TestInput(1.0f, tolerance, false));
+        builder.add(new TestInput(250f, tolerance, false));
+        builder.add(new TestInput(250.7f, tolerance, false));
+        builder.add(new TestInput(250.72f, tolerance, false));
+        builder.add(new TestInput(250.73f, tolerance, false));
+        builder.add(new TestInput(250.74f, tolerance, false));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, false));
+        builder.add(new TestInput(250.77f, tolerance, false));
+        builder.add(new TestInput(250.78f, tolerance, false));
+        builder.add(new TestInput(250.8f, tolerance, false));
+        builder.add(new TestInput(251f, tolerance, false));
+        builder.add(new TestInput(300.0f, tolerance, false));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
+        // Tolerance 0.01
+        tolerance = new Tolerance(0.01, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0f, tolerance, false));
+        builder.add(new TestInput(0.0f, tolerance, false));
+        builder.add(new TestInput(0.01f, tolerance, false));
+        builder.add(new TestInput(0.02f, tolerance, false));
+        builder.add(new TestInput(0.1f, tolerance, false));
+        builder.add(new TestInput(0.2f, tolerance, false));
+        builder.add(new TestInput(1.0f, tolerance, false));
+        builder.add(new TestInput(250f, tolerance, false));
+        builder.add(new TestInput(250.7f, tolerance, false));
+        builder.add(new TestInput(250.72f, tolerance, false));
+        builder.add(new TestInput(250.73f, tolerance, false));
+        builder.add(new TestInput(250.74f, tolerance, true));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, true));
+        builder.add(new TestInput(250.77f, tolerance, false));
+        builder.add(new TestInput(250.78f, tolerance, false));
+        builder.add(new TestInput(250.8f, tolerance, false));
+        builder.add(new TestInput(251f, tolerance, false));
+        builder.add(new TestInput(300.0f, tolerance, false));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
+        // Tolerance 0.02
+        tolerance = new Tolerance(0.02, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0f, tolerance, false));
+        builder.add(new TestInput(0.0f, tolerance, false));
+        builder.add(new TestInput(0.01f, tolerance, false));
+        builder.add(new TestInput(0.02f, tolerance, false));
+        builder.add(new TestInput(0.1f, tolerance, false));
+        builder.add(new TestInput(0.2f, tolerance, false));
+        builder.add(new TestInput(1.0f, tolerance, false));
+        builder.add(new TestInput(250f, tolerance, false));
+        builder.add(new TestInput(250.7f, tolerance, false));
+        builder.add(new TestInput(250.72f, tolerance, false));
+        builder.add(new TestInput(250.73f, tolerance, true));
+        builder.add(new TestInput(250.74f, tolerance, true));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, true));
+        builder.add(new TestInput(250.77f, tolerance, true));
+        builder.add(new TestInput(250.78f, tolerance, false));
+        builder.add(new TestInput(250.8f, tolerance, false));
+        builder.add(new TestInput(251f, tolerance, false));
+        builder.add(new TestInput(300.0f, tolerance, false));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0f, tolerance, false));
+        builder.add(new TestInput(0.0f, tolerance, false));
+        builder.add(new TestInput(0.01f, tolerance, false));
+        builder.add(new TestInput(0.02f, tolerance, false));
+        builder.add(new TestInput(0.1f, tolerance, false));
+        builder.add(new TestInput(0.2f, tolerance, false));
+        builder.add(new TestInput(1.0f, tolerance, false));
+        builder.add(new TestInput(248.2424f, tolerance, false));
+        builder.add(new TestInput(248.2425f, tolerance, false));
+        builder.add(new TestInput(248.2426f, tolerance, false));
+        builder.add(new TestInput(250f, tolerance, false));
+        builder.add(new TestInput(250.7f, tolerance, false));
+        builder.add(new TestInput(250.72f, tolerance, false));
+        builder.add(new TestInput(250.73f, tolerance, false));
+        builder.add(new TestInput(250.74f, tolerance, false));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, false));
+        builder.add(new TestInput(250.77f, tolerance, false));
+        builder.add(new TestInput(250.78f, tolerance, false));
+        builder.add(new TestInput(250.8f, tolerance, false));
+        builder.add(new TestInput(251f, tolerance, false));
+        builder.add(new TestInput(253.2574f, tolerance, false));
+        builder.add(new TestInput(253.2575f, tolerance, false));
+        builder.add(new TestInput(253.2576f, tolerance, false));
+        builder.add(new TestInput(300.0f, tolerance, false));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1.0f, tolerance, false));
+        builder.add(new TestInput(0.0f, tolerance, false));
+        builder.add(new TestInput(0.01f, tolerance, false));
+        builder.add(new TestInput(0.02f, tolerance, false));
+        builder.add(new TestInput(0.1f, tolerance, false));
+        builder.add(new TestInput(0.2f, tolerance, false));
+        builder.add(new TestInput(1.0f, tolerance, false));
+        builder.add(new TestInput(248.2424f, tolerance, false));
+        builder.add(new TestInput(248.2425f, tolerance, true));
+        builder.add(new TestInput(248.2426f, tolerance, true));
+        builder.add(new TestInput(250f, tolerance, true));
+        builder.add(new TestInput(250.7f, tolerance, true));
+        builder.add(new TestInput(250.72f, tolerance, true));
+        builder.add(new TestInput(250.73f, tolerance, true));
+        builder.add(new TestInput(250.74f, tolerance, true));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, true));
+        builder.add(new TestInput(250.77f, tolerance, true));
+        builder.add(new TestInput(250.78f, tolerance, true));
+        builder.add(new TestInput(250.8f, tolerance, true));
+        builder.add(new TestInput(251f, tolerance, true));
+        builder.add(new TestInput(253.2574f, tolerance, true));
+        builder.add(new TestInput(253.2575f, tolerance, true));
+        builder.add(new TestInput(253.2576f, tolerance, false));
+        builder.add(new TestInput(300.0f, tolerance, false));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Float.MIN_VALUE, tolerance, true));
+        builder.add(new TestInput(-1.0f, tolerance, true));
+        builder.add(new TestInput(0.0f, tolerance, true));
+        builder.add(new TestInput(0.01f, tolerance, true));
+        builder.add(new TestInput(0.02f, tolerance, true));
+        builder.add(new TestInput(0.1f, tolerance, true));
+        builder.add(new TestInput(0.2f, tolerance, true));
+        builder.add(new TestInput(1.0f, tolerance, true));
+        builder.add(new TestInput(248.2424f, tolerance, true));
+        builder.add(new TestInput(248.2425f, tolerance, true));
+        builder.add(new TestInput(248.2426f, tolerance, true));
+        builder.add(new TestInput(250f, tolerance, true));
+        builder.add(new TestInput(250.7f, tolerance, true));
+        builder.add(new TestInput(250.72f, tolerance, true));
+        builder.add(new TestInput(250.73f, tolerance, true));
+        builder.add(new TestInput(250.74f, tolerance, true));
+        builder.add(new TestInput(250.75f, tolerance, true)); // Equals value
+        builder.add(new TestInput(250.76f, tolerance, true));
+        builder.add(new TestInput(250.77f, tolerance, true));
+        builder.add(new TestInput(250.78f, tolerance, true));
+        builder.add(new TestInput(250.8f, tolerance, true));
+        builder.add(new TestInput(251f, tolerance, true));
+        builder.add(new TestInput(253.2574f, tolerance, true));
+        builder.add(new TestInput(253.2575f, tolerance, true));
+        builder.add(new TestInput(253.2576f, tolerance, true));
+        builder.add(new TestInput(300.0f, tolerance, true));
+        builder.add(new TestInput(Float.MAX_VALUE, tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_WEIGHT);
+    }
+
+    @Test
+    public void testIntegerProperty() throws SmartUriException {
+        System.out.println("Integer Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1, tolerance, false));
+        builder.add(new TestInput(0, tolerance, false));
+        builder.add(new TestInput(1, tolerance, false));
+        builder.add(new TestInput(49997, tolerance, false));
+        builder.add(new TestInput(49998, tolerance, false));
+        builder.add(new TestInput(49999, tolerance, false));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, false));
+        builder.add(new TestInput(50002, tolerance, false));
+        builder.add(new TestInput(50003, tolerance, false));
+        builder.add(new TestInput(60000, tolerance, false));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1, tolerance, false));
+        builder.add(new TestInput(0, tolerance, false));
+        builder.add(new TestInput(1, tolerance, false));
+        builder.add(new TestInput(49997, tolerance, false));
+        builder.add(new TestInput(49998, tolerance, false));
+        builder.add(new TestInput(49999, tolerance, true));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, true));
+        builder.add(new TestInput(50002, tolerance, false));
+        builder.add(new TestInput(50003, tolerance, false));
+        builder.add(new TestInput(60000, tolerance, false));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1, tolerance, false));
+        builder.add(new TestInput(0, tolerance, false));
+        builder.add(new TestInput(1, tolerance, false));
+        builder.add(new TestInput(49997, tolerance, false));
+        builder.add(new TestInput(49998, tolerance, true));
+        builder.add(new TestInput(49999, tolerance, true));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, true));
+        builder.add(new TestInput(50002, tolerance, true));
+        builder.add(new TestInput(50003, tolerance, false));
+        builder.add(new TestInput(60000, tolerance, false));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1, tolerance, false));
+        builder.add(new TestInput(0, tolerance, false));
+        builder.add(new TestInput(1, tolerance, false));
+        builder.add(new TestInput(48999, tolerance, false));
+        builder.add(new TestInput(49000, tolerance, false));
+        builder.add(new TestInput(49001, tolerance, false));
+        builder.add(new TestInput(49499, tolerance, false));
+        builder.add(new TestInput(49500, tolerance, false));
+        builder.add(new TestInput(49501, tolerance, false));
+        builder.add(new TestInput(49997, tolerance, false));
+        builder.add(new TestInput(49998, tolerance, false));
+        builder.add(new TestInput(49999, tolerance, false));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, false));
+        builder.add(new TestInput(50002, tolerance, false));
+        builder.add(new TestInput(50003, tolerance, false));
+        builder.add(new TestInput(50499, tolerance, false));
+        builder.add(new TestInput(50500, tolerance, false));
+        builder.add(new TestInput(50501, tolerance, false));
+        builder.add(new TestInput(50999, tolerance, false));
+        builder.add(new TestInput(51000, tolerance, false));
+        builder.add(new TestInput(51001, tolerance, false));
+        builder.add(new TestInput(60000, tolerance, false));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1, tolerance, false));
+        builder.add(new TestInput(0, tolerance, false));
+        builder.add(new TestInput(1, tolerance, false));
+        builder.add(new TestInput(48999, tolerance, false));
+        builder.add(new TestInput(49000, tolerance, false));
+        builder.add(new TestInput(49001, tolerance, false));
+        builder.add(new TestInput(49499, tolerance, false));
+        builder.add(new TestInput(49500, tolerance, true));
+        builder.add(new TestInput(49501, tolerance, true));
+        builder.add(new TestInput(49997, tolerance, true));
+        builder.add(new TestInput(49998, tolerance, true));
+        builder.add(new TestInput(49999, tolerance, true));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, true));
+        builder.add(new TestInput(50002, tolerance, true));
+        builder.add(new TestInput(50003, tolerance, true));
+        builder.add(new TestInput(50499, tolerance, true));
+        builder.add(new TestInput(50500, tolerance, true));
+        builder.add(new TestInput(50501, tolerance, false));
+        builder.add(new TestInput(50999, tolerance, false));
+        builder.add(new TestInput(51000, tolerance, false));
+        builder.add(new TestInput(51001, tolerance, false));
+        builder.add(new TestInput(60000, tolerance, false));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Integer.MIN_VALUE, tolerance, true));
+        builder.add(new TestInput(-1, tolerance, true));
+        builder.add(new TestInput(0, tolerance, true));
+        builder.add(new TestInput(1, tolerance, true));
+        builder.add(new TestInput(48999, tolerance, true));
+        builder.add(new TestInput(49000, tolerance, true));
+        builder.add(new TestInput(49001, tolerance, true));
+        builder.add(new TestInput(49499, tolerance, true));
+        builder.add(new TestInput(49500, tolerance, true));
+        builder.add(new TestInput(49501, tolerance, true));
+        builder.add(new TestInput(49997, tolerance, true));
+        builder.add(new TestInput(49998, tolerance, true));
+        builder.add(new TestInput(49999, tolerance, true));
+        builder.add(new TestInput(50000, tolerance, true)); // Equals value
+        builder.add(new TestInput(50001, tolerance, true));
+        builder.add(new TestInput(50002, tolerance, true));
+        builder.add(new TestInput(50003, tolerance, true));
+        builder.add(new TestInput(50499, tolerance, true));
+        builder.add(new TestInput(50500, tolerance, true));
+        builder.add(new TestInput(50501, tolerance, true));
+        builder.add(new TestInput(50999, tolerance, true));
+        builder.add(new TestInput(51000, tolerance, true));
+        builder.add(new TestInput(51001, tolerance, true));
+        builder.add(new TestInput(60000, tolerance, true));
+        builder.add(new TestInput(Integer.MAX_VALUE, tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_INCOME);
+    }
+
+    @Test
+    public void testLongProperty() throws SmartUriException {
+        System.out.println("Long Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1L, tolerance, false));
+        builder.add(new TestInput(0L, tolerance, false));
+        builder.add(new TestInput(1L, tolerance, false));
+        builder.add(new TestInput(123456789009L, tolerance, false));
+        builder.add(new TestInput(123456789010L, tolerance, false));
+        builder.add(new TestInput(123456789011L, tolerance, false));
+        builder.add(new TestInput(123456789012L, tolerance, true)); // Equals value
+        builder.add(new TestInput(123456789013L, tolerance, false));
+        builder.add(new TestInput(123456789014L, tolerance, false));
+        builder.add(new TestInput(123456789015L, tolerance, false));
+        builder.add(new TestInput(223456789012L, tolerance, false));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1L, tolerance, false));
+        builder.add(new TestInput(0L, tolerance, false));
+        builder.add(new TestInput(1L, tolerance, false));
+        builder.add(new TestInput(123456789009L, tolerance, false));
+        builder.add(new TestInput(123456789010L, tolerance, false));
+        builder.add(new TestInput(123456789011L, tolerance, true));
+        builder.add(new TestInput(123456789012L, tolerance, true)); // Equals value
+        builder.add(new TestInput(123456789013L, tolerance, true));
+        builder.add(new TestInput(123456789014L, tolerance, false));
+        builder.add(new TestInput(123456789015L, tolerance, false));
+        builder.add(new TestInput(223456789012L, tolerance, false));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1L, tolerance, false));
+        builder.add(new TestInput(0L, tolerance, false));
+        builder.add(new TestInput(1L, tolerance, false));
+        builder.add(new TestInput(123456789009L, tolerance, false));
+        builder.add(new TestInput(123456789010L, tolerance, true));
+        builder.add(new TestInput(123456789011L, tolerance, true));
+        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
+        builder.add(new TestInput(123456789013L, tolerance, true));
+        builder.add(new TestInput(123456789014L, tolerance, true));
+        builder.add(new TestInput(123456789015L, tolerance, false));
+        builder.add(new TestInput(223456789012L, tolerance, false));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.0, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1L, tolerance, false));
+        builder.add(new TestInput(0L, tolerance, false));
+        builder.add(new TestInput(1L, tolerance, false));
+        builder.add(new TestInput(122222221121L, tolerance, false));
+        builder.add(new TestInput(122222221122L, tolerance, false));
+        builder.add(new TestInput(122222221123L, tolerance, false));
+        builder.add(new TestInput(123456789009L, tolerance, false));
+        builder.add(new TestInput(123456789010L, tolerance, false));
+        builder.add(new TestInput(123456789011L, tolerance, false));
+        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
+        builder.add(new TestInput(123456789013L, tolerance, false));
+        builder.add(new TestInput(123456789014L, tolerance, false));
+        builder.add(new TestInput(123456789015L, tolerance, false));
+        builder.add(new TestInput(124691356901L, tolerance, false));
+        builder.add(new TestInput(124691356902L, tolerance, false));
+        builder.add(new TestInput(124691356903L, tolerance, false));
+        builder.add(new TestInput(223456789012L, tolerance, false));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0%
+        tolerance = new Tolerance(0.01, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput(-1L, tolerance, false));
+        builder.add(new TestInput(0L, tolerance, false));
+        builder.add(new TestInput(1L, tolerance, false));
+        builder.add(new TestInput(122222221121L, tolerance, false));
+        builder.add(new TestInput(122222221122L, tolerance, true));
+        builder.add(new TestInput(122222221123L, tolerance, true));
+        builder.add(new TestInput(123456789009L, tolerance, true));
+        builder.add(new TestInput(123456789010L, tolerance, true));
+        builder.add(new TestInput(123456789011L, tolerance, true));
+        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
+        builder.add(new TestInput(123456789013L, tolerance, true));
+        builder.add(new TestInput(123456789014L, tolerance, true));
+        builder.add(new TestInput(123456789015L, tolerance, true));
+        builder.add(new TestInput(124691356901L, tolerance, true));
+        builder.add(new TestInput(124691356902L, tolerance, true));
+        builder.add(new TestInput(124691356903L, tolerance, false));
+        builder.add(new TestInput(223456789012L, tolerance, false));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, false));
+        // Tolerance 100.0%
+        tolerance = new Tolerance(1.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Long.MIN_VALUE, tolerance, true));
+        builder.add(new TestInput(-1L, tolerance, true));
+        builder.add(new TestInput(0L, tolerance, true));
+        builder.add(new TestInput(1L, tolerance, true));
+        builder.add(new TestInput(122222221121L, tolerance, true));
+        builder.add(new TestInput(122222221122L, tolerance, true));
+        builder.add(new TestInput(122222221123L, tolerance, true));
+        builder.add(new TestInput(123456789009L, tolerance, true));
+        builder.add(new TestInput(123456789010L, tolerance, true));
+        builder.add(new TestInput(123456789011L, tolerance, true));
+        builder.add(new TestInput(123456789012L, tolerance, true));// Equals value
+        builder.add(new TestInput(123456789013L, tolerance, true));
+        builder.add(new TestInput(123456789014L, tolerance, true));
+        builder.add(new TestInput(123456789015L, tolerance, true));
+        builder.add(new TestInput(124691356901L, tolerance, true));
+        builder.add(new TestInput(124691356902L, tolerance, true));
+        builder.add(new TestInput(124691356903L, tolerance, true));
+        builder.add(new TestInput(223456789012L, tolerance, true));
+        builder.add(new TestInput(Long.MAX_VALUE, tolerance, true));
+
+        final ImmutableList<TestInput> testInputs = builder.build();
+
+        testProperty(testInputs, PERSON_TYPE_URI, HAS_LICENSE_NUMBER);
+    }
+
+    @Test
+    public void testShortProperty() throws SmartUriException {
+        System.out.println("Short Property Test");
+        final ImmutableList.Builder<TestInput> builder = ImmutableList.<TestInput>builder();
+        // Tolerance 0.0
+        Tolerance tolerance = new Tolerance(0.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((short) -1, tolerance, false));
+        builder.add(new TestInput((short) 0, tolerance, false));
+        builder.add(new TestInput((short) 1, tolerance, false));
+        builder.add(new TestInput((short) 37, tolerance, false));
+        builder.add(new TestInput((short) 38, tolerance, false));
+        builder.add(new TestInput((short) 39, tolerance, false));
+        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
+        builder.add(new TestInput((short) 41, tolerance, false));
+        builder.add(new TestInput((short) 42, tolerance, false));
+        builder.add(new TestInput((short) 43, tolerance, false));
+        builder.add(new TestInput((short) 100, tolerance, false));
+        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
+        // Tolerance 1.0
+        tolerance = new Tolerance(1.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((short) -1, tolerance, false));
+        builder.add(new TestInput((short) 0, tolerance, false));
+        builder.add(new TestInput((short) 1, tolerance, false));
+        builder.add(new TestInput((short) 37, tolerance, false));
+        builder.add(new TestInput((short) 38, tolerance, false));
+        builder.add(new TestInput((short) 39, tolerance, true));
+        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
+        builder.add(new TestInput((short) 41, tolerance, true));
+        builder.add(new TestInput((short) 42, tolerance, false));
+        builder.add(new TestInput((short) 43, tolerance, false));
+        builder.add(new TestInput((short) 100, tolerance, false));
+        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
+        // Tolerance 2.0
+        tolerance = new Tolerance(2.0, ToleranceType.DIFFERENCE);
+        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((short) -1, tolerance, false));
+        builder.add(new TestInput((short) 0, tolerance, false));
+        builder.add(new TestInput((short) 1, tolerance, false));
+        builder.add(new TestInput((short) 37, tolerance, false));
+        builder.add(new TestInput((short) 38, tolerance, true));
+        builder.add(new TestInput((short) 39, tolerance, true));
+        builder.add(new TestInput((short) 40, tolerance, true)); // Equals value
+        builder.add(new TestInput((short) 41, tolerance, true));
+        builder.add(new TestInput((short) 42, tolerance, true));
+        builder.add(new TestInput((short) 43, tolerance, false));
+        builder.add(new TestInput((short) 100, tolerance, false));
+        builder.add(new TestInput(Short.MAX_VALUE, tolerance, false));
+
+        // Tolerance 0.0%
+        tolerance = new Tolerance(0.00, ToleranceType.PERCENTAGE);
+        builder.add(new TestInput(Short.MIN_VALUE, tolerance, false));
+        builder.add(new TestInput((short) -1, tolerance, false));
+        builder.add(new TestInput((short) 0, tolerance, false));
+        builder.add(new TestInput((short) 1, tolerance, false));
+        builder.add(new TestInput((short) 37, tolerance, false));
+        builder.add(new TestInp

<TRUNCATED>


[06/17] incubator-rya git commit: RYA-414 Removed mongo connection factory

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 d005ce7..aa700b1 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
@@ -27,12 +27,10 @@ import org.apache.rya.api.domain.RyaStatement;
 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.statement.metadata.matching.StatementMetadataNode;
 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;
 import org.junit.Test;
@@ -40,7 +38,6 @@ import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.impl.URIImpl;
 import org.openrdf.model.vocabulary.XMLSchema;
 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;
@@ -51,307 +48,321 @@ import org.openrdf.query.parser.sparql.SPARQLParser;
 import info.aduna.iteration.CloseableIteration;
 
 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 {
-        final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
-        conf.setUseStatementMetadata(true);
-        conf.setStatementMetadataProperties(propertySet);
-        
-        dao = new MongoDBRyaDAO(conf, super.getMongoClient());
-        dao.init();
-    }
-
-    @Test
-    public void simpleQueryWithoutBindingSet()
-            throws MalformedQueryException, QueryEvaluationException, RyaDAOException {
-        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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-
-        StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-        QueryBindingSet bs = new QueryBindingSet();
-        bs.addBinding("x", new LiteralImpl("CoffeeShop"));
-        bs.addBinding("y", new LiteralImpl("Joe"));
-
-        List<BindingSet> bsList = new ArrayList<>();
-        while (iteration.hasNext()) {
-            bsList.add(iteration.next());
-        }
-
-        Assert.assertEquals(1, bsList.size());
-        Assert.assertEquals(bs, bsList.get(0));
-        dao.delete(statement, conf);
-    }
-
-    /**
-     * 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 MalformedQueryException, QueryEvaluationException, RyaDAOException {
-        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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-        StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
-
-        List<BindingSet> bsList = new ArrayList<>();
-        while (iteration.hasNext()) {
-            bsList.add(iteration.next());
-        }
-        Assert.assertEquals(0, bsList.size());
-        dao.delete(statement, conf);
-    }
-
-    @Test
-    public void simpleQueryWithBindingSet() throws MalformedQueryException, QueryEvaluationException, RyaDAOException {
-
-        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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        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);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-        StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-        QueryBindingSet bsConstraint = new QueryBindingSet();
-        bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-        bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
-
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-        QueryBindingSet expected = new QueryBindingSet();
-        expected.addBinding("x", new LiteralImpl("CoffeeShop"));
-        expected.addBinding("y", new LiteralImpl("Joe"));
-        expected.addBinding("z", new LiteralImpl("Virginia"));
-
-        List<BindingSet> bsList = new ArrayList<>();
-        while (iteration.hasNext()) {
-            bsList.add(iteration.next());
-        }
-
-        Assert.assertEquals(1, bsList.size());
-        Assert.assertEquals(expected, bsList.get(0));
-
-        dao.delete(statement1, conf);
-        dao.delete(statement2, conf);
-    }
-
-    /**
-     * Tests to see if correct result is passed back when a metadata statement
-     * is joined with a StatementPattern statement (i.e. a common variable
-     * appears in a StatementPattern statement and a metadata statement).
-     * StatementPattern statements have either rdf:subject, rdf:predicate, or
-     * rdf:object as the predicate while a metadata statement is any statement
-     * in the reified query whose predicate is not rdf:type and not a
-     * StatementPattern predicate.
-     * 
-     * @throws MalformedQueryException
-     * @throws QueryEvaluationException
-     * @throws RyaDAOException
-     */
-    @Test
-    public void simpleQueryWithBindingSetJoinPropertyToSubject()
-            throws MalformedQueryException, QueryEvaluationException, RyaDAOException {
-
-        StatementMetadata metadata = new StatementMetadata();
-        metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
-                new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement1);
-        dao.add(statement2);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query2, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-        StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-        List<BindingSet> bsCollection = new ArrayList<>();
-        QueryBindingSet bsConstraint1 = new QueryBindingSet();
-        bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
-        bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-        QueryBindingSet bsConstraint2 = new QueryBindingSet();
-        bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
-        bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-        bsCollection.add(bsConstraint1);
-        bsCollection.add(bsConstraint2);
-
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-        QueryBindingSet expected = new QueryBindingSet();
-        expected.addBinding("y", new LiteralImpl("CoffeeShop"));
-        expected.addBinding("x", new URIImpl("http://Joe"));
-        expected.addBinding("z", new LiteralImpl("Virginia"));
-
-        List<BindingSet> bsList = new ArrayList<>();
-        while (iteration.hasNext()) {
-            bsList.add(iteration.next());
-        }
-
-        Assert.assertEquals(1, bsList.size());
-        Assert.assertEquals(expected, bsList.get(0));
-
-        dao.delete(statement1, conf);
-        dao.delete(statement2, conf);
-    }
-
-    /**
-     * Tests if the StatementMetadataNode joins BindingSet correctly for
-     * variables appearing in metadata statements. In this case, the metadata
-     * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
-     * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
-     * object in the above metadata statement and its values are joined to the
-     * constraint BindingSets in the example below.
-     * 
-     * @throws MalformedQueryException
-     * @throws QueryEvaluationException
-     * @throws RyaDAOException
-     */
-    @Test
-    public void simpleQueryWithBindingSetJoinOnProperty()
-            throws MalformedQueryException, QueryEvaluationException, RyaDAOException {
-
-        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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        dao.add(statement1);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-        StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-        QueryBindingSet bsConstraint = new QueryBindingSet();
-        bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
-        bsConstraint.addBinding("y", new LiteralImpl("Doug"));
-
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
-
-        List<BindingSet> bsList = new ArrayList<>();
-        while (iteration.hasNext()) {
-            bsList.add(iteration.next());
-        }
-
-        Assert.assertEquals(0, bsList.size());
-        dao.delete(statement1, conf);
-    }
-
-    /**
-     * Tests if StatementMetadataNode joins BindingSet values correctly for
-     * variables appearing as the object in one of the StatementPattern
-     * statements (in the case ?x appears as the Object in the statement
-     * _:blankNode rdf:object ?x). StatementPattern statements have either
-     * rdf:subject, rdf:predicate, or rdf:object as the predicate.
-     * 
-     * @throws MalformedQueryException
-     * @throws QueryEvaluationException
-     * @throws RyaDAOException
-     */
-    @Test
-    public void simpleQueryWithBindingSetCollection()
-            throws MalformedQueryException, QueryEvaluationException, RyaDAOException {
-
-        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"),
-                new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
-        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);
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq = parser.parseQuery(query, null);
-        List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
-        StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
-
-        List<BindingSet> bsCollection = new ArrayList<>();
-        QueryBindingSet bsConstraint1 = new QueryBindingSet();
-        bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
-        bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
-
-        QueryBindingSet bsConstraint2 = new QueryBindingSet();
-        bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
-        bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
-
-        QueryBindingSet bsConstraint3 = new QueryBindingSet();
-        bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
-        bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
-        bsCollection.add(bsConstraint1);
-        bsCollection.add(bsConstraint2);
-        bsCollection.add(bsConstraint3);
-
-        CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
-
-        Set<BindingSet> expected = new HashSet<>();
-        QueryBindingSet expected1 = new QueryBindingSet();
-        expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
-        expected1.addBinding("y", new LiteralImpl("Joe"));
-        expected1.addBinding("z", new LiteralImpl("Virginia"));
-
-        QueryBindingSet expected2 = new QueryBindingSet();
-        expected2.addBinding("x", new LiteralImpl("HardwareStore"));
-        expected2.addBinding("y", new LiteralImpl("Joe"));
-        expected2.addBinding("z", new LiteralImpl("Maryland"));
-        expected.add(expected1);
-        expected.add(expected2);
-
-        Set<BindingSet> bsSet = new HashSet<>();
-        while (iteration.hasNext()) {
-            bsSet.add(iteration.next());
-        }
-
-        Assert.assertEquals(expected, bsSet);
-
-        dao.delete(statement1, conf);
-        dao.delete(statement2, conf);
-    }
+	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 {
+		final Set<RyaURI> propertySet = new HashSet<RyaURI>(Arrays.asList(new RyaURI("http://createdBy"), new RyaURI("http://createdOn")));
+		conf.setUseStatementMetadata(true);
+		conf.setStatementMetadataProperties(propertySet);
+	}
+
+	@Test
+	public void simpleQueryWithoutBindingSet() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+
+			StatementMetadataNode<?> node = new StatementMetadataNode<>(spList, conf);
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+			QueryBindingSet bs = new QueryBindingSet();
+			bs.addBinding("x", new LiteralImpl("CoffeeShop"));
+			bs.addBinding("y", new LiteralImpl("Joe"));
+
+			List<BindingSet> bsList = new ArrayList<>();
+			while (iteration.hasNext()) {
+				bsList.add(iteration.next());
+			}
+
+			Assert.assertEquals(1, bsList.size());
+			Assert.assertEquals(bs, bsList.get(0));
+			dao.delete(statement, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
+
+	/**
+	 * 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.
+	 */
+	@Test
+	public void simpleQueryWithoutBindingSetInvalidProperty() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();	
+
+			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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(new QueryBindingSet());
+
+			List<BindingSet> bsList = new ArrayList<>();
+			while (iteration.hasNext()) {
+				bsList.add(iteration.next());
+			}
+			Assert.assertEquals(0, bsList.size());
+			dao.delete(statement, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
+
+	@Test
+	public void simpleQueryWithBindingSet() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			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);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+			QueryBindingSet bsConstraint = new QueryBindingSet();
+			bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+			bsConstraint.addBinding("z", new LiteralImpl("Virginia"));
+
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+			QueryBindingSet expected = new QueryBindingSet();
+			expected.addBinding("x", new LiteralImpl("CoffeeShop"));
+			expected.addBinding("y", new LiteralImpl("Joe"));
+			expected.addBinding("z", new LiteralImpl("Virginia"));
+
+			List<BindingSet> bsList = new ArrayList<>();
+			while (iteration.hasNext()) {
+				bsList.add(iteration.next());
+			}
+
+			Assert.assertEquals(1, bsList.size());
+			Assert.assertEquals(expected, bsList.get(0));
+
+			dao.delete(statement1, conf);
+			dao.delete(statement2, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
+
+	/**
+	 * Tests to see if correct result is passed back when a metadata statement
+	 * is joined with a StatementPattern statement (i.e. a common variable
+	 * appears in a StatementPattern statement and a metadata statement).
+	 * StatementPattern statements have either rdf:subject, rdf:predicate, or
+	 * rdf:object as the predicate while a metadata statement is any statement
+	 * in the reified query whose predicate is not rdf:type and not a
+	 * StatementPattern predicate.
+	 */
+	@Test
+	public void simpleQueryWithBindingSetJoinPropertyToSubject() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			StatementMetadata metadata = new StatementMetadata();
+			metadata.addMetadata(new RyaURI("http://createdBy"), new RyaURI("http://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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			RyaStatement statement2 = new RyaStatement(new RyaURI("http://Bob"), new RyaURI("http://worksAt"),
+					new RyaType("HardwareStore"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement1);
+			dao.add(statement2);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query2, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+			List<BindingSet> bsCollection = new ArrayList<>();
+			QueryBindingSet bsConstraint1 = new QueryBindingSet();
+			bsConstraint1.addBinding("y", new LiteralImpl("CoffeeShop"));
+			bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+			QueryBindingSet bsConstraint2 = new QueryBindingSet();
+			bsConstraint2.addBinding("y", new LiteralImpl("HardwareStore"));
+			bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+			bsCollection.add(bsConstraint1);
+			bsCollection.add(bsConstraint2);
+
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+			QueryBindingSet expected = new QueryBindingSet();
+			expected.addBinding("y", new LiteralImpl("CoffeeShop"));
+			expected.addBinding("x", new URIImpl("http://Joe"));
+			expected.addBinding("z", new LiteralImpl("Virginia"));
+
+			List<BindingSet> bsList = new ArrayList<>();
+			while (iteration.hasNext()) {
+				bsList.add(iteration.next());
+			}
+
+			Assert.assertEquals(1, bsList.size());
+			Assert.assertEquals(expected, bsList.get(0));
+
+			dao.delete(statement1, conf);
+			dao.delete(statement2, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
+
+	/**
+	 * Tests if the StatementMetadataNode joins BindingSet correctly for
+	 * variables appearing in metadata statements. In this case, the metadata
+	 * statements are (_:blankNode <http://createdOn 2017-01-04 ) and
+	 * (_:blankNode <http://createdBy> ?y). The variable ?y appears as the
+	 * object in the above metadata statement and its values are joined to the
+	 * constraint BindingSets in the example below.
+	 */
+	@Test
+	public void simpleQueryWithBindingSetJoinOnProperty() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			dao.add(statement1);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+			QueryBindingSet bsConstraint = new QueryBindingSet();
+			bsConstraint.addBinding("x", new LiteralImpl("CoffeeShop"));
+			bsConstraint.addBinding("y", new LiteralImpl("Doug"));
+
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsConstraint);
+
+			List<BindingSet> bsList = new ArrayList<>();
+			while (iteration.hasNext()) {
+				bsList.add(iteration.next());
+			}
+
+			Assert.assertEquals(0, bsList.size());
+			dao.delete(statement1, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
+
+	/**
+	 * Tests if StatementMetadataNode joins BindingSet values correctly for
+	 * variables appearing as the object in one of the StatementPattern
+	 * statements (in the case ?x appears as the Object in the statement
+	 * _:blankNode rdf:object ?x). StatementPattern statements have either
+	 * rdf:subject, rdf:predicate, or rdf:object as the predicate.
+	 */
+	@Test
+	public void simpleQueryWithBindingSetCollection() throws Exception {
+		MongoDBRyaDAO dao = new MongoDBRyaDAO();
+		try {
+			dao.setConf(conf);
+			dao.init();
+			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"),
+					new RyaType("CoffeeShop"), new RyaURI("http://context"), "", metadata);
+			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);
+
+			SPARQLParser parser = new SPARQLParser();
+			ParsedQuery pq = parser.parseQuery(query, null);
+			List<StatementPattern> spList = StatementPatternCollector.process(pq.getTupleExpr());
+			StatementMetadataNode<MongoDBRdfConfiguration> node = new StatementMetadataNode<>(spList, conf);
+
+			List<BindingSet> bsCollection = new ArrayList<>();
+			QueryBindingSet bsConstraint1 = new QueryBindingSet();
+			bsConstraint1.addBinding("x", new LiteralImpl("CoffeeShop"));
+			bsConstraint1.addBinding("z", new LiteralImpl("Virginia"));
+
+			QueryBindingSet bsConstraint2 = new QueryBindingSet();
+			bsConstraint2.addBinding("x", new LiteralImpl("HardwareStore"));
+			bsConstraint2.addBinding("z", new LiteralImpl("Maryland"));
+
+			QueryBindingSet bsConstraint3 = new QueryBindingSet();
+			bsConstraint3.addBinding("x", new LiteralImpl("BurgerShack"));
+			bsConstraint3.addBinding("z", new LiteralImpl("Delaware"));
+			bsCollection.add(bsConstraint1);
+			bsCollection.add(bsConstraint2);
+			bsCollection.add(bsConstraint3);
+
+			CloseableIteration<BindingSet, QueryEvaluationException> iteration = node.evaluate(bsCollection);
+
+			Set<BindingSet> expected = new HashSet<>();
+			QueryBindingSet expected1 = new QueryBindingSet();
+			expected1.addBinding("x", new LiteralImpl("CoffeeShop"));
+			expected1.addBinding("y", new LiteralImpl("Joe"));
+			expected1.addBinding("z", new LiteralImpl("Virginia"));
+
+			QueryBindingSet expected2 = new QueryBindingSet();
+			expected2.addBinding("x", new LiteralImpl("HardwareStore"));
+			expected2.addBinding("y", new LiteralImpl("Joe"));
+			expected2.addBinding("z", new LiteralImpl("Maryland"));
+			expected.add(expected1);
+			expected.add(expected2);
+
+			Set<BindingSet> bsSet = new HashSet<>();
+			while (iteration.hasNext()) {
+				bsSet.add(iteration.next());
+			}
+
+			Assert.assertEquals(expected, bsSet);
+
+			dao.delete(statement1, conf);
+			dao.delete(statement2, conf);
+		} finally {
+			dao.destroy();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/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 477e579..66d94db 100644
--- a/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
+++ b/extras/indexingExample/src/main/java/MongoRyaDirectExample.java
@@ -58,8 +58,8 @@ import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
 import com.mongodb.MongoClient;
-import com.mongodb.ServerAddress;
 
+import de.flapdoodle.embed.mongo.config.IMongoConfig;
 import info.aduna.iteration.Iterations;
 
 
@@ -295,12 +295,12 @@ public class MongoRyaDirectExample {
             .setUseMockMongo(USE_MOCK).setUseInference(USE_INFER).setAuths("U");
 
         if (USE_MOCK) {
-            final MongoClient c = EmbeddedMongoFactory.newFactory().newMongoClient();
-            final ServerAddress address = c.getAddress();
-            final String url = address.getHost();
-            final String port = Integer.toString(address.getPort());
-            c.close();
-            builder.setMongoHost(url).setMongoPort(port);
+        	final EmbeddedMongoFactory factory = EmbeddedMongoFactory.newFactory();
+            final MongoClient c = factory.newMongoClient();
+            final IMongoConfig connectionConfig = factory.getMongoServerDetails();
+            //c.close();
+            builder.setMongoHost(connectionConfig.net().getServerAddress().getHostAddress())
+            	   .setMongoPort(connectionConfig.net().getPort() + "");
         } else {
             // User name and password must be filled in:
             builder = builder.setMongoUser(MONGO_USER)

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/conf/MergeConfigHadoopAdapter.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/conf/MergeConfigHadoopAdapter.java b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/conf/MergeConfigHadoopAdapter.java
index c51d637..3b722f1 100644
--- a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/conf/MergeConfigHadoopAdapter.java
+++ b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/conf/MergeConfigHadoopAdapter.java
@@ -28,8 +28,8 @@ import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 public class MergeConfigHadoopAdapter {
     public static MongoDBRdfConfiguration getMongoConfiguration(final MergeConfiguration config) {
         final MongoDBRdfConfiguration configuration = new MongoDBRdfConfiguration();
-        configuration.setMongoInstance(config.getChildHostname());
-        configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, config.getChildPort() + "");
+        configuration.setMongoHostname(config.getChildHostname());
+        configuration.set(MongoDBRdfConfiguration.MONGO_PORT, config.getChildPort() + "");
         configuration.set(MongoDBRdfConfiguration.MONGO_DB_NAME, config.getChildRyaInstanceName());
         return configuration;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
index c96a2b6..2f7dca7 100644
--- a/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
+++ b/extras/rya.export/export.client/src/main/java/org/apache/rya/export/client/merge/StatementStoreFactory.java
@@ -20,6 +20,7 @@ package org.apache.rya.export.client.merge;
 
 import static java.util.Objects.requireNonNull;
 
+import java.util.ArrayList;
 import java.util.Date;
 
 import org.apache.rya.accumulo.AccumuloRyaDAO;
@@ -37,8 +38,8 @@ import org.apache.rya.export.api.store.RyaStatementStore;
 import org.apache.rya.export.client.conf.MergeConfigHadoopAdapter;
 import org.apache.rya.export.mongo.MongoRyaStatementStore;
 import org.apache.rya.export.mongo.policy.TimestampPolicyMongoRyaStatementStore;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
 import com.mongodb.MongoClient;
 
@@ -121,7 +122,9 @@ public class StatementStoreFactory {
 
     private MongoRyaStatementStore getBaseMongoStore(final String hostname, final int port, final String ryaInstanceName) throws RyaDAOException {
         final MongoClient client = new MongoClient(hostname, port);
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO(new MongoDBRdfConfiguration(MergeConfigHadoopAdapter.getMongoConfiguration(configuration)), client);
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        dao.setConf(new StatefulMongoDBRdfConfiguration(MergeConfigHadoopAdapter.getMongoConfiguration(configuration), client, new ArrayList<>()));
+        dao.init();
         return new MongoRyaStatementStore(client, ryaInstanceName, dao);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
index 2eef621..c982d21 100644
--- a/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
+++ b/extras/rya.export/export.integration/src/test/java/org/apache/rya/indexing/export/StoreToStoreIT.java
@@ -41,6 +41,7 @@ import org.apache.rya.export.client.merge.VisibilityStatementMerger;
 import org.apache.rya.export.mongo.MongoRyaStatementStore;
 import org.apache.rya.export.mongo.policy.TimestampPolicyMongoRyaStatementStore;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -65,7 +66,9 @@ public class StoreToStoreIT extends ITBase {
 
     private static TimestampPolicyMongoRyaStatementStore getParentMongo() throws Exception {
         final MongoClient mongo = getNewMongoResources(RYA_INSTANCE);
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO(ITBase.getConf(mongo), mongo);
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo, new ArrayList<>()));
+        dao.init();
         final MongoRyaStatementStore store = new MongoRyaStatementStore(mongo, RYA_INSTANCE, dao);
         final TimestampPolicyMongoRyaStatementStore timeStore = new TimestampPolicyMongoRyaStatementStore(store, currentDate, RYA_INSTANCE);
         clients.add(mongo);
@@ -74,7 +77,9 @@ public class StoreToStoreIT extends ITBase {
 
     private static MongoRyaStatementStore getChildMongo() throws Exception {
         final MongoClient mongo = getNewMongoResources(RYA_INSTANCE);
-        final MongoDBRyaDAO dao = new MongoDBRyaDAO(ITBase.getConf(mongo), mongo);
+        final MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        dao.setConf(new StatefulMongoDBRdfConfiguration(ITBase.getConf(mongo), mongo, new ArrayList<>()));
+        dao.init();
         final MongoRyaStatementStore store = new MongoRyaStatementStore(mongo, RYA_INSTANCE, dao);
         clients.add(mongo);
         return store;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
index 6ad0edc..eee7310 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoEnabledFilterFunctionOptimizer.java
@@ -1,6 +1,4 @@
-package org.apache.rya.indexing;
-
-/*
+/**
  * 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
@@ -18,7 +16,7 @@ package org.apache.rya.indexing;
  * specific language governing permissions and limitations
  * under the License.
  */
-
+package org.apache.rya.indexing;
 
 import java.io.IOException;
 import java.net.UnknownHostException;
@@ -43,11 +41,10 @@ import org.apache.rya.indexing.accumulo.geo.GeoParseUtils;
 import org.apache.rya.indexing.accumulo.geo.GeoTupleSet;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.indexing.accumulo.temporal.AccumuloTemporalIndexer;
-import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import org.apache.rya.indexing.mongodb.temporal.MongoTemporalIndexer;
+import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.model.Resource;
 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;
@@ -103,12 +100,16 @@ public class GeoEnabledFilterFunctionOptimizer implements QueryOptimizer, Config
         if (!init) {
 			if (ConfigUtils.getUseMongo(conf)) {
 				// create a new MongoGeoIndexer() without having it at compile time.
-				geoIndexer = instantiate(GeoIndexerType.MONGO_DB.getGeoIndexerClassString(), GeoIndexer.class);
-				geoIndexer.setConf(conf);
-				freeTextIndexer = new MongoFreeTextIndexer();
-				freeTextIndexer.setConf(conf);
-				temporalIndexer = new MongoTemporalIndexer();
-				temporalIndexer.setConf(conf);
+				StatefulMongoDBRdfConfiguration stateConf = (StatefulMongoDBRdfConfiguration) conf;
+            	for(final MongoSecondaryIndex indexer : stateConf.getAdditionalIndexers()) {
+        			if(indexer instanceof FreeTextIndexer) {
+        				freeTextIndexer = (FreeTextIndexer) indexer;
+        			} else if(indexer instanceof TemporalIndexer) {
+        				temporalIndexer = (TemporalIndexer) indexer;
+        			} else if(indexer instanceof GeoIndexer) {
+        				geoIndexer = (GeoIndexer) indexer;
+        			}
+            	}
 			} else {
 				GeoIndexerType geoIndexerType = OptionalConfigUtils.getGeoIndexerType(conf);
 				if (geoIndexerType == GeoIndexerType.UNSPECIFIED) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
index 3c01bf6..ae6af10 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/GeoRyaSailFactory.java
@@ -20,25 +20,17 @@ package org.apache.rya.indexing;
 
 import static java.util.Objects.requireNonNull;
 
-import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Objects;
 
 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.Instance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.commons.configuration.ConfigurationRuntimeException;
 import org.apache.hadoop.conf.Configuration;
-import org.openrdf.sail.Sail;
-import org.openrdf.sail.SailException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.mongodb.MongoClient;
-
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.AccumuloRyaDAO;
-import org.apache.rya.accumulo.instance.AccumuloRyaInstanceDetailsRepository;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.instance.RyaDetailsRepository.RyaDetailsRepositoryException;
 import org.apache.rya.api.instance.RyaDetailsToConfiguration;
@@ -47,14 +39,24 @@ import org.apache.rya.api.persist.RyaDAO;
 import org.apache.rya.api.persist.RyaDAOException;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
+import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.apache.rya.mongodb.instance.MongoRyaInstanceDetailsRepository;
 import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
 import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
 import org.apache.rya.sail.config.RyaSailFactory;
+import org.openrdf.sail.Sail;
+import org.openrdf.sail.SailException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mongodb.MongoClient;
+import com.mongodb.MongoCredential;
+import com.mongodb.MongoException;
+import com.mongodb.ServerAddress;
 
 public class GeoRyaSailFactory {
     private static final Logger LOG = LoggerFactory.getLogger(GeoRyaSailFactory.class);
@@ -84,16 +86,36 @@ public class GeoRyaSailFactory {
         Objects.requireNonNull(ryaInstance, "RyaInstance or table prefix is missing from configuration."+RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX);
 
         if(ConfigUtils.getUseMongo(config)) {
-            final MongoDBRdfConfiguration mongoConfig = new MongoDBRdfConfiguration(config);
-            rdfConfig = mongoConfig;
-            final MongoClient client = MongoConnectorFactory.getMongoClient(config);
+        	// Get a reference to a Mongo DB configuration object.
+            final MongoDBRdfConfiguration mongoConfig = (config instanceof MongoDBRdfConfiguration) ?
+                    (MongoDBRdfConfiguration)config : new MongoDBRdfConfiguration(config);
+
+            // Create the MongoClient that will be used by the Sail object's components.
+            final MongoClient client = createMongoClient(mongoConfig);
+            
+            // Add the Indexer and Optimizer names to the configuration object that are configured to be used.
+            OptionalConfigUtils.setIndexers(mongoConfig);
+            
+            // Initialize the indexer and optimizer objects that will be used within the Sail object.
+            final List<MongoSecondaryIndex> indexers = mongoConfig.getInstances(AccumuloRdfConfiguration.CONF_ADDITIONAL_INDEXERS, MongoSecondaryIndex.class);
+
+            // Populate the configuration using previously stored Rya Details if this instance uses them.
             try {
-                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getCollectionName());
+                final MongoRyaInstanceDetailsRepository ryaDetailsRepo = new MongoRyaInstanceDetailsRepository(client, mongoConfig.getRyaInstance());
                 RyaDetailsToConfiguration.addRyaDetailsToConfiguration(ryaDetailsRepo.getRyaInstanceDetails(), mongoConfig);
             } catch (final RyaDetailsRepositoryException e) {
                 LOG.info("Instance does not have a rya details collection, skipping.");
             }
-            dao = getMongoDAO((MongoDBRdfConfiguration)rdfConfig, client);
+
+            // Set the configuration to the stateful configuration that is used to pass the constructed objects around.
+            final StatefulMongoDBRdfConfiguration statefulConfig = new StatefulMongoDBRdfConfiguration(mongoConfig, client, indexers);
+            rdfConfig = statefulConfig;
+
+            // Create the DAO that is able to interact with MongoDB.
+            final MongoDBRyaDAO mongoDao = new MongoDBRyaDAO();
+            mongoDao.setConf(statefulConfig);
+            mongoDao.init();
+            dao = mongoDao;
         } else {
             rdfConfig = new AccumuloRdfConfiguration(config);
             user = rdfConfig.get(ConfigUtils.CLOUDBASE_USER);
@@ -120,20 +142,39 @@ public class GeoRyaSailFactory {
         return store;
     }
 
-    private static MongoDBRyaDAO getMongoDAO(final MongoDBRdfConfiguration config, final MongoClient client) throws RyaDAOException {
-        MongoDBRyaDAO dao = null;
-        OptionalConfigUtils.setIndexers(config);
-        if(client != null) {
-            dao = new MongoDBRyaDAO(config, client);
+    /**
+     * Create a {@link MongoClient} that is connected to the configured database.
+     *
+     * @param mongoConf - Configures what will be connected to. (not null)
+     * @throws ConfigurationRuntimeException An invalid port was provided by {@code mongoConf}.
+     * @throws MongoException Couldn't connect to the MongoDB database.
+     */
+    private static MongoClient createMongoClient(final MongoDBRdfConfiguration mongoConf) throws ConfigurationRuntimeException, MongoException {
+        requireNonNull(mongoConf);
+        requireNonNull(mongoConf.getMongoHostname());
+        requireNonNull(mongoConf.getMongoPort());
+        requireNonNull(mongoConf.getMongoDBName());
+
+        // Connect to a running MongoDB server.
+        final int port;
+        try {
+            port = Integer.parseInt( mongoConf.getMongoPort() );
+        } catch(final NumberFormatException e) {
+            throw new ConfigurationRuntimeException("Port '" + mongoConf.getMongoPort() + "' must be an integer.");
+        }
+
+        final ServerAddress server = new ServerAddress(mongoConf.getMongoHostname(), port);
+
+        // Connect to a specific MongoDB Database if that information is provided.
+        final String username = mongoConf.getMongoUser();
+        final String database = mongoConf.getMongoDBName();
+        final String password = mongoConf.getMongoPassword();
+        if(username != null && password != null) {
+            final MongoCredential cred = MongoCredential.createCredential(username, database, password.toCharArray());
+            return new MongoClient(server, Arrays.asList(cred));
         } else {
-            try {
-                dao = new MongoDBRyaDAO(config);
-            } catch (NumberFormatException | UnknownHostException e) {
-                throw new RyaDAOException("Unable to connect to mongo at the configured location.", e);
-            }
+            return new MongoClient(server);
         }
-        dao.init();
-        return dao;
     }
 
     private static AccumuloRyaDAO getAccumuloDAO(final AccumuloRdfConfiguration config) throws AccumuloException, AccumuloSecurityException, RyaDAOException {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
index 106588b..cb34bd0 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexer.java
@@ -18,7 +18,6 @@
  */
 package org.apache.rya.indexing.geotemporal;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.api.persist.index.RyaSecondaryIndexer;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
@@ -33,15 +32,15 @@ public interface GeoTemporalIndexer extends RyaSecondaryIndexer {
 	/**
 	 * initialize after setting configuration.
 	 */
-    public void init();  
+    @Override
+	public void init();  
 
     /**
-     * Creates the {@link Eventtorage} that will be used by the indexer.
-     *
-     * @param conf - Indicates how the {@link EventStorage} is initialized. (not null)
+     * Creates the {@link EventStorage} that will be used by the indexer.
+     * NOTE: {@link #setConf(org.apache.hadoop.conf.Configuration)} must be called before calling this.
      * @return The {@link EventStorage} that will be used by this indexer.
      */
-    public abstract EventStorage getEventStorage(final Configuration conf);
+    public abstract EventStorage getEventStorage();
 
     /**
      * Used to indicate which geo filter functions to use in a query.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexerFactory.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexerFactory.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexerFactory.java
index f4df8bc..24e9666 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexerFactory.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalIndexerFactory.java
@@ -19,13 +19,11 @@
 package org.apache.rya.indexing.geotemporal;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.rya.indexing.GeoEnabledFilterFunctionOptimizer;
-import org.apache.rya.indexing.GeoIndexer;
-import org.apache.rya.indexing.GeoIndexerType;
-import org.apache.rya.indexing.GeoTemporalIndexerType;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
+
+import com.google.common.base.Preconditions;
 
 /**
  * Factory for retrieving a {@link GeoTemporalIndexer} based on a provided {@link Configuration}.
@@ -38,17 +36,16 @@ public class GeoTemporalIndexerFactory {
      */
     public GeoTemporalIndexer getIndexer(final Configuration conf) {
         if(ConfigUtils.getUseMongo(conf)) {
-            final MongoDBRdfConfiguration config = new MongoDBRdfConfiguration(conf);
-            for(final MongoSecondaryIndex index : config.getAdditionalIndexers()) {
+        	Preconditions.checkArgument(conf instanceof StatefulMongoDBRdfConfiguration, 
+        			"The configuration provided must be a StatefulMongoDBRdfConfiguration, found: " + conf.getClass().getSimpleName());
+            final StatefulMongoDBRdfConfiguration statefulConf = (StatefulMongoDBRdfConfiguration) conf;
+            for(final MongoSecondaryIndex index : statefulConf.getAdditionalIndexers()) {
                 if(index instanceof GeoTemporalIndexer) {
                     return (GeoTemporalIndexer) index;
                 }
             }
-            /* Created a  MongoGeoTemporalIndexer */
-            final GeoTemporalIndexer index = GeoEnabledFilterFunctionOptimizer.instantiate(GeoTemporalIndexerType.MONGO_GEO_TEMPORAL.getGeoTemporalIndexerClassString(), GeoTemporalIndexer.class);
-            index.setConf(conf);
-            index.init();
-            return index;
+            
+            throw new IllegalStateException("Geo Temporal Indexing is not turned on. Check configuration.");
         } else {
             //TODO: add Accumulo here.
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalOptimizer.java b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalOptimizer.java
index d626adc..f6e57a7 100644
--- a/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalOptimizer.java
+++ b/extras/rya.geoindexing/geo.common/src/main/java/org/apache/rya/indexing/geotemporal/GeoTemporalOptimizer.java
@@ -44,7 +44,7 @@ public class GeoTemporalOptimizer extends AbstractExternalSetOptimizer<EventQuer
         indexer = factory.getIndexer(conf);
 
         //conf here does not matter since EventStorage has already been set in the indexer.
-        provider = new GeoTemporalIndexSetProvider(indexer.getEventStorage(conf));
+        provider = new GeoTemporalIndexSetProvider(indexer.getEventStorage());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/pom.xml
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/pom.xml b/extras/rya.geoindexing/geo.mongo/pom.xml
index 2631e06..261f061 100644
--- a/extras/rya.geoindexing/geo.mongo/pom.xml
+++ b/extras/rya.geoindexing/geo.mongo/pom.xml
@@ -43,5 +43,11 @@
             <artifactId>gt-xsd-gml3</artifactId>
             <version>${geotools.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.rya</groupId>
+            <artifactId>mongodb.rya</artifactId>
+			<type>test-jar</type>
+			<scope>test</scope>            
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geoExamples/RyaMongoGeoDirectExample.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geoExamples/RyaMongoGeoDirectExample.java b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geoExamples/RyaMongoGeoDirectExample.java
index ede3f98..04488bb 100644
--- a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geoExamples/RyaMongoGeoDirectExample.java
+++ b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geoExamples/RyaMongoGeoDirectExample.java
@@ -30,18 +30,14 @@ import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
 import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration.MongoDBIndexingConfigBuilder;
 import org.apache.rya.mongodb.EmbeddedMongoFactory;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 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.TupleQuery;
 import org.openrdf.query.TupleQueryResultHandler;
 import org.openrdf.query.TupleQueryResultHandlerException;
 import org.openrdf.query.Update;
-import org.openrdf.query.UpdateExecutionException;
 import org.openrdf.repository.RepositoryException;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
@@ -90,7 +86,6 @@ public class RyaMongoGeoDirectExample {
             if (mock != null) {
                 mock.shutdown();
             }
-            MongoConnectorFactory.closeMongoClient();
         }
     }
 /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
index 2561c23..62a2181 100644
--- a/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
+++ b/extras/rya.geoindexing/geo.mongo/src/main/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.rya.indexing.geotemporal.mongo;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import static java.util.Objects.requireNonNull;
 
@@ -43,8 +44,7 @@ import org.apache.rya.indexing.geotemporal.storage.EventStorage;
 import org.apache.rya.indexing.mongodb.AbstractMongoIndexer;
 import org.apache.rya.indexing.mongodb.IndexingException;
 import org.apache.rya.indexing.mongodb.geo.GmlParser;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.joda.time.DateTime;
 import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
@@ -77,12 +77,13 @@ public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMon
     private static final Logger LOG = Logger.getLogger(MongoGeoTemporalIndexer.class);
     public static final String GEO_TEMPORAL_COLLECTION = "geo_temporal";
 
-    private final AtomicReference<MongoDBRdfConfiguration> configuration = new AtomicReference<>();
     private final AtomicReference<EventStorage> events = new AtomicReference<>();
 
     @Override
     public void init() {
         initCore();
+        events.set(getEventStorage());
+
         predicates = ConfigUtils.getGeoPredicates(conf);
         predicates.addAll(ConfigUtils.getTemporalPredicates(conf));
         storageStrategy = new GeoTemporalMongoDBStorageStrategy();
@@ -91,10 +92,10 @@ public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMon
     @Override
     public void setConf(final Configuration conf) {
         requireNonNull(conf);
-        events.set(null);
-        events.set(getEventStorage(conf));
-        super.conf = conf;
-        configuration.set(new MongoDBRdfConfiguration(conf));
+    	checkArgument(conf instanceof StatefulMongoDBRdfConfiguration, 
+    			"The configuration must be of type StatefulMongoDBRdfConfiguration but you provided: " 
+    		    + conf.getClass().getSimpleName());
+        super.conf = (StatefulMongoDBRdfConfiguration) conf;
     }
 
     @Override
@@ -206,22 +207,14 @@ public class MongoGeoTemporalIndexer extends AbstractMongoIndexer<GeoTemporalMon
     }
 
     @Override
-    public EventStorage getEventStorage(final Configuration conf) {
+    public EventStorage getEventStorage() {
         requireNonNull(conf);
 
         if(events.get() != null) {
             return events.get();
         }
 
-
-        final MongoDBRdfConfiguration mongoConf = new MongoDBRdfConfiguration(conf);
-        mongoClient = mongoConf.getMongoClient();
-        configuration.set(mongoConf);
-        if (mongoClient == null) {
-            mongoClient = MongoConnectorFactory.getMongoClient(conf);
-        }
-        final String ryaInstanceName = mongoConf.getMongoDBName();
-        events.set(new MongoEventStorage(mongoClient, ryaInstanceName));
+        events.set(new MongoEventStorage(conf.getMongoClient(), conf.getRyaInstance()));
         return events.get();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
index 7151b56..fa2f52b 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalProviderTest.java
@@ -36,7 +36,7 @@ import org.openrdf.model.Value;
 import org.openrdf.model.ValueFactory;
 import org.openrdf.model.impl.ValueFactoryImpl;
 
-public class GeoTemporalProviderTest extends GeoTemporalTestBase {
+public class GeoTemporalProviderTest extends GeoTemporalTestUtils {
     private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
     private GeoTemporalIndexSetProvider provider;
     private EventStorage events;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestBase.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestBase.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestBase.java
deleted file mode 100644
index 6b6bf15..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestBase.java
+++ /dev/null
@@ -1,140 +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.geotemporal;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.rya.indexing.TemporalInstant;
-import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.external.matching.QuerySegment;
-import org.apache.rya.indexing.geotemporal.model.EventQueryNode;
-import org.junit.ComparisonFailure;
-import org.mockito.Mockito;
-import org.openrdf.query.algebra.FunctionCall;
-import org.openrdf.query.algebra.QueryModelNode;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
-import org.openrdf.query.algebra.helpers.StatementPatternCollector;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import com.vividsolutions.jts.geom.Coordinate;
-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;
-
-public class GeoTemporalTestBase {
-    private static final GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
-
-    /**
-     * Make an uniform instant with given seconds.
-     */
-    protected static TemporalInstant makeInstant(final int secondsMakeMeUnique) {
-        return new TemporalInstantRfc3339(2015, 12, 30, 12, 00, secondsMakeMeUnique);
-    }
-
-    protected 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;
-    }
-
-    protected static Point point(final double x, final double y) {
-        return gf.createPoint(new Coordinate(x, y));
-    }
-
-    protected 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);
-    }
-
-    protected 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 };
-    }
-
-    protected void assertEqualMongo(final Object expected, final Object actual) throws ComparisonFailure {
-        try {
-            assertEquals(expected, actual);
-        } catch(final Throwable e) {
-            throw new ComparisonFailure(e.getMessage(), expected.toString(), actual.toString());
-        }
-    }
-
-    public List<FunctionCall> getFilters(final String query) throws Exception {
-        final FunctionCallCollector collector = new FunctionCallCollector();
-        new SPARQLParser().parseQuery(query, null).getTupleExpr().visit(collector);
-        return collector.getTupleExpr();
-    }
-
-    public List<StatementPattern> getSps(final String query) throws Exception {
-        final StatementPatternCollector collector = new StatementPatternCollector();
-        new SPARQLParser().parseQuery(query, null).getTupleExpr().visit(collector);
-        return collector.getStatementPatterns();
-    }
-
-    public QuerySegment<EventQueryNode> getQueryNode(final String query) throws Exception {
-        final List<QueryModelNode> exprs = getNodes(query);
-        final QuerySegment<EventQueryNode> node = Mockito.mock(QuerySegment.class);
-        //provider only cares about Ordered nodes.
-        Mockito.when(node.getOrderedNodes()).thenReturn(exprs);
-        return node;
-    }
-
-    private static List<QueryModelNode> getNodes(final String sparql) throws Exception {
-        final NodeCollector collector = new NodeCollector();
-        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(collector);
-        return collector.getTupleExpr();
-    }
-
-    private static class NodeCollector extends QueryModelVisitorBase<RuntimeException> {
-        private final List<QueryModelNode> stPatterns = new ArrayList<>();
-
-        public List<QueryModelNode> getTupleExpr() {
-            return stPatterns;
-        }
-
-        @Override
-        public void meet(final FunctionCall node) {
-            stPatterns.add(node);
-        }
-
-        @Override
-        public void meet(final StatementPattern node) {
-            stPatterns.add(node);
-        }
-    }
-
-    private static class FunctionCallCollector extends QueryModelVisitorBase<RuntimeException> {
-        private final List<FunctionCall> filters = new ArrayList<>();
-
-        public List<FunctionCall> getTupleExpr() {
-            return filters;
-        }
-
-        @Override
-        public void meet(final FunctionCall node) {
-            filters.add(node);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
new file mode 100644
index 0000000..23399c8
--- /dev/null
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/GeoTemporalTestUtils.java
@@ -0,0 +1,140 @@
+/*
+ * 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.geotemporal;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.rya.indexing.TemporalInstant;
+import org.apache.rya.indexing.TemporalInstantRfc3339;
+import org.apache.rya.indexing.external.matching.QuerySegment;
+import org.apache.rya.indexing.geotemporal.model.EventQueryNode;
+import org.junit.ComparisonFailure;
+import org.mockito.Mockito;
+import org.openrdf.query.algebra.FunctionCall;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.algebra.helpers.StatementPatternCollector;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import com.vividsolutions.jts.geom.Coordinate;
+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;
+
+public class GeoTemporalTestUtils {
+    private static final GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
+
+    /**
+     * Make an uniform instant with given seconds.
+     */
+    public static TemporalInstant makeInstant(final int secondsMakeMeUnique) {
+        return new TemporalInstantRfc3339(2015, 12, 30, 12, 00, secondsMakeMeUnique);
+    }
+
+    public 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;
+    }
+
+    public static Point point(final double x, final double y) {
+        return gf.createPoint(new Coordinate(x, y));
+    }
+
+    public 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);
+    }
+
+    public 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 static void assertEqualMongo(final Object expected, final Object actual) throws ComparisonFailure {
+        try {
+            assertEquals(expected, actual);
+        } catch(final Throwable e) {
+            throw new ComparisonFailure(e.getMessage(), expected.toString(), actual.toString());
+        }
+    }
+
+    public static List<FunctionCall> getFilters(final String query) throws Exception {
+        final FunctionCallCollector collector = new FunctionCallCollector();
+        new SPARQLParser().parseQuery(query, null).getTupleExpr().visit(collector);
+        return collector.getTupleExpr();
+    }
+
+    public static List<StatementPattern> getSps(final String query) throws Exception {
+        final StatementPatternCollector collector = new StatementPatternCollector();
+        new SPARQLParser().parseQuery(query, null).getTupleExpr().visit(collector);
+        return collector.getStatementPatterns();
+    }
+
+    public static QuerySegment<EventQueryNode> getQueryNode(final String query) throws Exception {
+        final List<QueryModelNode> exprs = getNodes(query);
+        final QuerySegment<EventQueryNode> node = Mockito.mock(QuerySegment.class);
+        //provider only cares about Ordered nodes.
+        Mockito.when(node.getOrderedNodes()).thenReturn(exprs);
+        return node;
+    }
+
+    private static List<QueryModelNode> getNodes(final String sparql) throws Exception {
+        final NodeCollector collector = new NodeCollector();
+        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(collector);
+        return collector.getTupleExpr();
+    }
+
+    private static class NodeCollector extends QueryModelVisitorBase<RuntimeException> {
+        private final List<QueryModelNode> stPatterns = new ArrayList<>();
+
+        public List<QueryModelNode> getTupleExpr() {
+            return stPatterns;
+        }
+
+        @Override
+        public void meet(final FunctionCall node) {
+            stPatterns.add(node);
+        }
+
+        @Override
+        public void meet(final StatementPattern node) {
+            stPatterns.add(node);
+        }
+    }
+
+    private static class FunctionCallCollector extends QueryModelVisitorBase<RuntimeException> {
+        private final List<FunctionCall> filters = new ArrayList<>();
+
+        public List<FunctionCall> getTupleExpr() {
+            return filters;
+        }
+
+        @Override
+        public void meet(final FunctionCall node) {
+            filters.add(node);
+        }
+    }
+}


[05/17] incubator-rya git commit: RYA-414 Removed mongo connection factory

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
index ff778ba..cff0996 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/MongoGeoTemporalIndexIT.java
@@ -26,21 +26,17 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.GeoRyaSailFactory;
 import org.apache.rya.indexing.TemporalInstantRfc3339;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.indexing.geotemporal.model.Event;
 import org.apache.rya.indexing.geotemporal.mongo.MongoGeoTemporalIndexer;
-import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-import org.junit.Before;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -54,123 +50,136 @@ import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
-import com.mongodb.MongoClient;
-
-public class MongoGeoTemporalIndexIT extends MongoITBase {
-    private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
-
-    private static final ValueFactory VF = ValueFactoryImpl.getInstance();
-    private MongoDBRdfConfiguration conf;
-    private SailRepositoryConnection conn;
-    private MongoClient mongoClient;
-    private static final AtomicInteger COUNTER = new AtomicInteger(1);
-
-    @Before
-    public void setUp() throws Exception{
-        mongoClient = super.getMongoClient();
-        conf = new MongoDBRdfConfiguration();
-        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, MongoGeoTemporalIndexIT.class.getSimpleName() + "_" + COUNTER.getAndIncrement());
-        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya");
-        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya");
-        conf.setBoolean(ConfigUtils.USE_MONGO, true);
-        conf.setBoolean(OptionalConfigUtils.USE_GEOTEMPORAL, true);
-        conf.setMongoClient(mongoClient);
-
-        final Sail sail = GeoRyaSailFactory.getInstance(conf);
-        conn = new SailRepository(sail).getConnection();
-        conn.begin();
-
-        addStatements();
-    }
-
-    @Test
-    public void ensureInEventStore_Test() throws Exception {
-        final MongoGeoTemporalIndexer indexer = new MongoGeoTemporalIndexer();
-        indexer.initIndexer(conf, mongoClient);
-
-        final EventStorage events = indexer.getEventStorage(conf);
-        final RyaURI subject = new RyaURI("urn:event1");
-        final Optional<Event> event = events.get(subject);
-        assertTrue(event.isPresent());
-    }
-
-    @Test
-    public void constantSubjQuery_Test() throws Exception {
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT * "
-              + "WHERE { "
-                + "  <urn:event1> time:atTime ?time . "
-                + "  <urn:event1> geo:asWKT ?point . "
-                + "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-              + "}";
-
-        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        final Set<BindingSet> results = new HashSet<>();
-        while(rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        final MapBindingSet expected = new MapBindingSet();
-        expected.addBinding("point", VF.createLiteral("POINT (0 0)"));
-        expected.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-        assertEquals(1, results.size());
-        assertEquals(expected, results.iterator().next());
-    }
-
-    @Test
-    public void variableSubjQuery_Test() throws Exception {
-        final String query =
-                "PREFIX time: <http://www.w3.org/2006/time#> \n"
-              + "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
-              + "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
-              + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
-              + "SELECT * "
-              + "WHERE { "
-                + "  ?subj time:atTime ?time . "
-                + "  ?subj geo:asWKT ?point . "
-                + "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
-                + "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
-              + "}";
-
-        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        final List<BindingSet> results = new ArrayList<>();
-        while(rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        final MapBindingSet expected1 = new MapBindingSet();
-        expected1.addBinding("point", VF.createLiteral("POINT (0 0)"));
-        expected1.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-        final MapBindingSet expected2 = new MapBindingSet();
-        expected2.addBinding("point", VF.createLiteral("POINT (1 1)"));
-        expected2.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
-
-        assertEquals(2, results.size());
-        assertEquals(expected1, results.get(0));
-        assertEquals(expected2, results.get(1));
-    }
-
-    private void addStatements() throws Exception {
-        URI subject = VF.createURI("urn:event1");
-        final URI predicate = VF.createURI(URI_PROPERTY_AT_TIME);
-        Value object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
-        conn.add(VF.createStatement(subject, predicate, object));
-
-        object = VF.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
-        conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
-
-        subject = VF.createURI("urn:event2");
-        object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
-        conn.add(VF.createStatement(subject, predicate, object));
-
-        object = VF.createLiteral("Point(1 1)", GeoConstants.XMLSCHEMA_OGC_WKT);
-        conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
-    }
+public class MongoGeoTemporalIndexIT extends MongoTestBase {
+	private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
+
+	private static final ValueFactory VF = ValueFactoryImpl.getInstance();
+	@Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+		//        mongoClient = super.getMongoClient();
+		//        conf = new MongoDBRdfConfiguration();
+		//        conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, MongoGeoTemporalIndexIT.class.getSimpleName() + "_" + COUNTER.getAndIncrement());
+		//        conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya");
+		//        conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya");
+		//        conf.setBoolean(ConfigUtils.USE_MONGO, true);
+		conf.setBoolean(OptionalConfigUtils.USE_GEOTEMPORAL, true);
+
+		//        conn = new SailRepository(sail).getConnection();
+		//        conn.begin();
+
+	}
+
+	@Test
+	public void ensureInEventStore_Test() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepository repo = new SailRepository(sail);
+		try(final MongoGeoTemporalIndexer indexer = new MongoGeoTemporalIndexer()) {
+			indexer.setConf(conf);
+			indexer.init();
+
+			addStatements(repo.getConnection());
+			final EventStorage events = indexer.getEventStorage();
+			final RyaURI subject = new RyaURI("urn:event1");
+			final Optional<Event> event = events.get(subject);
+			assertTrue(event.isPresent());
+		} finally {
+			sail.shutDown();
+		}
+	}
+
+	@Test
+	public void constantSubjQuery_Test() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+
+		try {
+			final String query =
+					"PREFIX time: <http://www.w3.org/2006/time#> \n"
+							+ "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+							+ "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+							+ "SELECT * "
+							+ "WHERE { "
+							+ "  <urn:event1> time:atTime ?time . "
+							+ "  <urn:event1> geo:asWKT ?point . "
+							+ "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+							+ "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+							+ "}";
+
+			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			final Set<BindingSet> results = new HashSet<>();
+			while(rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			final MapBindingSet expected = new MapBindingSet();
+			expected.addBinding("point", VF.createLiteral("POINT (0 0)"));
+			expected.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+			assertEquals(1, results.size());
+			assertEquals(expected, results.iterator().next());
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	@Test
+	public void variableSubjQuery_Test() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+
+		try {
+			final String query =
+					"PREFIX time: <http://www.w3.org/2006/time#> \n"
+							+ "PREFIX tempo: <tag:rya-rdf.org,2015:temporal#> \n"
+							+ "PREFIX geo: <http://www.opengis.net/ont/geosparql#>"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>"
+							+ "SELECT * "
+							+ "WHERE { "
+							+ "  ?subj time:atTime ?time . "
+							+ "  ?subj geo:asWKT ?point . "
+							+ "  FILTER(geof:sfWithin(?point, \"POLYGON((-3 -2, -3 2, 1 2, 1 -2, -3 -2))\"^^geo:wktLiteral)) "
+							+ "  FILTER(tempo:equals(?time, \"2015-12-30T12:00:00Z\")) "
+							+ "}";
+
+			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			final List<BindingSet> results = new ArrayList<>();
+			while(rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			final MapBindingSet expected1 = new MapBindingSet();
+			expected1.addBinding("point", VF.createLiteral("POINT (0 0)"));
+			expected1.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+			final MapBindingSet expected2 = new MapBindingSet();
+			expected2.addBinding("point", VF.createLiteral("POINT (1 1)"));
+			expected2.addBinding("time", VF.createLiteral("2015-12-30T12:00:00Z"));
+
+			assertEquals(2, results.size());
+			assertEquals(expected1, results.get(0));
+			assertEquals(expected2, results.get(1));
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	private void addStatements(final SailRepositoryConnection conn) throws Exception {
+		URI subject = VF.createURI("urn:event1");
+		final URI predicate = VF.createURI(URI_PROPERTY_AT_TIME);
+		Value object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
+		conn.add(VF.createStatement(subject, predicate, object));
+
+		object = VF.createLiteral("Point(0 0)", GeoConstants.XMLSCHEMA_OGC_WKT);
+		conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
+
+		subject = VF.createURI("urn:event2");
+		object = VF.createLiteral(new TemporalInstantRfc3339(2015, 12, 30, 12, 00, 0).toString());
+		conn.add(VF.createStatement(subject, predicate, object));
+
+		object = VF.createLiteral("Point(1 1)", GeoConstants.XMLSCHEMA_OGC_WKT);
+		conn.add(VF.createStatement(subject, GeoConstants.GEO_AS_WKT, object));
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
index f35eeb7..f0099c8 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/model/EventQueryNodeTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.indexing.geotemporal.model;
 
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getFilters;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.getSps;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -33,8 +35,8 @@ 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.mongo.MongoEventStorage;
-import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
@@ -60,7 +62,7 @@ import info.aduna.iteration.CloseableIteration;
 /**
  * Unit tests the methods of {@link EventQueryNode}.
  */
-public class EventQueryNodeTest extends MongoITBase {
+public class EventQueryNodeTest extends MongoTestBase {
     private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
index 4a31599..392d381 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/GeoTemporalMongoDBStorageStrategyTest.java
@@ -28,7 +28,7 @@ import org.apache.rya.indexing.IndexingFunctionRegistry;
 import org.apache.rya.indexing.IndexingFunctionRegistry.FUNCTION_TYPE;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.GeoPolicy;
 import org.apache.rya.indexing.geotemporal.GeoTemporalIndexer.TemporalPolicy;
-import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
+import org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -56,7 +56,7 @@ import com.mongodb.util.JSON;
  * @see TemporalPolicy Temporal Filter Functions
  * @see GeoPolicy Geo Filter Functions
  */
-public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestBase {
+public class GeoTemporalMongoDBStorageStrategyTest extends GeoTemporalTestUtils {
     private GeoTemporalMongoDBStorageStrategy adapter;
     @Before
     public void setup() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
index 5b07460..6bd0b6d 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoEventStorageTest.java
@@ -31,6 +31,7 @@ import org.apache.rya.indexing.geotemporal.model.Event;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventAlreadyExistsException;
 import org.apache.rya.indexing.geotemporal.storage.EventStorage.EventStorageException;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.joda.time.DateTime;
 import org.junit.Test;
 
@@ -42,7 +43,7 @@ import com.vividsolutions.jts.geom.PrecisionModel;
 /**
  * Integration tests the methods of {@link MongoEventStorage}.
  */
-public class MongoEventStorageTest extends MongoITBase {
+public class MongoEventStorageTest extends MongoTestBase {
 
     private static final String RYA_INSTANCE_NAME = "testInstance";
     private static final GeometryFactory GF = new GeometryFactory(new PrecisionModel(), 4326);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
index f2d0868..9bfe148 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoGeoTemporalIndexerIT.java
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.indexing.geotemporal.mongo;
 
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.makeInstant;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.point;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -29,6 +31,7 @@ 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.storage.EventStorage;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
@@ -43,7 +46,7 @@ import com.vividsolutions.jts.geom.Geometry;
 /**
  * Integration tests the methods of {@link MongoGeoTemporalIndexer}.
  */
-public class MongoGeoTemporalIndexerIT extends MongoITBase {
+public class MongoGeoTemporalIndexerIT extends MongoTestBase {
     private MongoGeoTemporalIndexer indexer;
 
     @Before
@@ -58,7 +61,7 @@ public class MongoGeoTemporalIndexerIT extends MongoITBase {
         final RyaStatement geoStmnt = statement(point(0, 0));
         final RyaStatement timeStmnt = statement(makeInstant(0));
 
-        final EventStorage store = indexer.getEventStorage(conf);
+        final EventStorage store = indexer.getEventStorage();
 
         indexer.storeStatement(geoStmnt);
         Optional<Event> evnt = store.get(geoStmnt.getSubject());

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
deleted file mode 100644
index 2ca2780..0000000
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/geotemporal/mongo/MongoITBase.java
+++ /dev/null
@@ -1,64 +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.geotemporal.mongo;
-
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.geotemporal.GeoTemporalTestBase;
-import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
-import org.apache.rya.mongodb.EmbeddedMongoSingleton;
-import org.junit.After;
-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 extends GeoTemporalTestBase {
-
-    private static MongoClient mongoClient = null;
-    protected static MongoIndexingConfiguration conf;
-
-    @Before
-    public void setupTest() throws Exception {
-        mongoClient = EmbeddedMongoSingleton.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()) {
-            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/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
index 81b45d1..4ca6b4c 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.resolver.RdfToRyaConversions;
@@ -31,10 +30,8 @@ import org.apache.rya.indexing.GeoConstants;
 import org.apache.rya.indexing.GeoRyaSailFactory;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
-import org.apache.rya.indexing.geotemporal.mongo.MongoITBase;
-import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -48,12 +45,10 @@ import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.QueryEvaluationException;
 import org.openrdf.query.QueryLanguage;
 import org.openrdf.query.TupleQueryResult;
-import org.openrdf.repository.RepositoryException;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
 import org.openrdf.sail.Sail;
 
-import com.mongodb.MongoClient;
 import com.vividsolutions.jts.geom.Coordinate;
 import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
@@ -61,238 +56,242 @@ import com.vividsolutions.jts.io.ParseException;
 import com.vividsolutions.jts.io.WKTReader;
 import com.vividsolutions.jts.io.WKTWriter;
 
-public class MongoGeoIndexerFilterIT extends MongoITBase {
-    private static final GeometryFactory GF = new GeometryFactory();
-    private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
-    private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));
-    private static final Geometry CAPITAL_BUILDING = GF.createPoint(new Coordinate(38.8899, 77.0091));
-    private static final Geometry WHITE_HOUSE = GF.createPoint(new Coordinate(38.8977, 77.0365));
-
-    private static final AtomicInteger COUNTER = new AtomicInteger(1);
-
-    private MongoClient client;
-    private Sail sail;
-    private SailRepositoryConnection conn;
-    @Before
-    public void before() throws Exception {
-        final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
-            .setMongoCollectionPrefix("rya_")
-            .setMongoDBName(MongoGeoIndexerFilterIT.class.getSimpleName() + "_" + COUNTER.getAndIncrement())
-            .setUseMongoFreetextIndex(false)
-            .setUseMongoTemporalIndex(false)
-            .build();
-
-        client = super.getMongoClient();
-        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);
-        indxrConf.setMongoClient(client);
-
-        sail = GeoRyaSailFactory.getInstance(indxrConf);
-        conn = new SailRepository(sail).getConnection();
-    }
-
-    @After
-    public void after() throws RepositoryException {
-        if(conn != null) {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void nearHappyUsesTest() throws Exception {
-        populateRya();
-
-        //Only captial
-        String query =
-             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-           + "SELECT * \n" //
-           + "WHERE { \n"
-           + "  <urn:geo> geo:asWKT ?point .\n"
-           + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 0.0, 2000))"
-           + "}";
-
-        TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        final List<BindingSet> results = new ArrayList<>();
-        while (rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        assertEquals(1, results.size());
-        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(0)));
-
-        //all but capital
-        query =
-            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-          + "SELECT * \n" //
-          + "WHERE { \n"
-          + "  <urn:geo> geo:asWKT ?point .\n"
-          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000))"
-          + "}";
-
-        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        results.clear();
-        while (rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        assertEquals(3, results.size());
-        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-
-        // all of them
-        query =
-            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-          + "SELECT * \n" //
-          + "WHERE { \n"
-          + "  <urn:geo> geo:asWKT ?point .\n"
-          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 6000, 000))"
-          + "}";
-
-        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        results.clear();
-        while (rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        assertEquals(4, results.size());
-        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
-
-        // donut, only 2
-        query =
-            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-          + "SELECT * \n" //
-          + "WHERE { \n"
-          + "  <urn:geo> geo:asWKT ?point .\n"
-          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000, 100))"
-          + "}";
-
-        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        results.clear();
-        while (rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        assertEquals(2, results.size());
-        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(0)));
-        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(1)));
-
-        // all of them
-        query =
-            "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-          + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-          + "SELECT * \n" //
-          + "WHERE { \n"
-          + "  <urn:geo> geo:asWKT ?point .\n"
-          + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral))"
-          + "}";
-        rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        results.clear();
-        while (rez.hasNext()) {
-            final BindingSet bs = rez.next();
-            results.add(bs);
-        }
-        assertEquals(4, results.size());
-        assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-        assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-        assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-        assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
-    }
-
-    @Test(expected = MalformedQueryException.class)
-    public void near_invalidDistance() throws Exception {
-        populateRya();
-
-        //Only captial
-        final String query =
-             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-           + "SELECT * \n" //
-           + "WHERE { \n"
-           + "  <urn:geo> geo:asWKT ?point .\n"
-                + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, distance))"
-           + "}";
-
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void near_negativeDistance() throws Exception {
-        populateRya();
-
-        //Only captial
-        final String query =
-             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-           + "SELECT * \n" //
-           + "WHERE { \n"
-           + "  <urn:geo> geo:asWKT ?point .\n"
-           + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, -100))"
-           + "}";
-
-        final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-        while(rez.hasNext()) {
-            rez.next();
-        }
-    }
-
-    @Test(expected = QueryEvaluationException.class)
-    public void tooManyArgumentsTest() throws Exception {
-        populateRya();
-
-        // Only captial
-        final String query =
-              "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-            + "SELECT * \n" //
-            + "WHERE { \n" + "  <urn:geo> geo:asWKT ?point .\n"
-            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 100, 1000, 10))"
-            + "}";
-
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-    }
-
-    private void populateRya() throws Exception {
-        // geo 2x2 points
-        conn.begin();
-        RyaStatement stmnt = statement(WASHINGTON_MONUMENT);
-        Statement statement = RyaToRdfConversions.convertStatement(stmnt);
-        conn.add(statement);
-
-        stmnt = statement(LINCOLN_MEMORIAL);
-        statement = RyaToRdfConversions.convertStatement(stmnt);
-        conn.add(statement);
-
-        stmnt = statement(CAPITAL_BUILDING);
-        statement = RyaToRdfConversions.convertStatement(stmnt);
-        conn.add(statement);
-
-        stmnt = statement(WHITE_HOUSE);
-        statement = RyaToRdfConversions.convertStatement(stmnt);
-        conn.add(statement);
-        conn.commit();
-    }
-
-    private static Geometry bindingToGeo(final BindingSet bs) throws ParseException {
-        final WKTReader w = new WKTReader();
-        return w.read(bs.getValue("point").stringValue());
-    }
-
-    private static RyaStatement statement(final Geometry geo) {
-        final ValueFactory vf = new ValueFactoryImpl();
-        final Resource subject = vf.createURI("urn:geo");
-        final URI predicate = GeoConstants.GEO_AS_WKT;
-        final WKTWriter w = new WKTWriter();
-        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
-        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
-    }
+public class MongoGeoIndexerFilterIT extends MongoTestBase {
+	private static final GeometryFactory GF = new GeometryFactory();
+	private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
+	private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));
+	private static final Geometry CAPITAL_BUILDING = GF.createPoint(new Coordinate(38.8899, 77.0091));
+	private static final Geometry WHITE_HOUSE = GF.createPoint(new Coordinate(38.8977, 77.0365));
+
+	@Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+		conf.setBoolean(OptionalConfigUtils.USE_GEO, true);
+		conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+		conf.setBoolean(ConfigUtils.USE_MONGO, true);
+	}
+
+	@Test
+	public void nearHappyUsesTest() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+		try {
+			populateRya(conn);
+
+			//Only captial
+			String query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 0.0, 2000))"
+							+ "}";
+
+			TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			final List<BindingSet> results = new ArrayList<>();
+			while (rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			assertEquals(1, results.size());
+			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(0)));
+
+			//all but capital
+			query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000))"
+							+ "}";
+
+			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			results.clear();
+			while (rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			assertEquals(3, results.size());
+			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+
+			// all of them
+			query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 6000, 000))"
+							+ "}";
+
+			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			results.clear();
+			while (rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			assertEquals(4, results.size());
+			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+
+			// donut, only 2
+			query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000, 100))"
+							+ "}";
+
+			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			results.clear();
+			while (rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			assertEquals(2, results.size());
+			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(0)));
+			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(1)));
+
+			// all of them
+			query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral))"
+							+ "}";
+			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			results.clear();
+			while (rez.hasNext()) {
+				final BindingSet bs = rez.next();
+				results.add(bs);
+			}
+			assertEquals(4, results.size());
+			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	@Test(expected = MalformedQueryException.class)
+	public void near_invalidDistance() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+		try {
+			populateRya(conn);
+
+			//Only captial
+			final String query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, distance))"
+							+ "}";
+
+			conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void near_negativeDistance() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+		try {
+			populateRya(conn);
+
+			//Only captial
+			final String query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n"
+							+ "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, -100))"
+							+ "}";
+
+			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+			while(rez.hasNext()) {
+				rez.next();
+			}
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	@Test(expected = QueryEvaluationException.class)
+	public void tooManyArgumentsTest() throws Exception {
+		final Sail sail = GeoRyaSailFactory.getInstance(conf);
+		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+		try {
+			populateRya(conn);
+
+			// Only captial
+			final String query =
+					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+							+ "SELECT * \n" //
+							+ "WHERE { \n" + "  <urn:geo> geo:asWKT ?point .\n"
+							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 100, 1000, 10))"
+							+ "}";
+
+			conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+		} finally {
+			conn.close();
+			sail.shutDown();
+		}
+	}
+
+	private void populateRya(final SailRepositoryConnection conn) throws Exception {
+		// geo 2x2 points
+		conn.begin();
+		RyaStatement stmnt = statement(WASHINGTON_MONUMENT);
+		Statement statement = RyaToRdfConversions.convertStatement(stmnt);
+		conn.add(statement);
+
+		stmnt = statement(LINCOLN_MEMORIAL);
+		statement = RyaToRdfConversions.convertStatement(stmnt);
+		conn.add(statement);
+
+		stmnt = statement(CAPITAL_BUILDING);
+		statement = RyaToRdfConversions.convertStatement(stmnt);
+		conn.add(statement);
+
+		stmnt = statement(WHITE_HOUSE);
+		statement = RyaToRdfConversions.convertStatement(stmnt);
+		conn.add(statement);
+		conn.commit();
+	}
+
+	private static Geometry bindingToGeo(final BindingSet bs) throws ParseException {
+		final WKTReader w = new WKTReader();
+		return w.read(bs.getValue("point").stringValue());
+	}
+
+	private static RyaStatement statement(final Geometry geo) {
+		final ValueFactory vf = new ValueFactoryImpl();
+		final Resource subject = vf.createURI("urn:geo");
+		final URI predicate = GeoConstants.GEO_AS_WKT;
+		final WKTWriter w = new WKTWriter();
+		final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+		return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+	}
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
index 57873fd..2d09a6c 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
@@ -19,6 +19,10 @@ package org.apache.rya.indexing.mongo;
  */
 
 import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.bbox;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.line;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.point;
+import static org.apache.rya.indexing.geotemporal.GeoTemporalTestUtils.poly;
 
 import java.util.Map;
 import java.util.Set;
@@ -29,11 +33,11 @@ import org.apache.rya.api.resolver.RyaToRdfConversions;
 import org.apache.rya.indexing.GeoConstants;
 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.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -46,21 +50,16 @@ import org.openrdf.model.impl.ValueFactoryImpl;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.vividsolutions.jts.geom.Geometry;
-import com.vividsolutions.jts.geom.GeometryFactory;
 import com.vividsolutions.jts.geom.LineString;
 import com.vividsolutions.jts.geom.Point;
 import com.vividsolutions.jts.geom.Polygon;
-import com.vividsolutions.jts.geom.PrecisionModel;
 
 import info.aduna.iteration.CloseableIteration;
 
 /**
  * Tests all of the "simple functions" of the geoindexer.
  */
-public class MongoGeoIndexerSfTest extends MongoITBase {
-    private static GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
-    private static MongoGeoIndexer g;
-
+public class MongoGeoIndexerSfTest extends MongoTestBase {
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
 
     // Here is the landscape:
@@ -99,19 +98,10 @@ public class MongoGeoIndexerSfTest extends MongoITBase {
         names.put(F, "F");
     }
 
-    @Before
-    public void before() throws Exception {
+    @Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         conf.set(OptionalConfigUtils.USE_GEO, "true");
-
-        g = new MongoGeoIndexer();
-        g.initIndexer(conf, super.getMongoClient());
-        g.storeStatement(statement(A));
-        g.storeStatement(statement(B));
-        g.storeStatement(statement(C));
-        g.storeStatement(statement(D));
-        g.storeStatement(statement(F));
-        g.storeStatement(statement(E));
     }
 
     private static RyaStatement statement(final Geometry geo) {
@@ -136,18 +126,28 @@ public class MongoGeoIndexerSfTest extends MongoITBase {
 
     @Test
     public void testEquals() throws Exception {
-        // point
-        compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
-        compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-        // line
-        compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
-        compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-        // poly
-        compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
-        compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
+    	try(final MongoGeoIndexer g = new MongoGeoIndexer()) {
+    		g.setConf(conf);
+    		g.init();
+
+    		g.storeStatement(statement(A));
+    		g.storeStatement(statement(B));
+    		g.storeStatement(statement(C));
+    		g.storeStatement(statement(D));
+    		g.storeStatement(statement(F));
+    		g.storeStatement(statement(E));
+    		// point
+    		compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
+    		compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+    		// line
+    		compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
+    		compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+    		// poly
+    		compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
+    		compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+    	}
     }
 
 //    @Test
@@ -163,19 +163,19 @@ public class MongoGeoIndexerSfTest extends MongoITBase {
 //        compare(g.queryDisjoint(B, EMPTY_CONSTRAINTS), C, D, F, E);
 //    }
 
-    @Test
-    public void testIntersectsPoint() throws Exception {
+//    @Test
+//    public void testIntersectsPoint() throws Exception {
         // This seems like a bug
         // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), A, F);
         // compare(g.queryIntersects(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
-
-    @Test
-    public void testIntersectsLine() throws Exception {
+//    }
+//
+//    @Test
+//    public void testIntersectsLine() throws Exception {
         // This seems like a bug
         // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), A, E);
         // compare(g.queryIntersects(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
+//    }
 
 //    @Test
 //    public void testIntersectsPoly() throws Exception {
@@ -202,10 +202,10 @@ public class MongoGeoIndexerSfTest extends MongoITBase {
 //        compare(g.queryCrosses(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
 //    }
 
-    @Test
-    public void testCrossesLine() throws Exception {
+//    @Test
+//    public void testCrossesLine() throws Exception {
         // compare(g.queryCrosses(E, EMPTY_CONSTRAINTS), A);
-    }
+//    }
 
 //    @Test
 //    public void testCrossesPoly() throws Exception {
@@ -229,30 +229,30 @@ public class MongoGeoIndexerSfTest extends MongoITBase {
 //        compare(g.queryContains(F, EMPTY_CONSTRAINTS), A, F);
 //    }
 
-    @Test
-    public void testContainsLine() throws Exception {
+//    @Test
+//    public void testContainsLine() throws Exception {
         // compare(g.queryContains(E, EMPTY_CONSTRAINTS), E);
-    }
+//    }
 
 //    @Test
 //    public void testContainsPoly() throws Exception {
 //        compare(g.queryContains(A, EMPTY_CONSTRAINTS), A);
 //        compare(g.queryContains(B, EMPTY_CONSTRAINTS), A, B);
 //    }
-
-    @Test
-    public void testOverlapsPoint() throws Exception {
+//
+//    @Test
+//    public void testOverlapsPoint() throws Exception {
         // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), F);
         // You cannot have overlapping points
         // compare(g.queryOverlaps(F, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
+//    }
 
-    @Test
-    public void testOverlapsLine() throws Exception {
+//    @Test
+//    public void testOverlapsLine() throws Exception {
         // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), A, E);
         // You cannot have overlapping lines
         // compare(g.queryOverlaps(E, EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    }
+//    }
 
 //    @Test
 //    public void testOverlapsPoly() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
index 93cabc4..0e0fff2 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
@@ -1,6 +1,4 @@
-package org.apache.rya.indexing.mongo;
-
-/*
+/**
  * 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
@@ -18,11 +16,12 @@ package org.apache.rya.indexing.mongo;
  * specific language governing permissions and limitations
  * under the License.
  */
-
-
+package org.apache.rya.indexing.mongo;
 
 import static org.apache.rya.api.resolver.RdfToRyaConversions.convertStatement;
 import static org.apache.rya.indexing.GeoIndexingTestUtils.getSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.util.Collections;
 import java.util.Set;
@@ -30,11 +29,10 @@ import java.util.Set;
 import org.apache.rya.indexing.GeoConstants;
 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.accumulo.geo.OptionalConfigUtils;
 import org.apache.rya.indexing.mongodb.geo.MongoGeoIndexer;
-import org.junit.Assert;
-import org.junit.Before;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -54,21 +52,22 @@ import com.vividsolutions.jts.geom.Polygon;
 import com.vividsolutions.jts.geom.PrecisionModel;
 import com.vividsolutions.jts.geom.impl.PackedCoordinateSequence;
 
-public class MongoGeoIndexerTest extends MongoITBase {
+public class MongoGeoIndexerTest extends MongoTestBase {
     private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
     GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
 
-    @Before
-    public void before() throws Exception {
+    @Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         conf.set(OptionalConfigUtils.USE_GEO, "true");
     }
 
     @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, super.getMongoClient());
+        	conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
 
@@ -98,16 +97,17 @@ public class MongoGeoIndexerTest extends MongoITBase {
             f.flush();
 
             final Set<Statement> actual = getSet(f.queryEquals(point, EMPTY_CONSTRAINTS));
-            Assert.assertEquals(2, actual.size());
-            Assert.assertTrue(actual.contains(s3));
-            Assert.assertTrue(actual.contains(s4));
+            assertEquals(2, actual.size());
+            assertTrue(actual.contains(s3));
+            assertTrue(actual.contains(s4));
         }
     }
 
     @Test
     public void testPrimeMeridianSearch() throws Exception {
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -131,19 +131,19 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final Polygon p2 = gf.createPolygon(r2, new LinearRing[] {});
             final Polygon p3 = gf.createPolygon(r3, new LinearRing[] {});
 
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p2, EMPTY_CONSTRAINTS)));
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p3, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p2, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p3, EMPTY_CONSTRAINTS)));
 
             // Test a ring with a hole in it
             final Polygon p3m2 = gf.createPolygon(r3, new LinearRing[] { r2 });
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p3m2, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p3m2, EMPTY_CONSTRAINTS)));
 
             // test a ring outside the point
             final double[] OUT = { 3, 3, 1, 3, 1, 1, 3, 1, 3, 3 };
             final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
             final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
         }
     }
 
@@ -151,7 +151,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDcSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -166,13 +167,13 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final double[] IN = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
             final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(IN, 2));
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
 
             // test a ring outside the point
             final double[] OUT = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
             final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(OUT, 2));
             final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
         }
     }
 
@@ -180,7 +181,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDeleteSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -198,20 +200,20 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final double[] in = { -78, 39, -77, 39, -77, 38, -78, 38, -78, 39 };
             final LinearRing r1 = gf.createLinearRing(new PackedCoordinateSequence.Double(in, 2));
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
 
             // test a ring that the point would be outside of if not deleted
             final double[] out = { -77, 39, -76, 39, -76, 38, -77, 38, -77, 39 };
             final LinearRing rOut = gf.createLinearRing(new PackedCoordinateSequence.Double(out, 2));
             final Polygon pOut = gf.createPolygon(rOut, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pOut, EMPTY_CONSTRAINTS)));
 
             // test a ring for the whole world and make sure the point is gone
             // Geomesa is a little sensitive around lon 180, so we only go to 179
             final double[] world = { -180, 90, 179, 90, 179, -90, -180, -90, -180, 90 };
             final LinearRing rWorld = gf.createLinearRing(new PackedCoordinateSequence.Double(world, 2));
             final Polygon pWorld = gf.createPolygon(rWorld, new LinearRing[] {});
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pWorld, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(pWorld, EMPTY_CONSTRAINTS)));
         }
     }
 
@@ -219,7 +221,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDcSearchWithContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -236,10 +239,10 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
 
             // query with correct context
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setContext(context))));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setContext(context))));
 
             // query with wrong context
-            Assert.assertEquals(Sets.newHashSet(),
+            assertEquals(Sets.newHashSet(),
                     getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
         }
     }
@@ -248,7 +251,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDcSearchWithSubject() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -265,10 +269,10 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
 
             // query with correct subject
-            Assert.assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(subject))));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(subject))));
 
             // query with wrong subject
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
         }
     }
 
@@ -276,7 +280,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDcSearchWithSubjectAndContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -293,15 +298,15 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
 
             // query with correct context subject
-            Assert.assertEquals(Sets.newHashSet(statement),
+            assertEquals(Sets.newHashSet(statement),
                     getSet(f.queryWithin(p1, new StatementConstraints().setContext(context).setSubject(subject))));
 
             // query with wrong context
-            Assert.assertEquals(Sets.newHashSet(),
+            assertEquals(Sets.newHashSet(),
                     getSet(f.queryWithin(p1, new StatementConstraints().setContext(vf.createURI("foo:context2")))));
 
             // query with wrong subject
-            Assert.assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
+            assertEquals(Sets.newHashSet(), getSet(f.queryWithin(p1, new StatementConstraints().setSubject(vf.createURI("foo:subj2")))));
         }
     }
 
@@ -309,7 +314,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     public void testDcSearchWithPredicate() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-            f.initIndexer(conf, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource subject = vf.createURI("foo:subj");
@@ -326,11 +332,11 @@ public class MongoGeoIndexerTest extends MongoITBase {
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
 
             // query with correct Predicate
-            Assert.assertEquals(Sets.newHashSet(statement),
+            assertEquals(Sets.newHashSet(statement),
                     getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(predicate)))));
 
             // query with wrong predicate
-            Assert.assertEquals(Sets.newHashSet(),
+            assertEquals(Sets.newHashSet(),
                     getSet(f.queryWithin(p1, new StatementConstraints().setPredicates(Collections.singleton(vf.createURI("other:pred"))))));
         }
     }
@@ -339,7 +345,8 @@ public class MongoGeoIndexerTest extends MongoITBase {
     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, super.getMongoClient());
+        	f.setConf(conf);
+            f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
             final Resource context = vf.createURI("foo:context");
@@ -364,7 +371,7 @@ public class MongoGeoIndexerTest extends MongoITBase {
 
             final Polygon p1 = gf.createPolygon(r1, new LinearRing[] {});
 
-            Assert.assertEquals(Sets.newHashSet(statementEast, statementWest), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statementEast, statementWest), getSet(f.queryWithin(p1, EMPTY_CONSTRAINTS)));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
index 5751887..cbb53e5 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
@@ -29,10 +29,8 @@ import org.apache.rya.indexing.TemporalInstant;
 import org.apache.rya.indexing.TemporalInstantRfc3339;
 import org.apache.rya.indexing.accumulo.ConfigUtils;
 import org.apache.rya.indexing.accumulo.geo.OptionalConfigUtils;
-import org.apache.rya.indexing.mongodb.MongoIndexingConfiguration;
-import org.apache.rya.mongodb.EmbeddedMongoSingleton;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoTestBase;
 import org.junit.Test;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -55,89 +53,75 @@ import com.vividsolutions.jts.geom.Geometry;
 import com.vividsolutions.jts.geom.GeometryFactory;
 import com.vividsolutions.jts.io.WKTWriter;
 
-public class MongoIndexerDeleteIT {
-    private MongoClient client;
-    private Sail sail;
-    private SailRepositoryConnection conn;
-
-    @Before
-    public void before() throws Exception {
-        final MongoIndexingConfiguration indxrConf = MongoIndexingConfiguration.builder()
-                .setMongoCollectionPrefix("rya_").setMongoDBName("indexerTests")
-            .setUseMongoFreetextIndex(true)
-            .setUseMongoTemporalIndex(true)
-            .setMongoFreeTextPredicates(RDFS.LABEL.stringValue())
-            .setMongoTemporalPredicates("Property:atTime")
-            .build();
-
-        client = EmbeddedMongoSingleton.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);
-        indxrConf.setMongoClient(client);
-
-        sail = GeoRyaSailFactory.getInstance(indxrConf);
-        conn = new SailRepository(sail).getConnection();
-        conn.begin();
-    }
-
-    @After
-    public void after() throws Exception {
-        if (conn != null) {
-            conn.clear();
-        }
+public class MongoIndexerDeleteIT extends MongoTestBase {
+    @Override
+	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, new String[] {RDFS.LABEL.stringValue()});
+        conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, new String[] {"Property:atTime"});
+        conf.setBoolean(ConfigUtils.USE_FREETEXT, true);
+        conf.setBoolean(ConfigUtils.USE_TEMPORAL, true);
+        conf.setBoolean(OptionalConfigUtils.USE_GEO, true);
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
     }
 
     @Test
     public void deleteTest() throws Exception {
-        populateRya();
-
-        //The extra 1 is from the person type defined in freetext
-        assertEquals(8, client.getDatabase("indexerTests").getCollection("rya__triples").count());
-        assertEquals(4, client.getDatabase("indexerTests").getCollection("rya_rya_geo").count());
-        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
-        assertEquals(2, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
-
-        //free text -- remove one from many
-        String delete = "DELETE DATA \n" //
-           + "{\n"
-           + "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
-           + "}";
-        Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-        update.execute();
-
-        // temporal -- remove one from one
-        delete = "DELETE DATA \n" //
-           + "{\n"
-           + "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
-           + "}";
-
-        update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-        update.execute();
-
-        //geo -- remove many from many
-        delete =
-             "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-           + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-           + "DELETE \n" //
-           + "{\n"
-           + "  <urn:geo> geo:asWKT ?point \n"
-           + "}"
-           + "WHERE { \n"
-           + "  <urn:geo> geo:asWKT ?point .\n"
-           + "  FILTER(geof:sfWithin(?point, \"POLYGON((0 0, 2 0, 2 1, 0 1, 0 0))\"^^geo:wktLiteral))"
-           + "}";
-
-        update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-        update.execute();
-
-        assertEquals(2, client.getDatabase("indexerTests").getCollection("rya_rya_geo").count());
-        assertEquals(0, client.getDatabase("indexerTests").getCollection("rya_rya_temporal").count());
-        assertEquals(1, client.getDatabase("indexerTests").getCollection("rya_rya_freetext").count());
-        assertEquals(4, client.getDatabase("indexerTests").getCollection("rya__triples").count());
+    	final Sail sail = GeoRyaSailFactory.getInstance(conf);
+    	final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+    	try {
+    		populateRya(conn);
+    		final MongoClient client = conf.getMongoClient();
+    		
+    		//The extra 1 is from the person type defined in freetext
+    		assertEquals(8, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
+    		assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
+    		assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
+    		assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
+
+    		//free text -- remove one from many
+    		String delete = "DELETE DATA \n" //
+    				+ "{\n"
+    				+ "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
+    				+ "}";
+    		Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+    		update.execute();
+
+    		// temporal -- remove one from one
+    		delete = "DELETE DATA \n" //
+    				+ "{\n"
+    				+ "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
+    				+ "}";
+
+    		update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+    		update.execute();
+
+    		//geo -- remove many from many
+    		delete =
+    				"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+    						+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+    						+ "DELETE \n" //
+    						+ "{\n"
+    						+ "  <urn:geo> geo:asWKT ?point \n"
+    						+ "}"
+    						+ "WHERE { \n"
+    						+ "  <urn:geo> geo:asWKT ?point .\n"
+    						+ "  FILTER(geof:sfWithin(?point, \"POLYGON((0 0, 2 0, 2 1, 0 1, 0 0))\"^^geo:wktLiteral))"
+    						+ "}";
+
+    		update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+    		update.execute();
+
+    		assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
+    		assertEquals(0, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
+    		assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
+    		assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
+    	} finally {
+    		conn.close();
+    		sail.shutDown();
+    	}
     }
 
-    private void populateRya() throws Exception {
+    private void populateRya(final SailRepositoryConnection conn) throws Exception {
         final ValueFactory VF = new ValueFactoryImpl();
         // geo 2x2 points
         final GeometryFactory GF = new GeometryFactory();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
index d102bfc..e401589 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStore.java
@@ -2,12 +2,6 @@ package org.apache.rya.rdftriplestore;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.sail.SailConnection;
-import org.openrdf.sail.SailException;
-import org.openrdf.sail.helpers.SailBase;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -37,6 +31,11 @@ import org.apache.rya.api.persist.joinselect.SelectivityEvalDAO;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
 import org.apache.rya.rdftriplestore.namespace.NamespaceManager;
 import org.apache.rya.rdftriplestore.provenance.ProvenanceCollector;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.sail.SailConnection;
+import org.openrdf.sail.SailException;
+import org.openrdf.sail.helpers.SailBase;
 
 public class RdfCloudTripleStore extends SailBase {
 
@@ -53,7 +52,7 @@ public class RdfCloudTripleStore extends SailBase {
 
     @Override
     protected SailConnection getConnectionInternal() throws SailException {
-        return new RdfCloudTripleStoreConnection(this, conf.clone(), vf);
+        return new RdfCloudTripleStoreConnection(this, conf, vf);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/17cebae3/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
index 921acaa..eaf3033 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
@@ -284,8 +284,8 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
         }
 
         try {
-            final List<Class<QueryOptimizer>> optimizers = queryConf.getOptimizers();
-            final Class<QueryOptimizer> pcjOptimizer = queryConf.getPcjOptimizer();
+            final List<Class<QueryOptimizer>> optimizers = conf.getOptimizers();
+            final Class<QueryOptimizer> pcjOptimizer = conf.getPcjOptimizer();
 
             if(pcjOptimizer != null) {
                 QueryOptimizer opt = null;
@@ -304,7 +304,7 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
             }
 
             final ParallelEvaluationStrategyImpl strategy = new ParallelEvaluationStrategyImpl(
-                    new StoreTripleSource(queryConf, ryaDAO), inferenceEngine, dataset, queryConf);
+                    new StoreTripleSource(conf, ryaDAO), inferenceEngine, dataset, queryConf);
 
                 (new BindingAssigner()).optimize(tupleExpr, dataset, bindings);
                 (new ConstantOptimizer(strategy)).optimize(tupleExpr, dataset,


[03/17] incubator-rya git commit: RYA-414 Introduced the stateful mongo configuratino object so that it is the arbitor of MongoDB state within a Sail object.

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 a014e8f..7043870 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
@@ -1,5 +1,4 @@
-package org.apache.rya.mongodb;
-/*
+/**
  * 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
@@ -17,6 +16,7 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.mongodb;
 
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
 import static org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy.TIMESTAMP;
@@ -36,461 +36,484 @@ import org.apache.rya.mongodb.document.util.AuthorizationsUtil;
 import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
 import org.bson.Document;
 import org.calrissian.mango.collect.CloseableIterable;
-import org.junit.AfterClass;
-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 MongoTestBase {
-    private MongoClient client;
-    private static MongoDBRyaDAO dao;
 
-    @Before
-    public void setUp() throws IOException, RyaDAOException{
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.setAuths("A", "B", "C");
-        client = super.getMongoClient();
-        dao = new MongoDBRyaDAO(conf, client);
-    }
-
-    @AfterClass
-    public static void tearDown() throws RyaDAOException {
-        dao.destroy();
     }
 
     @Test
     public void testDeleteWildcard() throws RyaDAOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setColumnVisibility(new DocumentVisibility("A").flatten());
-        dao.delete(builder.build(), conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setColumnVisibility(new DocumentVisibility("A").flatten());
+            dao.delete(builder.build(), conf);
+        } finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testAdd() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
-        builder.setColumnVisibility(new DocumentVisibility("B").flatten());
-
-        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
-
-        dao.add(builder.build());
-
-        assertEquals(coll.count(),1);
-
-        final Document dbo = coll.find().first();
-        assertTrue(dbo.containsKey(DOCUMENT_VISIBILITY));
-        assertTrue(dbo.containsKey(TIMESTAMP));
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            builder.setColumnVisibility(new DocumentVisibility("B").flatten());
+
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(builder.build());
+
+            assertEquals(coll.count(),1);
+
+            final Document dbo = coll.find().first();
+            assertTrue(dbo.containsKey(DOCUMENT_VISIBILITY));
+            assertTrue(dbo.containsKey(TIMESTAMP));
+        }  finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testDelete() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
-        builder.setColumnVisibility(new DocumentVisibility("C").flatten());
-        final RyaStatement statement = builder.build();
-        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
-
-        dao.add(statement);
-
-        assertEquals(1, coll.count());
-
-        dao.delete(statement, conf);
-
-        assertEquals(0, coll.count());
-
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            builder.setColumnVisibility(new DocumentVisibility("C").flatten());
+            final RyaStatement statement = builder.build();
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(statement);
+            assertEquals(1, coll.count());
+
+            dao.delete(statement, conf);
+            assertEquals(0, coll.count());
+        } finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
-        builder.setContext(new RyaURI("http://context.com"));
-        builder.setColumnVisibility(new DocumentVisibility("A&B&C").flatten());
-        final RyaStatement statement = builder.build();
-
-        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
-
-        dao.add(statement);
-
-        assertEquals(1, coll.count());
-
-        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
-        builder2.setPredicate(new RyaURI("http://temp.com"));
-        builder2.setObject(new RyaURI("http://object.com"));
-        builder2.setContext(new RyaURI("http://context3.com"));
-        final RyaStatement query = builder2.build();
-
-        dao.delete(query, conf);
-
-        assertEquals(1, coll.count());
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            builder.setContext(new RyaURI("http://context.com"));
+            builder.setColumnVisibility(new DocumentVisibility("A&B&C").flatten());
+            final RyaStatement statement = builder.build();
+
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(statement);
+            assertEquals(1, coll.count());
+
+            final RyaStatementBuilder builder2 = new RyaStatementBuilder();
+            builder2.setPredicate(new RyaURI("http://temp.com"));
+            builder2.setObject(new RyaURI("http://object.com"));
+            builder2.setContext(new RyaURI("http://context3.com"));
+            final RyaStatement query = builder2.build();
+
+            dao.delete(query, conf);
+            assertEquals(1, coll.count());
+        } finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testVisibility() throws RyaDAOException, MongoException, IOException {
-        // Doc requires "A" and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            // Doc requires "A" and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A", new Authorizations("B")));
 
-        // Doc requires "A" and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
+            // Doc requires "A" and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A", new Authorizations("A")));
 
-        // Doc requires "A" and "B" and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
+            // Doc requires "A" and "B" and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A&B", new Authorizations("A", "B")));
 
-        // Doc requires "A" or "B" and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
+            // Doc requires "A" or "B" and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", new Authorizations("A", "B")));
 
-        // Doc requires "A" and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
+            // Doc requires "A" and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A", new Authorizations("A", "B")));
 
-        // Doc requires "A" and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
+            // Doc requires "A" and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A", new Authorizations("A", "B", "C")));
 
-        // Doc requires "A" and "B" and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
+            // Doc requires "A" and "B" and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B", new Authorizations("A")));
 
-        // Doc requires "A" and "B" and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B", new Authorizations("B")));
+            // Doc requires "A" and "B" and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B", new Authorizations("B")));
 
-        // Doc requires "A" and "B" and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B", new Authorizations("C")));
+            // Doc requires "A" and "B" and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B", new Authorizations("C")));
 
-        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
+            // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A&B&C", new Authorizations("A", "B", "C")));
 
-        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
+            // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B&C", new Authorizations("A", "B")));
 
-        // Doc requires "A" and "B" and "C" and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A")));
+            // Doc requires "A" and "B" and "C" and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B&C", new Authorizations("A")));
 
-        // Doc requires "A" and "B" and "C" and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("B")));
+            // Doc requires "A" and "B" and "C" and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B&C", new Authorizations("B")));
 
-        // Doc requires "A" and "B" and "C" and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("C")));
+            // Doc requires "A" and "B" and "C" and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&B&C", new Authorizations("C")));
 
-        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
+            // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A&B", new Authorizations("A", "B", "C")));
 
-        // Doc requires "A" or "B" and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
+            // Doc requires "A" or "B" and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", new Authorizations("A")));
 
-        // Doc requires "A" or "B" and user has "B" = User can view
-        assertTrue(testVisibilityStatement("A|B", new Authorizations("B")));
+            // Doc requires "A" or "B" and user has "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", new Authorizations("B")));
 
-        // Doc requires "A" or "B" and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|B", new Authorizations("C")));
+            // Doc requires "A" or "B" and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|B", new Authorizations("C")));
 
-        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
+            // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C", new Authorizations("A", "B", "C")));
 
-        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
+            // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C", new Authorizations("A", "B")));
 
-        // Doc requires "A" or "B" or "C" and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
+            // Doc requires "A" or "B" or "C" and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C", new Authorizations("A")));
 
-        // Doc requires "A" or "B" or "C" and user has "B" = User can view
-        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("B")));
+            // Doc requires "A" or "B" or "C" and user has "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C", new Authorizations("B")));
 
-        // Doc requires "A" or "B" or "C" and user has "C" = User can view
-        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("C")));
+            // Doc requires "A" or "B" or "C" and user has "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C", new Authorizations("C")));
 
-        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
+            // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", new Authorizations("A", "B", "C")));
 
-        // Doc requires "A" and user has ALL_AUTHORIZATIONS = User can view
-        assertTrue(testVisibilityStatement("A", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
+            // Doc requires "A" and user has ALL_AUTHORIZATIONS = User can view
+            assertTrue(testVisibilityStatement(dao, "A", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
 
-        // Doc requires "A" and "B" and user has ALL_AUTHORIZATIONS = User can view
-        assertTrue(testVisibilityStatement("A&B", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
+            // Doc requires "A" and "B" and user has ALL_AUTHORIZATIONS = User can view
+            assertTrue(testVisibilityStatement(dao, "A&B", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
 
-        // Doc requires "A" or "B" and user has ALL_AUTHORIZATIONS = User can view
-        assertTrue(testVisibilityStatement("A|B", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
+            // Doc requires "A" or "B" and user has ALL_AUTHORIZATIONS = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
 
-        // Doc has no requirement and user has ALL_AUTHORIZATIONS = User can view
-        assertTrue(testVisibilityStatement("", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
+            // Doc has no requirement and user has ALL_AUTHORIZATIONS = User can view
+            assertTrue(testVisibilityStatement(dao, "", MongoDbRdfConstants.ALL_AUTHORIZATIONS));
 
-        // Doc has no requirement and user has "A" = User can view
-        assertTrue(testVisibilityStatement("", new Authorizations("A")));
+            // Doc has no requirement and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "", new Authorizations("A")));
 
-        // Doc has no requirement and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("", new Authorizations("A", "B")));
+            // Doc has no requirement and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "", new Authorizations("A", "B")));
 
-        // Doc requires "A" or ("B" and "C") and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("A")));
+            // Doc requires "A" or ("B" and "C") and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|(B&C)", new Authorizations("A")));
 
-        // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("B", "C")));
+            // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|(B&C)", new Authorizations("B", "C")));
 
-        // Doc requires "A" or ("B" and "C") and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C)", new Authorizations("B")));
+            // Doc requires "A" or ("B" and "C") and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C)", new Authorizations("B")));
 
-        // Doc requires "A" or ("B" and "C") and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C)", new Authorizations("C")));
+            // Doc requires "A" or ("B" and "C") and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C)", new Authorizations("C")));
 
-        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "B")));
+            // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("A", "B")));
 
-        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "C")));
+            // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("A", "C")));
 
-        // Doc requires "A" and ("B" or "C") and user has "B" and "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&(B|C)", new Authorizations("B", "C")));
+            // Doc requires "A" and ("B" or "C") and user has "B" and "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("B", "C")));
 
-        // Doc requires "A" and ("B" or "C") and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&(B|C)", new Authorizations("A")));
+            // Doc requires "A" and ("B" or "C") and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("A")));
 
-        // Doc requires "A" and ("B" or "C") and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A&(B|C)", new Authorizations("B")));
+            // Doc requires "A" and ("B" or "C") and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("B")));
 
-        // Doc requires "A" and ("B" or "C") and user has "C" = User can view
-        assertFalse(testVisibilityStatement("A&(B|C)", new Authorizations("C")));
+            // Doc requires "A" and ("B" or "C") and user has "C" = User can view
+            assertFalse(testVisibilityStatement(dao, "A&(B|C)", new Authorizations("C")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("B")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("B")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("C")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("C")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("E")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("E")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("C", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("C", "D")));
 
-        // 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 "A" and "B" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B", "E")));
 
-        // 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")mongoClient or ("C" and "D") and user has
+            // "C" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(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
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "C")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "C")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("B", "C")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("B", "C")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("B", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("B", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B", "C")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B", "C")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "C", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "C", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("B", "C", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("B", "C", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" and "D" and "E"= User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("B", "C", "D", "E")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "B" and "C" and "D" and "E"= User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("B", "C", "D", "E")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B", "C", "D")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B", "C", "D")));
 
-        // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" and "D" and "E" = User can view
-        assertTrue(testVisibilityStatement("(A&B)|(C&D)", new Authorizations("A", "B", "C", "D", "E")));
+            // Doc requires ("A" and "B") or ("C" and "D") and user has "A" and "B" and "C" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A&B)|(C&D)", new Authorizations("A", "B", "C", "D", "E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("C")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("C")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "C" and "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("C", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "C" and "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("C", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B", "E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B", "E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "C" and "D" and "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("C", "D", "E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "C" and "D" and "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("C", "D", "E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "C")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "C")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B", "C")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B", "C")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "D" and "E" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B", "D", "E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B", "D", "E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B", "C")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B", "C")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "C", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "C", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B", "C", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B", "C", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" and "D" and "E"= User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("B", "C", "D", "E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "B" and "C" and "D" and "E"= User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("B", "C", "D", "E")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B", "C", "D")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B", "C", "D")));
 
-        // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" and "D" and "E" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|D)", new Authorizations("A", "B", "C", "D", "E")));
+            // Doc requires ("A" or "B") and ("C" or "D") and user has "A" and "B" and "C" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|D)", new Authorizations("A", "B", "C", "D", "E")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "C")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("A", "C")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "C" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "C")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "C" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("B", "C")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" and "E" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "D", "E")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("A", "D", "E")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "D" and "E" = User can view
-        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "D", "E")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "D" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("B", "D", "E")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "A" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "A" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("A")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("B")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("C")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("C")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("D")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("D")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("E")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("E")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "D" and "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("D", "E")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "D" and "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("D", "E")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "D")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("A", "D")));
 
-        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "E")));
+            // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "(A|B)&(C|(D&E))", new Authorizations("B", "E")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("A")));
+            // Doc requires "A|(B&C&(D|E))" and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("A")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "D" = User can view
-        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "D")));
+            // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "D" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("B", "C", "D")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "E" = User can view
-        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "E")));
+            // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "E" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("B", "C", "E")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "B" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B")));
+            // Doc requires "A|(B&C&(D|E))" and user has "B" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("B")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("C")));
+            // Doc requires "A|(B&C&(D|E))" and user has "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("C")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "D" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("D")));
+            // Doc requires "A|(B&C&(D|E))" and user has "D" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("D")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("E")));
+            // Doc requires "A|(B&C&(D|E))" and user has "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("E")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C")));
+            // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("B", "C")));
 
-        // Doc requires "A|(B&C&(D|E))" and user has "D" and "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("D", "E")));
+            // Doc requires "A|(B&C&(D|E))" and user has "D" and "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|(B&C&(D|E))", new Authorizations("D", "E")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" = User can view
-        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("A")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("E")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("E", "F")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "I" = User CANNOT view
-        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("I")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "I" = User CANNOT view
+            assertFalse(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("I")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" and "I" = User can view
-        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A", "I")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" and "I" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("A", "I")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" = User can view
-        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H")));
 
-        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" and "I" = User can view
-        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H", "I")));
+            // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" and "I" = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H", "I")));
 
-        // Doc has no requirement and user has ALL_AUTHORIZATIONS = User can view
-        assertTrue(testVisibilityStatement(null, MongoDbRdfConstants.ALL_AUTHORIZATIONS));
+            // Doc has no requirement and user has ALL_AUTHORIZATIONS = User can view
+            assertTrue(testVisibilityStatement(dao, null, MongoDbRdfConstants.ALL_AUTHORIZATIONS));
 
-        // Doc has no requirement and user has "A" = User can view
-        assertTrue(testVisibilityStatement(null, new Authorizations("A")));
+            // Doc has no requirement and user has "A" = User can view
+            assertTrue(testVisibilityStatement(dao, null, new Authorizations("A")));
 
-        // Doc has no requirement and user has "A" and "B" = User can view
-        assertTrue(testVisibilityStatement(null, new Authorizations("A", "B")));
+            // Doc has no requirement and user has "A" and "B" = User can view
+            assertTrue(testVisibilityStatement(dao, null, new Authorizations("A", "B")));
 
-        // Doc has no requirement and user has no authorizations = User can view
-        assertTrue(testVisibilityStatement(null, null));
+            // Doc has no requirement and user has no authorizations = User can view
+            assertTrue(testVisibilityStatement(dao, null, null));
 
-        // Doc has no requirement and user has no authorizations = User can view
-        assertTrue(testVisibilityStatement("", null));
+            // Doc has no requirement and user has no authorizations = User can view
+            assertTrue(testVisibilityStatement(dao, "", null));
 
-        // Doc requires "A" and user has no authorizations = User can view
-        assertTrue(testVisibilityStatement("A", null));
+            // Doc requires "A" and user has no authorizations = User can view
+            assertTrue(testVisibilityStatement(dao, "A", null));
 
-        // Doc requires "A" and "B" and user has no authorizations = User can view
-        assertTrue(testVisibilityStatement("A&B", null));
+            // Doc requires "A" and "B" and user has no authorizations = User can view
+            assertTrue(testVisibilityStatement(dao, "A&B", null));
 
-        // Doc requires "A" or "B" and user has no authorizations = User can view
-        assertTrue(testVisibilityStatement("A|B", null));
+            // Doc requires "A" or "B" and user has no authorizations = User can view
+            assertTrue(testVisibilityStatement(dao, "A|B", null));
+        } finally {
+            dao.destroy();
+        }
     }
 
     /**
@@ -503,8 +526,11 @@ public class MongoDBRyaDAOIT extends MongoTestBase {
      * in the collection. {@code false} otherwise.
      * @throws RyaDAOException
      */
-    private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
-        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+    private boolean testVisibilityStatement(
+            final MongoDBRyaDAO dao,
+            final String documentVisibility,
+            final Authorizations userAuthorizations) throws RyaDAOException {
+        final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
         final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
         final RyaStatement statement = buildVisibilityTestRyaStatement(documentVisibility);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 8cf75f1..200be9a 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
@@ -1,5 +1,4 @@
-package org.apache.rya.mongodb;
-/*
+/**
  * 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
@@ -17,6 +16,7 @@ package org.apache.rya.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.mongodb;
 
 import static org.junit.Assert.assertEquals;
 
@@ -27,93 +27,112 @@ import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.api.persist.RyaDAOException;
 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;
 
+/**
+ * Integration tests the methods of {@link MongoDBRyaDAO}.
+ */
 public class MongoDBRyaDAOTest extends MongoTestBase {
-    private MongoClient client;
-    private MongoDBRyaDAO dao;
-
-    @Before
-    public void setUp() throws IOException, RyaDAOException{
-        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(), conf);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            dao.delete(builder.build(), conf);
+        } finally {
+            dao.destroy();
+        }
     }
 
 
     @Test
     public void testAdd() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
 
-        final MongoDatabase db = client.getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
-        final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
 
-        dao.add(builder.build());
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
 
-        assertEquals(coll.count(),1);
+            dao.add(builder.build());
 
+            assertEquals(coll.count(),1);
+        }  finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testDelete() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
-        final RyaStatement statement = builder.build();
-
-        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, conf);
-
-        assertEquals(coll.count(),0);
-
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            final RyaStatement statement = builder.build();
+
+            final MongoDatabase db = conf.getMongoClient().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, conf);
+            assertEquals(coll.count(),0);
+        } finally {
+            dao.destroy();
+        }
     }
 
     @Test
     public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
-        final RyaStatementBuilder builder = new RyaStatementBuilder();
-        builder.setPredicate(new RyaURI("http://temp.com"));
-        builder.setSubject(new RyaURI("http://subject.com"));
-        builder.setObject(new RyaURI("http://object.com"));
-        builder.setContext(new RyaURI("http://context.com"));
-        final RyaStatement statement = builder.build();
-
-        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);
-
-        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
-        builder2.setPredicate(new RyaURI("http://temp.com"));
-        builder2.setObject(new RyaURI("http://object.com"));
-        builder2.setContext(new RyaURI("http://context3.com"));
-        final RyaStatement query = builder2.build();
-
-        dao.delete(query, conf);
-
-        assertEquals(coll.count(),1);
+        MongoDBRyaDAO dao = new MongoDBRyaDAO();
+        try {
+            dao.setConf(conf);
+            dao.init();
+
+            final RyaStatementBuilder builder = new RyaStatementBuilder();
+            builder.setPredicate(new RyaURI("http://temp.com"));
+            builder.setSubject(new RyaURI("http://subject.com"));
+            builder.setObject(new RyaURI("http://object.com"));
+            builder.setContext(new RyaURI("http://context.com"));
+            final RyaStatement statement = builder.build();
+
+            final MongoDatabase db = conf.getMongoClient().getDatabase(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
+            final MongoCollection<Document> coll = db.getCollection(conf.getTriplesCollectionName());
+
+            dao.add(statement);
+            assertEquals(coll.count(),1);
+
+            final RyaStatementBuilder builder2 = new RyaStatementBuilder();
+            builder2.setPredicate(new RyaURI("http://temp.com"));
+            builder2.setObject(new RyaURI("http://object.com"));
+            builder2.setContext(new RyaURI("http://context3.com"));
+            final RyaStatement query = builder2.build();
+
+            dao.delete(query, conf);
+            assertEquals(coll.count(),1);
+        } finally {
+            dao.destroy();
+        }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java b/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
deleted file mode 100644
index 3d95818..0000000
--- a/dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoRyaTestBase.java
+++ /dev/null
@@ -1,73 +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.mongodb;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.rya.api.persist.RyaDAOException;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-import com.mongodb.MongoClient;
-
-import de.flapdoodle.embed.mongo.distribution.Version;
-
-public class MongoRyaTestBase {
-
-    private static final AtomicInteger db = new AtomicInteger(1);
-
-    protected static EmbeddedMongoFactory testsFactory;
-    protected MongoClient mongoClient;
-    private int currentTestDb = -1;
-
-    @BeforeClass()
-    public static void beforeClass() throws Exception {
-        testsFactory = EmbeddedMongoFactory.with(Version.Main.PRODUCTION);
-    }
-
-    @Before
-    public void MongoRyaTestBaseSetUp() throws IOException, RyaDAOException {
-        mongoClient = testsFactory.newMongoClient();
-        currentTestDb = db.getAndIncrement();
-    }
-
-    @After
-    public void MongoRyaTestBaseAfter() throws Exception {
-        if (mongoClient != null) {
-            mongoClient.close();
-        }
-        currentTestDb = -1;
-        MongoConnectorFactory.closeMongoClient();
-    }
-
-    @AfterClass()
-    public static void afterClass() throws Exception {
-        if (testsFactory != null) {
-            testsFactory.shutdown();
-        }
-    }
-
-    public String getDbName() {
-        return "rya_" + currentTestDb;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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
index e325e82..a3db535 100644
--- 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
@@ -18,9 +18,10 @@
  */
 package org.apache.rya.mongodb;
 
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.bson.Document;
-import org.junit.After;
 import org.junit.Before;
 
 import com.mongodb.DBCollection;
@@ -33,28 +34,43 @@ import com.mongodb.client.MongoCollection;
  */
 public class MongoTestBase {
 
-    private static MongoClient mongoClient = null;
-    protected static MongoDBRdfConfiguration conf;
+    private MongoClient mongoClient = null;
+    protected StatefulMongoDBRdfConfiguration conf;
 
     @Before
     public void setupTest() throws Exception {
-        conf = new MongoDBRdfConfiguration( new Configuration() );
+        // Setup the configuration that will be used within the test.
+        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new Configuration() );
         conf.setBoolean("sc.useMongo", true);
         conf.setTablePrefix("test_");
         conf.setMongoDBName("testDB");
-        mongoClient = EmbeddedMongoSingleton.getInstance();
-        conf.setMongoClient(mongoClient);
-    }
+        conf.setMongoHostname(EmbeddedMongoSingleton.getMongodConfig().net().getServerAddress().getHostAddress());
+        conf.setMongoPort(Integer.toString(EmbeddedMongoSingleton.getMongodConfig().net().getPort()));
+
+        // Let tests update the configuration.
+        updateConfiguration(conf);
+
+        // Create the stateful configuration object.
+        mongoClient = EmbeddedMongoSingleton.getNewMongoClient();
+        final List<MongoSecondaryIndex> indexers = conf.getInstances("ac.additional.indexers", MongoSecondaryIndex.class);
+        this.conf = new StatefulMongoDBRdfConfiguration(conf, mongoClient, indexers);
 
-    @After
-    public void cleanupTest() {
-        // Remove any DBs that were created by the test.
+        // Remove any DBs that were created by previous tests.
         for(final String dbName : mongoClient.listDatabaseNames()) {
             mongoClient.dropDatabase(dbName);
         }
     }
 
     /**
+     * TODO doc
+     *
+     * @param conf
+     */
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        // By default, do nothing.
+    }
+
+    /**
      * @return A {@link MongoClient} that is connected to the embedded instance of Mongo DB.
      */
     public MongoClient getMongoClient() {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/dao/mongodb.rya/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/dao/mongodb.rya/src/test/resources/log4j.properties b/dao/mongodb.rya/src/test/resources/log4j.properties
new file mode 100644
index 0000000..73b9512
--- /dev/null
+++ b/dao/mongodb.rya/src/test/resources/log4j.properties
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+# Root logger option
+log4j.rootLogger=INFO, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
index cd5278e..507fde2 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/EntityIndexOptimizer.java
@@ -19,6 +19,7 @@
 package org.apache.rya.indexing.entity;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
@@ -35,6 +36,7 @@ import org.apache.rya.indexing.external.matching.ExternalSetProvider;
 import org.apache.rya.indexing.external.matching.QueryNodeListRater;
 import org.apache.rya.indexing.external.matching.QuerySegment;
 import org.apache.rya.indexing.external.matching.TopOfQueryFilterRelocator;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.Dataset;
 import org.openrdf.query.algebra.TupleExpr;
@@ -68,6 +70,9 @@ public class EntityIndexOptimizer extends AbstractExternalSetOptimizer<EntityQue
 
     @Override
     public void setConf(final Configuration conf) {
+        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The provided Configuration must be of type StatefulMongoDBRdfConfiguration, but was "  + conf.getClass().getName());
+
         this.conf = conf;
         indexer.setConf(conf);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
index 1ab48b6..d1d00fb 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/mongo/MongoEntityIndexer.java
@@ -18,6 +18,8 @@
  */
 package org.apache.rya.indexing.entity.update.mongo;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.EntityStorage.EntityStorageException;
@@ -26,8 +28,7 @@ import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
 import org.apache.rya.indexing.entity.storage.mongo.MongoTypeStorage;
 import org.apache.rya.indexing.entity.update.BaseEntityIndexer;
 import org.apache.rya.indexing.entity.update.EntityIndexer;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
 import com.mongodb.MongoClient;
 
@@ -43,29 +44,27 @@ public class MongoEntityIndexer extends BaseEntityIndexer {
 
     @Override
     public EntityStorage getEntityStorage(final Configuration conf) throws EntityStorageException {
-        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
+        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The provided Configuration must be of type StatefulMongoDBRdfConfiguration, but was "  + conf.getClass().getName());
+
+        final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
         if (client == null) {
-            if(mongoConf.getMongoClient() != null) {
-                client = mongoConf.getMongoClient();
-            } else {
-                client = MongoConnectorFactory.getMongoClient(conf);
-            }
+            client = mongoConf.getMongoClient();
         }
-        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
+        final String ryaInstanceName = mongoConf.getMongoDBName();
         return new MongoEntityStorage(client, ryaInstanceName);
     }
 
     @Override
     public TypeStorage getTypeStorage(final Configuration conf) {
-        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
+        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The provided Configuration must be of type StatefulMongoDBRdfConfiguration, but was "  + conf.getClass().getName());
+
+        final StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration) conf;
         if (client == null) {
-            if(mongoConf.getMongoClient() != null) {
-                client = mongoConf.getMongoClient();
-            } else {
-                client = MongoConnectorFactory.getMongoClient(conf);
-            }
+            client = mongoConf.getMongoClient();
         }
-        final String ryaInstanceName = new MongoDBRdfConfiguration(conf).getMongoDBName();
+        final String ryaInstanceName = mongoConf.getMongoDBName();
         return new MongoTypeStorage(client, ryaInstanceName);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 9ce6e22..a815040 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
@@ -1,6 +1,4 @@
-package org.apache.rya.indexing.mongodb;
-
-/*
+/**
  * 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
@@ -18,6 +16,9 @@ package org.apache.rya.indexing.mongodb;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.indexing.mongodb;
+
+import static com.google.common.base.Preconditions.checkState;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -29,10 +30,10 @@ import org.apache.rya.api.domain.RyaStatement;
 import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.api.resolver.RyaToRdfConversions;
 import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.apache.rya.mongodb.MongoDBRyaDAO;
 import org.apache.rya.mongodb.MongoSecondaryIndex;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.apache.rya.mongodb.batch.MongoDbBatchWriter;
 import org.apache.rya.mongodb.batch.MongoDbBatchWriterConfig;
 import org.apache.rya.mongodb.batch.MongoDbBatchWriterException;
@@ -43,14 +44,12 @@ import org.openrdf.model.Statement;
 import org.openrdf.model.URI;
 import org.openrdf.query.QueryEvaluationException;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.mongodb.DB;
 import com.mongodb.DBCollection;
 import com.mongodb.DBCursor;
 import com.mongodb.DBObject;
 import com.mongodb.MongoClient;
 import com.mongodb.QueryBuilder;
-import com.mongodb.ServerAddress;
 
 import info.aduna.iteration.CloseableIteration;
 
@@ -63,7 +62,7 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
 
     private boolean isInit = false;
     private boolean flushEachUpdate = true;
-    protected Configuration conf;
+    protected StatefulMongoDBRdfConfiguration conf;
     protected MongoDBRyaDAO dao;
     protected MongoClient mongoClient;
     protected String dbName;
@@ -76,7 +75,7 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
     private MongoDbBatchWriter<DBObject> mongoDbBatchWriter;
 
     protected void initCore() {
-        dbName = conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME);
+        dbName = conf.getMongoDBName();
         db = this.mongoClient.getDB(dbName);
         final String collectionName = conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + getCollectionName();
         collection = db.getCollection(collectionName);
@@ -84,7 +83,7 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
         flushEachUpdate = ((MongoDBRdfConfiguration)conf).flushEachUpdate();
 
         final MongoDbBatchWriterConfig mongoDbBatchWriterConfig = MongoDbBatchWriterUtils.getMongoDbBatchWriterConfig(conf);
-        mongoDbBatchWriter = new MongoDbBatchWriter<DBObject>(new DbCollectionType(collection), mongoDbBatchWriterConfig);
+        mongoDbBatchWriter = new MongoDbBatchWriter<>(new DbCollectionType(collection), mongoDbBatchWriterConfig);
         try {
             mongoDbBatchWriter.start();
         } catch (final MongoDbBatchWriterException e) {
@@ -97,24 +96,13 @@ public abstract class AbstractMongoIndexer<T extends IndexingMongoDBStorageStrat
         this.mongoClient = client;
     }
 
-    @VisibleForTesting
-    public void initIndexer(final Configuration conf, final MongoClient client) {
-        setClient(client);
-        final ServerAddress address = client.getAddress();
-        conf.set(MongoDBRdfConfiguration.MONGO_INSTANCE, address.getHost());
-        conf.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, Integer.toString(address.getPort()));
-        setConf(conf);
-        if (!isInit) {
-            init();
-            isInit = true;
-        }
-    }
-
     @Override
     public void setConf(final Configuration conf) {
-        this.conf = conf;
+        checkState(conf instanceof StatefulMongoDBRdfConfiguration,
+                "The provided Configuration must be a StatefulMongoDBRdfConfiguration, but it was " + conf.getClass().getName());
+        this.conf = (StatefulMongoDBRdfConfiguration) conf;
         if (!isInit){
-            setClient(MongoConnectorFactory.getMongoClient(conf));
+            setClient(this.conf.getMongoClient());
             init();
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/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 5d353b4..249bde5 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
@@ -39,8 +39,7 @@ import org.apache.rya.indexing.mongodb.update.RyaObjectStorage.ObjectStorageExce
 import org.apache.rya.indexing.smarturi.SmartUriAdapter;
 import org.apache.rya.indexing.smarturi.SmartUriException;
 import org.apache.rya.indexing.smarturi.SmartUriStorage;
-import org.apache.rya.mongodb.MongoConnectorFactory;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
 
@@ -52,15 +51,15 @@ import com.mongodb.MongoException;
  */
 public class MongoDbSmartUri implements SmartUriStorage {
     private boolean isInit = false;
-    private final MongoDBRdfConfiguration conf;
+    private final StatefulMongoDBRdfConfiguration conf;
     private MongoClient mongoClient = null;
     private EntityStorage entityStorage;
 
     /**
      * Creates a new instance of {@link MongoDbSmartUri}.
-     * @param conf the {@link MongoDBRdfConfiguration}. (not {@code null})
+     * @param conf the {@link StatefulMongoDBRdfConfiguration}. (not {@code null})
      */
-    public MongoDbSmartUri(final MongoDBRdfConfiguration conf) {
+    public MongoDbSmartUri(final StatefulMongoDBRdfConfiguration conf) {
         this.conf = checkNotNull(conf);
     }
 
@@ -147,13 +146,9 @@ public class MongoDbSmartUri implements SmartUriStorage {
      * @throws MongoException
      * @throws EntityStorageException
      */
-    private void setupClient(final Configuration conf) throws UnknownHostException, MongoException, EntityStorageException {
-        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) conf;
-        mongoClient = mongoConf.getMongoClient();
-        if (mongoClient == null) {
-            mongoClient = MongoConnectorFactory.getMongoClient(conf);
-        }
-        entityStorage = new MongoEntityStorage(mongoClient, mongoConf.getMongoInstance());
+    private void setupClient(final StatefulMongoDBRdfConfiguration conf) throws UnknownHostException, MongoException, EntityStorageException {
+        mongoClient = conf.getMongoClient();
+        entityStorage = new MongoEntityStorage(mongoClient, conf.getRyaInstance());
         isInit = true;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/767349da/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java b/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
index 91f281c..1025537 100644
--- a/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
+++ b/extras/indexing/src/main/java/org/apache/rya/indexing/statement/metadata/matching/RyaQueryEngineFactory.java
@@ -1,5 +1,4 @@
-package org.apache.rya.indexing.statement.metadata.matching;
-/*
+/**
  * 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
@@ -7,9 +6,9 @@ package org.apache.rya.indexing.statement.metadata.matching;
  * 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
@@ -17,6 +16,8 @@ package org.apache.rya.indexing.statement.metadata.matching;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.indexing.statement.metadata.matching;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
@@ -28,23 +29,18 @@ import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.query.AccumuloRyaQueryEngine;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
 import org.apache.rya.api.persist.query.RyaQueryEngine;
-import org.apache.rya.mongodb.MongoConnectorFactory;
 import org.apache.rya.mongodb.MongoDBQueryEngine;
-import org.apache.rya.mongodb.MongoDBRdfConfiguration;
-
-import com.mongodb.MongoClient;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 
 /**
  * THis class creates the appropriate {@link RyaQueryEngine} based on the type of
  * {@link RdfCloudTripleStoreConfiguration} object that is passed in and whether or not
  * Rya is configured to use Mongo.
- *
  */
 public class RyaQueryEngineFactory {
 
-    
     @SuppressWarnings("unchecked")
-    public static <C extends RdfCloudTripleStoreConfiguration> RyaQueryEngine<C> getQueryEngine(RdfCloudTripleStoreConfiguration conf) { 
+    public static <C extends RdfCloudTripleStoreConfiguration> RyaQueryEngine<C> getQueryEngine(RdfCloudTripleStoreConfiguration conf) {
         if(conf instanceof AccumuloRdfConfiguration) {
             AccumuloRdfConfiguration aConf = (AccumuloRdfConfiguration) conf;
             Instance instance;
@@ -64,12 +60,13 @@ public class RyaQueryEngineFactory {
                 throw new RuntimeException(e);
             }
             return (RyaQueryEngine<C>) new AccumuloRyaQueryEngine(conn, aConf);
-        } else if(conf instanceof MongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
-            MongoClient client = MongoConnectorFactory.getMongoClient(conf);
-            return (RyaQueryEngine<C>) new MongoDBQueryEngine((MongoDBRdfConfiguration) conf, client);
+        } else if(conf instanceof StatefulMongoDBRdfConfiguration && conf.getBoolean("sc.useMongo", false)) {
+            StatefulMongoDBRdfConfiguration mongoConf = (StatefulMongoDBRdfConfiguration)conf;
+            MongoDBQueryEngine mongoQueryEngine = new MongoDBQueryEngine();
+            mongoQueryEngine.setConf(mongoConf);
+            return (RyaQueryEngine<C>) mongoQueryEngine;
         } else {
             throw new IllegalArgumentException("Invalid configuration type.");
         }
     }
-    
-}
+}
\ No newline at end of file


[16/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
index d5fe97c..6b73691 100644
--- 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
@@ -51,7 +51,7 @@ 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.apache.rya.mongodb.MongoITBase;
 import org.joda.time.DateTime;
 import org.junit.Before;
 import org.junit.Test;
@@ -77,7 +77,7 @@ import info.aduna.iteration.CloseableIteration;
 /**
  * Tests for MongoDB based Smart URI.
  */
-public class MongoDbSmartUriIT extends MongoTestBase {
+public class MongoDbSmartUriIT extends MongoITBase {
     private static final String NAMESPACE = RyaSchema.NAMESPACE;
     private static final ValueFactory VALUE_FACTORY = ValueFactoryImpl.getInstance();
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndex2IT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndex2IT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndex2IT.java
new file mode 100644
index 0000000..ee3869f
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndex2IT.java
@@ -0,0 +1,248 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.domain.RyaURI;
+import org.apache.rya.indexing.entity.EntityIndexOptimizer;
+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.query.EntityQueryNode;
+import org.apache.rya.indexing.entity.storage.EntityStorage;
+import org.apache.rya.indexing.entity.storage.TypeStorage;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.query.algebra.helpers.StatementPatternCollector;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+
+import com.google.common.collect.ImmutableSet;
+
+public class MongoEntityIndex2IT extends MongoITBase {
+    private static final Type PERSON_TYPE =
+            new Type(new RyaURI("urn:person"),
+                ImmutableSet.<RyaURI>builder()
+                    .add(new RyaURI("urn:name"))
+                    .add(new RyaURI("urn:age"))
+                    .add(new RyaURI("urn:eye"))
+                    .build());
+    private static final RyaURI RYA_PERSON_TYPE = new RyaURI("urn:person");
+
+    private EntityIndexOptimizer optimizer;
+    private EntityStorage entityStorage;
+
+    @Before
+    public void beforeClass() throws Exception {
+        optimizer = new EntityIndexOptimizer();
+        optimizer.setConf(conf);
+
+        final TypeStorage typeStorage = optimizer.getTypeStorage();
+        typeStorage.create(PERSON_TYPE);
+
+        final Entity entity = Entity.builder()
+                .setSubject(new RyaURI("urn:SSN:111-11-1111"))
+                .setExplicitType(RYA_PERSON_TYPE)
+                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:age"), new RyaType("25")))
+                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:eye"), new RyaType("blue")))
+                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:name"), new RyaType("bob")))
+                .build();
+        entityStorage = optimizer.getEntityStorage();
+        entityStorage.create(entity);
+    }
+
+    @Test()
+    public void queryIsFullEntity() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(query), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryIsPartEntity() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(query), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryIsPartEntityandExtra() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:222-22-2222> <urn:age> ?age . " +
+                "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
+                "<urn:SSN:222-22-2222> <urn:name> ?name . " +
+            "}";
+
+        final String expectedQuery = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryIsFullEntityWithExtra() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                "<urn:SSN:222-22-2222> <urn:age> ?age . " +
+                "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
+                "<urn:SSN:222-22-2222> <urn:name> ?name . " +
+            "}";
+
+        final String expectedQuery = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryIsFullEntityWithOptional() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+                "  OPTIONAL{" +
+                "    <urn:SSN:222-22-2222> <urn:age> ?age . " +
+                "    <urn:SSN:222-22-2222> <urn:eye> ?eye . " +
+                "    <urn:SSN:222-22-2222> <urn:name> ?name . " +
+                " } . " +
+            "}";
+
+        final String expectedQuery = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryIsSplitEntityWithOptional() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "  OPTIONAL{" +
+                "    <urn:SSN:111-11-1111> <urn:name> ?name . " +
+                " } . " +
+            "}";
+
+        final String expectedQuery = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    @Test()
+    public void queryEntityInOptional() throws Exception {
+        // A pattern that has two different subjects.
+        final String query = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
+                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
+                "  OPTIONAL{" +
+                "    <urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "    <urn:SSN:111-11-1111> <urn:name> ?name . " +
+                " } . " +
+            "}";
+
+        final String expectedQuery = "SELECT * WHERE { " +
+                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
+                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
+            "}";
+
+        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
+        assertOptimizer(query, expected);
+    }
+
+    private static List<StatementPattern> getSPs(final String sparql) throws MalformedQueryException {
+        final StatementPatternCollector spCollector = new StatementPatternCollector();
+        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
+        return spCollector.getStatementPatterns();
+    }
+
+    private void assertOptimizer(final String query, final EntityQueryNode expected) throws Exception {
+        final SPARQLParser parser = new SPARQLParser();
+        final TupleExpr expr = parser.parseQuery(query, null).getTupleExpr();
+
+        optimizer.optimize(expr, null, null);
+        expr.visit(new EntityFetchingAsserterVisitor(expected));
+    }
+
+    private class EntityFetchingAsserterVisitor extends QueryModelVisitorBase<Exception> {
+        private final EntityQueryNode expected;
+        public EntityFetchingAsserterVisitor(final EntityQueryNode expected) {
+            this.expected = expected;
+        }
+        @Override
+        protected void meetNode(final QueryModelNode node) throws Exception {
+            if(node instanceof EntityQueryNode) {
+                Assert.assertEquals(expected, node);
+            } else {
+                super.meetNode(node);
+            }
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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 e476874..463cabc 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
@@ -33,7 +33,7 @@ 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.MongoDBRdfConfiguration;
-import org.apache.rya.mongodb.MongoTestBase;
+import org.apache.rya.mongodb.MongoITBase;
 import org.apache.rya.sail.config.RyaSailFactory;
 import org.junit.Test;
 import org.openrdf.model.URI;
@@ -51,7 +51,7 @@ import org.openrdf.sail.Sail;
 
 import com.google.common.collect.ImmutableSet;
 
-public class MongoEntityIndexIT extends MongoTestBase {
+public class MongoEntityIndexIT extends MongoITBase {
     private static final ValueFactory VF = ValueFactoryImpl.getInstance();
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 2fe0cb3..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoEntityIndexTest.java
+++ /dev/null
@@ -1,248 +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 java.util.List;
-
-import org.apache.rya.api.domain.RyaType;
-import org.apache.rya.api.domain.RyaURI;
-import org.apache.rya.indexing.entity.EntityIndexOptimizer;
-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.query.EntityQueryNode;
-import org.apache.rya.indexing.entity.storage.EntityStorage;
-import org.apache.rya.indexing.entity.storage.TypeStorage;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.openrdf.model.vocabulary.RDF;
-import org.openrdf.query.MalformedQueryException;
-import org.openrdf.query.algebra.QueryModelNode;
-import org.openrdf.query.algebra.StatementPattern;
-import org.openrdf.query.algebra.TupleExpr;
-import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
-import org.openrdf.query.algebra.helpers.StatementPatternCollector;
-import org.openrdf.query.parser.sparql.SPARQLParser;
-
-import com.google.common.collect.ImmutableSet;
-
-public class MongoEntityIndexTest extends MongoTestBase {
-    private static final Type PERSON_TYPE =
-            new Type(new RyaURI("urn:person"),
-                ImmutableSet.<RyaURI>builder()
-                    .add(new RyaURI("urn:name"))
-                    .add(new RyaURI("urn:age"))
-                    .add(new RyaURI("urn:eye"))
-                    .build());
-    private static final RyaURI RYA_PERSON_TYPE = new RyaURI("urn:person");
-
-    private EntityIndexOptimizer optimizer;
-    private EntityStorage entityStorage;
-
-    @Before
-    public void beforeClass() throws Exception {
-        optimizer = new EntityIndexOptimizer();
-        optimizer.setConf(conf);
-
-        final TypeStorage typeStorage = optimizer.getTypeStorage();
-        typeStorage.create(PERSON_TYPE);
-
-        final Entity entity = Entity.builder()
-                .setSubject(new RyaURI("urn:SSN:111-11-1111"))
-                .setExplicitType(RYA_PERSON_TYPE)
-                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:age"), new RyaType("25")))
-                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:eye"), new RyaType("blue")))
-                .setProperty(RYA_PERSON_TYPE, new Property(new RyaURI("urn:name"), new RyaType("bob")))
-                .build();
-        entityStorage = optimizer.getEntityStorage();
-        entityStorage.create(entity);
-    }
-
-    @Test()
-    public void queryIsFullEntity() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(query), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryIsPartEntity() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(query), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryIsPartEntityandExtra() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:222-22-2222> <urn:age> ?age . " +
-                "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
-                "<urn:SSN:222-22-2222> <urn:name> ?name . " +
-            "}";
-
-        final String expectedQuery = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryIsFullEntityWithExtra() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                "<urn:SSN:222-22-2222> <urn:age> ?age . " +
-                "<urn:SSN:222-22-2222> <urn:eye> ?eye . " +
-                "<urn:SSN:222-22-2222> <urn:name> ?name . " +
-            "}";
-
-        final String expectedQuery = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryIsFullEntityWithOptional() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-                "  OPTIONAL{" +
-                "    <urn:SSN:222-22-2222> <urn:age> ?age . " +
-                "    <urn:SSN:222-22-2222> <urn:eye> ?eye . " +
-                "    <urn:SSN:222-22-2222> <urn:name> ?name . " +
-                " } . " +
-            "}";
-
-        final String expectedQuery = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryIsSplitEntityWithOptional() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "  OPTIONAL{" +
-                "    <urn:SSN:111-11-1111> <urn:name> ?name . " +
-                " } . " +
-            "}";
-
-        final String expectedQuery = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    @Test()
-    public void queryEntityInOptional() throws Exception {
-        // A pattern that has two different subjects.
-        final String query = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <urn:age> ?age . " +
-                "<urn:SSN:111-11-1111> <urn:eye> ?eye . " +
-                "  OPTIONAL{" +
-                "    <urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "    <urn:SSN:111-11-1111> <urn:name> ?name . " +
-                " } . " +
-            "}";
-
-        final String expectedQuery = "SELECT * WHERE { " +
-                "<urn:SSN:111-11-1111> <" + RDF.TYPE + "> <urn:person> ."+
-                "<urn:SSN:111-11-1111> <urn:name> ?name . " +
-            "}";
-
-        final EntityQueryNode expected = new EntityQueryNode(PERSON_TYPE, getSPs(expectedQuery), entityStorage);
-        assertOptimizer(query, expected);
-    }
-
-    private static List<StatementPattern> getSPs(final String sparql) throws MalformedQueryException {
-        final StatementPatternCollector spCollector = new StatementPatternCollector();
-        new SPARQLParser().parseQuery(sparql, null).getTupleExpr().visit(spCollector);
-        return spCollector.getStatementPatterns();
-    }
-
-    private void assertOptimizer(final String query, final EntityQueryNode expected) throws Exception {
-        final SPARQLParser parser = new SPARQLParser();
-        final TupleExpr expr = parser.parseQuery(query, null).getTupleExpr();
-
-        optimizer.optimize(expr, null, null);
-        expr.visit(new EntityFetchingAsserterVisitor(expected));
-    }
-
-    private class EntityFetchingAsserterVisitor extends QueryModelVisitorBase<Exception> {
-        private final EntityQueryNode expected;
-        public EntityFetchingAsserterVisitor(final EntityQueryNode expected) {
-            this.expected = expected;
-        }
-        @Override
-        protected void meetNode(final QueryModelNode node) throws Exception {
-            if(node instanceof EntityQueryNode) {
-                Assert.assertEquals(expected, node);
-            } else {
-                super.meetNode(node);
-            }
-        }
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerIT.java
new file mode 100644
index 0000000..a8d496e
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerIT.java
@@ -0,0 +1,188 @@
+/**
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.rya.api.domain.RyaStatement;
+import org.apache.rya.api.domain.RyaType;
+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.StatementConstraints;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+import org.openrdf.model.Statement;
+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.RDFS;
+
+import com.google.common.collect.Sets;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * Integration tests the methods of {@link MongoFreeTextIndexer}.
+ */
+public class MongoFreeTextIndexerIT extends MongoITBase {
+    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
+
+    @Test
+    public void testSearch() throws Exception {
+        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI subject = new URIImpl("foo:subj");
+            final URI predicate = RDFS.LABEL;
+            final Value object = vf.createLiteral("this is a new hat");
+
+            final URI context = new URIImpl("foo:context");
+
+            final Statement statement = vf.createStatement(subject, predicate, object, context);
+            f.storeStatement(RdfToRyaConversions.convertStatement(statement));
+            f.flush();
+
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("asdf", EMPTY_CONSTRAINTS)));
+
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat new", EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testDelete() throws Exception {
+        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI subject1 = new URIImpl("foo:subj");
+            final URI predicate1 = RDFS.LABEL;
+            final Value object1 = vf.createLiteral("this is a new hat");
+
+            final URI context1 = new URIImpl("foo:context");
+
+            final Statement statement1 = vf.createStatement(subject1, predicate1, object1, context1);
+            f.storeStatement(RdfToRyaConversions.convertStatement(statement1));
+
+            final URI subject2 = new URIImpl("foo:subject");
+            final URI predicate2 = RDFS.LABEL;
+            final Value object2 = vf.createLiteral("Do you like my new hat?");
+
+            final URI context2 = new URIImpl("foo:context");
+
+            final Statement statement2 = vf.createStatement(subject2, predicate2, object2, context2);
+            f.storeStatement(RdfToRyaConversions.convertStatement(statement2));
+
+            f.flush();
+
+
+            f.deleteStatement(RdfToRyaConversions.convertStatement(statement1));
+            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
+
+            // Check that "new" didn't get deleted from the term table after "this is a new hat"
+            // was deleted since "new" is still in "Do you like my new hat?"
+            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
+
+            f.deleteStatement(RdfToRyaConversions.convertStatement(statement2));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("this is a new hat", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
+        }
+    }
+
+    @Test
+    public void testRestrictPredicatesSearch() throws Exception {
+        conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, "pred:1,pred:2");
+
+        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            // 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")));
+            f.storeStatement(new RyaStatement(new RyaURI("foo:subj2"), new RyaURI(RDFS.COMMENT.toString()), new RyaType("invalid")));
+
+            final RyaURI pred1 = new RyaURI("pred:1");
+            final RyaURI pred2 = new RyaURI("pred:2");
+
+            // These should be stored because they are in the predicate list
+            final RyaStatement s3 = new RyaStatement(new RyaURI("foo:subj3"), pred1, new RyaType("valid"));
+            final RyaStatement s4 = new RyaStatement(new RyaURI("foo:subj4"), pred2, new RyaType("valid"));
+            f.storeStatement(s3);
+            f.storeStatement(s4);
+
+            // This should not be stored because the object is not a literal
+            f.storeStatement(new RyaStatement(new RyaURI("foo:subj5"), pred1, new RyaURI("in:validURI")));
+
+            f.flush();
+
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("invalid", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(), getSet(f.queryText("in:validURI", EMPTY_CONSTRAINTS)));
+
+            final Set<Statement> actual = getSet(f.queryText("valid", EMPTY_CONSTRAINTS));
+            assertEquals(2, actual.size());
+            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s3)));
+            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s4)));
+        }
+    }
+
+    @Test
+    public void testContextSearch() throws Exception {
+        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
+            f.setConf(conf);
+            f.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+            final URI subject = new URIImpl("foo:subj");
+            final URI predicate = new URIImpl(RDFS.COMMENT.toString());
+            final Value object = vf.createLiteral("this is a new hat");
+            final URI context = new URIImpl("foo:context");
+
+            final Statement statement = vf.createStatement(subject, predicate, object, context);
+            f.storeStatement(RdfToRyaConversions.convertStatement(statement));
+            f.flush();
+
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", EMPTY_CONSTRAINTS)));
+            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", new StatementConstraints().setContext(context))));
+            assertEquals(Sets.newHashSet(),
+                    getSet(f.queryText("hat", new StatementConstraints().setContext(vf.createURI("foo:context2")))));
+        }
+    }
+
+    private static <X> Set<X> getSet(final CloseableIteration<X, ?> iter) throws Exception {
+        final Set<X> set = new HashSet<>();
+        while (iter.hasNext()) {
+            set.add(iter.next());
+        }
+        return set;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 71840ea..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoFreeTextIndexerTest.java
+++ /dev/null
@@ -1,188 +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.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.rya.api.domain.RyaStatement;
-import org.apache.rya.api.domain.RyaType;
-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.StatementConstraints;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.mongodb.freetext.MongoFreeTextIndexer;
-import org.apache.rya.mongodb.MongoTestBase;
-import org.junit.Test;
-import org.openrdf.model.Statement;
-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.RDFS;
-
-import com.google.common.collect.Sets;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * Integration tests the methods of {@link MongoFreeTextIndexer}.
- */
-public class MongoFreeTextIndexerTest extends MongoTestBase {
-    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-
-    @Test
-    public void testSearch() throws Exception {
-        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            final URI subject = new URIImpl("foo:subj");
-            final URI predicate = RDFS.LABEL;
-            final Value object = vf.createLiteral("this is a new hat");
-
-            final URI context = new URIImpl("foo:context");
-
-            final Statement statement = vf.createStatement(subject, predicate, object, context);
-            f.storeStatement(RdfToRyaConversions.convertStatement(statement));
-            f.flush();
-
-            assertEquals(Sets.newHashSet(), getSet(f.queryText("asdf", EMPTY_CONSTRAINTS)));
-
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat new", EMPTY_CONSTRAINTS)));
-        }
-    }
-
-    @Test
-    public void testDelete() throws Exception {
-        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            final URI subject1 = new URIImpl("foo:subj");
-            final URI predicate1 = RDFS.LABEL;
-            final Value object1 = vf.createLiteral("this is a new hat");
-
-            final URI context1 = new URIImpl("foo:context");
-
-            final Statement statement1 = vf.createStatement(subject1, predicate1, object1, context1);
-            f.storeStatement(RdfToRyaConversions.convertStatement(statement1));
-
-            final URI subject2 = new URIImpl("foo:subject");
-            final URI predicate2 = RDFS.LABEL;
-            final Value object2 = vf.createLiteral("Do you like my new hat?");
-
-            final URI context2 = new URIImpl("foo:context");
-
-            final Statement statement2 = vf.createStatement(subject2, predicate2, object2, context2);
-            f.storeStatement(RdfToRyaConversions.convertStatement(statement2));
-
-            f.flush();
-
-
-            f.deleteStatement(RdfToRyaConversions.convertStatement(statement1));
-            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
-
-            // Check that "new" didn't get deleted from the term table after "this is a new hat"
-            // was deleted since "new" is still in "Do you like my new hat?"
-            assertEquals(Sets.newHashSet(statement2), getSet(f.queryText("new", EMPTY_CONSTRAINTS)));
-
-            f.deleteStatement(RdfToRyaConversions.convertStatement(statement2));
-            assertEquals(Sets.newHashSet(), getSet(f.queryText("this is a new hat", EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(), getSet(f.queryText("Do you like my new hat?", EMPTY_CONSTRAINTS)));
-        }
-    }
-
-    @Test
-    public void testRestrictPredicatesSearch() throws Exception {
-        conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, "pred:1,pred:2");
-
-        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            // 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")));
-            f.storeStatement(new RyaStatement(new RyaURI("foo:subj2"), new RyaURI(RDFS.COMMENT.toString()), new RyaType("invalid")));
-
-            final RyaURI pred1 = new RyaURI("pred:1");
-            final RyaURI pred2 = new RyaURI("pred:2");
-
-            // These should be stored because they are in the predicate list
-            final RyaStatement s3 = new RyaStatement(new RyaURI("foo:subj3"), pred1, new RyaType("valid"));
-            final RyaStatement s4 = new RyaStatement(new RyaURI("foo:subj4"), pred2, new RyaType("valid"));
-            f.storeStatement(s3);
-            f.storeStatement(s4);
-
-            // This should not be stored because the object is not a literal
-            f.storeStatement(new RyaStatement(new RyaURI("foo:subj5"), pred1, new RyaURI("in:validURI")));
-
-            f.flush();
-
-            assertEquals(Sets.newHashSet(), getSet(f.queryText("invalid", EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(), getSet(f.queryText("in:validURI", EMPTY_CONSTRAINTS)));
-
-            final Set<Statement> actual = getSet(f.queryText("valid", EMPTY_CONSTRAINTS));
-            assertEquals(2, actual.size());
-            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s3)));
-            assertTrue(actual.contains(RyaToRdfConversions.convertStatement(s4)));
-        }
-    }
-
-    @Test
-    public void testContextSearch() throws Exception {
-        try (MongoFreeTextIndexer f = new MongoFreeTextIndexer()) {
-            f.setConf(conf);
-            f.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-            final URI subject = new URIImpl("foo:subj");
-            final URI predicate = new URIImpl(RDFS.COMMENT.toString());
-            final Value object = vf.createLiteral("this is a new hat");
-            final URI context = new URIImpl("foo:context");
-
-            final Statement statement = vf.createStatement(subject, predicate, object, context);
-            f.storeStatement(RdfToRyaConversions.convertStatement(statement));
-            f.flush();
-
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", EMPTY_CONSTRAINTS)));
-            assertEquals(Sets.newHashSet(statement), getSet(f.queryText("hat", new StatementConstraints().setContext(context))));
-            assertEquals(Sets.newHashSet(),
-                    getSet(f.queryText("hat", new StatementConstraints().setContext(vf.createURI("foo:context2")))));
-        }
-    }
-
-    private static <X> Set<X> getSet(final CloseableIteration<X, ?> iter) throws Exception {
-        final Set<X> set = new HashSet<>();
-        while (iter.hasNext()) {
-            set.add(iter.next());
-        }
-        return set;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerIT.java
new file mode 100644
index 0000000..e92bcbb
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerIT.java
@@ -0,0 +1,722 @@
+package org.apache.rya.indexing.mongo;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+import static org.apache.rya.api.resolver.RdfToRyaConversions.convertStatement;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.HashSet;
+
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.rya.indexing.StatementConstraints;
+import org.apache.rya.indexing.TemporalInstant;
+import org.apache.rya.indexing.TemporalInstantRfc3339;
+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.MongoITBase;
+import org.junit.Test;
+import org.openrdf.model.Statement;
+import org.openrdf.model.URI;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.StatementImpl;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.QueryEvaluationException;
+
+import com.mongodb.DB;
+import com.mongodb.DBCollection;
+import com.mongodb.DBCursor;
+import com.mongodb.DBObject;
+import com.mongodb.MongoException;
+import com.mongodb.MongoSecurityException;
+
+import info.aduna.iteration.CloseableIteration;
+
+/**
+ * JUnit tests for TemporalIndexer and it's implementation MongoTemporalIndexer
+ *
+ * If you enjoy this test, please read RyaTemporalIndexerTest and YagoKBTest, which contain
+ * many example SPARQL queries and updates and attempts to test independently of Mongo:
+ *
+ *     extras/indexingSail/src/test/java/org.apache/rya/indexing/Mongo/RyaTemporalIndexerTest.java
+ *     {@link org.apache.rya.indexing.Mongo.RyaTemporalIndexerTest}
+ *     {@link org.apache.rya.indexing.Mongo.YagoKBTest.java}
+ *
+ * Remember, this class in instantiated fresh for each @test method.
+ * so fields are reset, unless they are static.
+ *
+ * These are covered:
+ *   Instance {before, equals, after} given Instance
+ *   Instance {before, after, inside} given Interval
+ *   Instance {hasBeginning, hasEnd} given Interval
+ * And a few more.
+ *
+ */
+public final class MongoTemporalIndexerIT extends MongoITBase {
+
+    private static final String URI_PROPERTY_EVENT_TIME = "Property:event:time";
+    private static final String URI_PROPERTY_CIRCA = "Property:circa";
+    private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
+    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
+
+    // Assign this in setUpBeforeClass, store them in each test.
+    // setup() deletes table before each test.
+    static final Statement spo_B00_E01;
+    static final Statement spo_B03_E20;
+    static final Statement spo_B02_E29;
+    static final Statement spo_B02_E30;
+    static final Statement spo_B02_E40;
+    static final Statement spo_B02_E31;
+    static final Statement spo_B29_E30;
+    static final Statement spo_B30_E32;
+
+    // Instants:
+    static final Statement spo_B02;
+    static final int SERIES_OF_SECONDS = 41;
+    static final Statement seriesSpo[] = new Statement[SERIES_OF_SECONDS];
+
+    // These are shared for several tests. Only the seconds are different.
+    // tvB03_E20 read as: interval Begins 3 seconds, ends at 20 seconds
+    static final TemporalInterval tvB00_E01 = new TemporalInterval(makeInstant(00), makeInstant(01));
+    static final TemporalInterval tvB29_E30= new TemporalInterval(makeInstant(29), makeInstant(30));
+    static final TemporalInterval tvB30_E32= new TemporalInterval(makeInstant(30), makeInstant(32));
+    static final TemporalInterval tvB03_E20 = new TemporalInterval(makeInstant(03), makeInstant(20));
+    // 30 seconds, Begins earlier, ends later
+    static final TemporalInterval tvB02_E30= new TemporalInterval(makeInstant(02), makeInstant(30));
+    // use for interval after
+    static final TemporalInterval tvB02_E29= new TemporalInterval(makeInstant(02), makeInstant(29));
+    // same as above, but ends in the middle
+    static final TemporalInterval tvB02_E31 = new TemporalInterval(makeInstant(02), makeInstant(31));
+    // same as above, but ends even later
+    static final TemporalInterval tvB02_E40 = new TemporalInterval(makeInstant(02), makeInstant(40));
+    // instant, match beginnings of several above, before tiB03_E20
+    static final TemporalInstant tsB02 = makeInstant(02);
+    // instant, after all above
+    static final TemporalInstant tsB04 = makeInstant(04);
+
+    // Create a series of instants about times 0 - 40 seconds
+    static final TemporalInstant seriesTs[];
+    static {
+        seriesTs = new TemporalInstant[SERIES_OF_SECONDS];
+        for (int i = 0; i <= 40; i++) {
+            seriesTs[i] = makeInstant(i);
+        }
+    };
+
+    /**
+     * Make an uniform instant with given seconds.
+     */
+    static TemporalInstant makeInstant(final int secondsMakeMeUnique) {
+        return new TemporalInstantRfc3339(2015, 12, 30, 12, 00, secondsMakeMeUnique);
+    }
+
+    static {
+        // Setup the statements only once. Each test will store some of these in there own index table.
+        final ValueFactory vf = new ValueFactoryImpl();
+        final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+        // tiB03_E20 read as: time interval that Begins 3 seconds, ends at 20 seconds,
+        // Each time element the same, except seconds. year, month, .... minute are the same for each statement below.
+        spo_B00_E01 = new StatementImpl(vf.createURI("foo:event0"), pred1_atTime, vf.createLiteral(tvB00_E01.toString()));
+        spo_B02_E29 = new StatementImpl(vf.createURI("foo:event2"), pred1_atTime, vf.createLiteral(tvB02_E29.toString()));
+        spo_B02_E30 = new StatementImpl(vf.createURI("foo:event2"), pred1_atTime, vf.createLiteral(tvB02_E30.toString()));
+        spo_B02_E31 = new StatementImpl(vf.createURI("foo:event3"), pred1_atTime, vf.createLiteral(tvB02_E31.toString()));
+        spo_B02_E40 = new StatementImpl(vf.createURI("foo:event4"), pred1_atTime, vf.createLiteral(tvB02_E40.toString()));
+        spo_B03_E20 = new StatementImpl(vf.createURI("foo:event5"), pred1_atTime, vf.createLiteral(tvB03_E20.toString()));
+        spo_B29_E30 = new StatementImpl(vf.createURI("foo:event1"), pred1_atTime, vf.createLiteral(tvB29_E30.toString()));
+        spo_B30_E32 = new StatementImpl(vf.createURI("foo:event1"), pred1_atTime, vf.createLiteral(tvB30_E32.toString()));
+        spo_B02 = new StatementImpl(vf.createURI("foo:event6"), pred1_atTime, vf.createLiteral(tsB02.getAsReadable()));
+
+        // Create statements about time instants 0 - 40 seconds
+        for (int i = 0; i < seriesTs.length; i++) {
+            seriesSpo[i] = new StatementImpl(vf.createURI("foo:event0" + i), pred1_atTime, vf.createLiteral(seriesTs[i].getAsReadable()));
+        }
+    }
+
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        // This is from http://linkedevents.org/ontology
+        // and http://motools.sourceforge.net/event/event.html
+        conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
+                + URI_PROPERTY_AT_TIME + ","
+                + URI_PROPERTY_CIRCA + ","
+                + URI_PROPERTY_EVENT_TIME);
+    }
+
+    /**
+     * Test method for {@link MongoTemporalIndexer#storeStatement(convertStatement(org.openrdf.model.Statement)}
+     */
+    @Test
+    public void testStoreStatement() throws IOException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
+
+            // Should not be stored because they are not in the predicate list
+            final String validDateStringWithThirteens = "1313-12-13T13:13:13Z";
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), RDFS.LABEL, vf.createLiteral(validDateStringWithThirteens))));
+
+            final String invalidDateString = "ThisIsAnInvalidDate";
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), pred1_atTime, vf.createLiteral(invalidDateString))));
+
+            // These are different datetimes instant but from different time zones.
+            // This is an arbitrary zone, BRST=Brazil, better if not local.
+            // same as "2015-01-01T01:59:59Z"
+            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
+            // next year, same as "2017-01-01T01:59:59Z"
+            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
+
+            // These should be stored because they are in the predicate list.
+            // BUT they will get converted to the same exact datetime in UTC.
+            final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
+            final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
+            tIndexer.storeStatement(convertStatement(s3));
+            tIndexer.storeStatement(convertStatement(s4));
+
+            // This should not be stored because the object is not a literal
+            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1_atTime, vf.createURI("in:valid"))));
+
+            printTables(tIndexer, "junit testing: Temporal entities stored in testStoreStatement");
+            assertEquals(2, tIndexer.getCollection().find().count());
+        }
+    }
+
+    @Test
+    public void testDelete() throws IOException, MongoException, TableNotFoundException, TableExistsException, NoSuchAlgorithmException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            final ValueFactory vf = new ValueFactoryImpl();
+
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
+
+            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
+            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
+
+            // These should be stored because they are in the predicate list.
+            // BUT they will get converted to the same exact datetime in UTC.
+            final Statement s1 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
+            final Statement s2 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
+            tIndexer.storeStatement(convertStatement(s1));
+            tIndexer.storeStatement(convertStatement(s2));
+
+            final String dbName = conf.getMongoDBName();
+            final DB db = super.getMongoClient().getDB(dbName);
+            DBCollection collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
+
+            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete before delete");
+            assertEquals("Number of rows stored.", 2, collection.count()); // 4 index entries per statement
+
+            tIndexer.deleteStatement(convertStatement(s1));
+            tIndexer.deleteStatement(convertStatement(s2));
+
+            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete after delete");
+            assertEquals("Number of rows stored after delete.", 0, collection.count());
+        }
+    }
+
+    /**
+     * Test instant after a given instant.
+     * From the series: instant {equal, before, after} instant
+     * @throws MongoSecurityException
+     * @throws MongoException
+     * @throws TableNotFoundException
+     */
+    @Test
+    public void testQueryInstantAfterInstant() throws IOException, QueryEvaluationException, TableNotFoundException, MongoException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 9;
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+    /**
+     * Test instant before a given instant.
+     * From the series: instant {equal, before, after} instant
+     */
+    @Test
+    public void testQueryInstantBeforeInstant() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 4;
+            for (int s = 0; s <= searchForSeconds + 15; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+
+            iter = tIndexer.queryInstantBeforeInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test instant before given interval.
+     * From the series:  Instance {before, after, inside} given Interval
+     */
+    @Test
+    public void testQueryInstantBeforeInterval() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchForSeconds = tvB02_E31;
+            final int expectedResultCount = 2; // 00 and 01 seconds.
+            for (int s = 0; s <= 40; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantBeforeInterval(searchForSeconds, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test instant after given interval.
+     * Instance {before, after, inside} given Interval
+     */
+    @Test
+    public void testQueryInstantAfterInterval() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchAfterInterval = tvB02_E31; // from 2 to 31 seconds
+            final int endingSeconds = 31;
+            final int expectedResultCount = 9; // 32,33,...,40 seconds.
+            for (int s = 0; s <= endingSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantAfterInterval(searchAfterInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count + endingSeconds + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test instant inside given interval.
+     * Instance {before, after, inside} given Interval
+     */
+    @Test
+    public void testQueryInstantInsideInterval() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int beginningSeconds = 2; // <== logic here, and next few lines.
+            final int endingSeconds = 31;
+            final int expectedResultCount = endingSeconds - beginningSeconds - 1; // 3,4,...,30 seconds.
+            for (int s = 0; s <= 40; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantInsideInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[count + beginningSeconds + 1]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test instant is the Beginning of the given interval.
+     * from the series: Instance {hasBeginning, hasEnd} Interval
+     */
+    @Test
+    public void testQueryInstantHasBeginningInterval() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int searchSeconds = 2; // <== logic here, and next few lines.
+            final int expectedResultCount = 1; // 2 seconds.
+            for (int s = 0; s <= 10; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantHasBeginningInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test instant is the end of the given interval.
+     * from the series: Instance {hasBeginning, hasEnd} Interval
+     */
+    @Test
+    public void testQueryInstantHasEndInterval()  throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
+            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
+            final int searchSeconds = 31; // <== logic here, and next few lines.
+            final int expectedResultCount = 1; // 31 seconds.
+            for (int s = 0; s <= 40; s++) {
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryInstantHasEndInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
+            int count = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
+                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                count++;
+            }
+            assertEquals("Should find count of rows.", expectedResultCount, count);
+        }
+    }
+
+    /**
+     * Test method for
+     * {@link org.apache.rya.indexing.Mongo.temporal.MongoTemporalIndexer#queryIntervalEquals(TemporalInterval, StatementConstraints)}
+     * .
+     * @throws IOException
+     * @throws QueryEvaluationException
+     *
+     */
+    @Test
+    public void testQueryIntervalEquals() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            tIndexer.storeStatement(convertStatement(seriesSpo[4])); // instance at 4 seconds
+
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalEquals(tvB02_E40, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but actually returned empty results. spo_B02_E40=" + spo_B02_E40, iter.hasNext());
+            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but does not match.", spo_B02_E40.equals(iter.next()));
+            assertFalse("queryIntervalEquals: Find no more than one, but actually has more.", iter.hasNext());
+        }
+    }
+
+    /**
+     * Test interval before a given interval, for method:
+     * {@link MongoTemporalIndexer#queryIntervalBefore(TemporalInterval, StatementConstraints)}.
+     *
+     * @throws IOException
+     * @throws QueryEvaluationException
+     */
+    @Test
+    public void testQueryIntervalBefore() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B00_E01));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            // instants should be ignored.
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
+            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
+            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
+
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalBefore(tvB02_E31, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("spo_B00_E01 should be found, but actually returned empty results. spo_B00_E01=" + spo_B00_E01, iter.hasNext());
+            assertTrue("spo_B00_E01 should be found, but found another.", spo_B00_E01.equals(iter.next()));
+            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
+        }
+    }
+
+    /**
+     * interval is after the given interval.  Find interval beginnings after the endings of the given interval.
+     * {@link MongoTemporalIndexer#queryIntervalAfter(TemporalInterval, StatementContraints).
+     *
+     * @throws IOException
+     * @throws QueryEvaluationException
+     */
+    @Test
+    public void testQueryIntervalAfter() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            tIndexer.storeStatement(convertStatement(spo_B00_E01));
+            tIndexer.storeStatement(convertStatement(spo_B02_E29)); //<- after this one.
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B29_E30));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+            // instants should be ignored.
+            tIndexer.storeStatement(convertStatement(spo_B02));
+            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
+            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
+            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            iter = tIndexer.queryIntervalAfter(tvB02_E29, EMPTY_CONSTRAINTS);
+            // Should be found twice:
+            assertTrue("spo_B30_E32 should be found, but actually returned empty results. spo_B30_E32=" + spo_B30_E32, iter.hasNext());
+            final Statement s = iter.next();
+            assertTrue("spo_B30_E32 should be found, but found another. spo_B30_E32="+spo_B30_E32+", but found="+s, spo_B30_E32.equals(s));
+            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
+        }
+    }
+
+    /**
+     * Test instant after a given instant WITH two different predicates as constraints.
+     */
+    @Test
+    public void testQueryWithMultiplePredicates() throws IOException, QueryEvaluationException {
+        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
+            tIndexer.setConf(conf);
+            tIndexer.init();
+
+            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
+            // these should not match as they are not instances.
+            tIndexer.storeStatement(convertStatement(spo_B03_E20));
+            tIndexer.storeStatement(convertStatement(spo_B02_E30));
+            tIndexer.storeStatement(convertStatement(spo_B02_E40));
+            tIndexer.storeStatement(convertStatement(spo_B02_E31));
+            tIndexer.storeStatement(convertStatement(spo_B30_E32));
+
+            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
+            final int searchForSeconds = 4;
+            final int expectedResultCount = 9;
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
+            }
+            final ValueFactory vf = new ValueFactoryImpl();
+            final URI pred3_CIRCA_ = vf.createURI(URI_PROPERTY_CIRCA);  // this one to ignore.
+            final URI pred2_eventTime = vf.createURI(URI_PROPERTY_EVENT_TIME);
+            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
+
+            // add the predicate = EventTime ; Store in an array for verification.
+            final Statement[] SeriesTs_EventTime = new Statement[expectedResultCount+1];
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                final Statement statement = new StatementImpl(vf.createURI("foo:EventTimeSubj0" + s), pred2_eventTime, vf.createLiteral(seriesTs[s].getAsReadable()));
+                tIndexer.storeStatement(convertStatement(statement));
+                if (s>searchForSeconds) {
+                    SeriesTs_EventTime[s - searchForSeconds -1 ] = statement;
+                }
+            }
+            // add the predicate = CIRCA ; to be ignored because it is not in the constraints.
+            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
+                final Statement statement = new StatementImpl(vf.createURI("foo:CircaEventSubj0" + s), pred3_CIRCA_, vf.createLiteral(seriesTs[s].getAsReadable()));
+                tIndexer.storeStatement(convertStatement(statement));
+            }
+
+            CloseableIteration<Statement, QueryEvaluationException> iter;
+            final StatementConstraints constraints = new StatementConstraints();
+            constraints.setPredicates(new HashSet<>(Arrays.asList( pred2_eventTime,  pred1_atTime )));
+
+            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], constraints); // EMPTY_CONSTRAINTS);//
+            int count_AtTime = 0;
+            int count_EventTime = 0;
+            while (iter.hasNext()) {
+                final Statement s = iter.next();
+                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count_AtTime + 1]; // <== logic here
+                if (s.getPredicate().equals(pred1_atTime)) {
+                    assertTrue("Should match atTime: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
+                    count_AtTime++;
+                }
+                else if (s.getPredicate().equals(pred2_eventTime)) {
+                    assertTrue("Should match eventTime: " + SeriesTs_EventTime[count_EventTime] + " == " + s, SeriesTs_EventTime[count_EventTime].equals(s));
+                    count_EventTime++;
+                } else {
+                    assertTrue("This predicate should not be returned: "+s, false);
+                }
+
+            }
+
+            assertEquals("Should find count of atTime    rows.", expectedResultCount, count_AtTime);
+            assertEquals("Should find count of eventTime rows.", expectedResultCount, count_EventTime);
+        }
+    }
+
+    /**
+     * Print and gather statistics on the entire index table.
+     *
+     * @param description
+     *            Printed to the console to find the test case.
+     * @param out
+     *            null or System.out or other output to send a listing.
+     * @param statistics
+     *            Hashes, sums, and counts for assertions.
+     * @return Count of entries in the index table.
+     * @throws IOException
+     */
+    public void printTables(MongoTemporalIndexer tIndexer, final String description) throws IOException {
+        System.out.println("-- start printTables() -- " + description);
+        System.out.println("Reading : " + tIndexer.getCollection().getFullName());
+        final DBCursor cursor = tIndexer.getCollection().find();
+        while(cursor.hasNext()) {
+            final DBObject dbo = cursor.next();
+            System.out.println(dbo.toString());
+        }
+        System.out.println();
+    }
+}


[08/17] incubator-rya git commit: RYA-414 Fixing broken tests, cleaning up documentate, cleaning up whitespace.

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
index 4ca6b4c..11c1d21 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerFilterIT.java
@@ -57,241 +57,241 @@ import com.vividsolutions.jts.io.WKTReader;
 import com.vividsolutions.jts.io.WKTWriter;
 
 public class MongoGeoIndexerFilterIT extends MongoTestBase {
-	private static final GeometryFactory GF = new GeometryFactory();
-	private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
-	private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));
-	private static final Geometry CAPITAL_BUILDING = GF.createPoint(new Coordinate(38.8899, 77.0091));
-	private static final Geometry WHITE_HOUSE = GF.createPoint(new Coordinate(38.8977, 77.0365));
-
-	@Override
-	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
-		conf.setBoolean(OptionalConfigUtils.USE_GEO, true);
-		conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
-		conf.setBoolean(ConfigUtils.USE_MONGO, true);
-	}
-
-	@Test
-	public void nearHappyUsesTest() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-		try {
-			populateRya(conn);
-
-			//Only captial
-			String query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 0.0, 2000))"
-							+ "}";
-
-			TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			final List<BindingSet> results = new ArrayList<>();
-			while (rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			assertEquals(1, results.size());
-			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(0)));
-
-			//all but capital
-			query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000))"
-							+ "}";
-
-			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			results.clear();
-			while (rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			assertEquals(3, results.size());
-			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-
-			// all of them
-			query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 6000, 000))"
-							+ "}";
-
-			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			results.clear();
-			while (rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			assertEquals(4, results.size());
-			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
-
-			// donut, only 2
-			query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000, 100))"
-							+ "}";
-
-			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			results.clear();
-			while (rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			assertEquals(2, results.size());
-			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(0)));
-			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(1)));
-
-			// all of them
-			query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral))"
-							+ "}";
-			rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			results.clear();
-			while (rez.hasNext()) {
-				final BindingSet bs = rez.next();
-				results.add(bs);
-			}
-			assertEquals(4, results.size());
-			assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
-			assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
-			assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
-			assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	@Test(expected = MalformedQueryException.class)
-	public void near_invalidDistance() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-		try {
-			populateRya(conn);
-
-			//Only captial
-			final String query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, distance))"
-							+ "}";
-
-			conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void near_negativeDistance() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-		try {
-			populateRya(conn);
-
-			//Only captial
-			final String query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n"
-							+ "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, -100))"
-							+ "}";
-
-			final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-			while(rez.hasNext()) {
-				rez.next();
-			}
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	@Test(expected = QueryEvaluationException.class)
-	public void tooManyArgumentsTest() throws Exception {
-		final Sail sail = GeoRyaSailFactory.getInstance(conf);
-		final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-		try {
-			populateRya(conn);
-
-			// Only captial
-			final String query =
-					"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-							+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-							+ "SELECT * \n" //
-							+ "WHERE { \n" + "  <urn:geo> geo:asWKT ?point .\n"
-							+ "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 100, 1000, 10))"
-							+ "}";
-
-			conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
-		} finally {
-			conn.close();
-			sail.shutDown();
-		}
-	}
-
-	private void populateRya(final SailRepositoryConnection conn) throws Exception {
-		// geo 2x2 points
-		conn.begin();
-		RyaStatement stmnt = statement(WASHINGTON_MONUMENT);
-		Statement statement = RyaToRdfConversions.convertStatement(stmnt);
-		conn.add(statement);
-
-		stmnt = statement(LINCOLN_MEMORIAL);
-		statement = RyaToRdfConversions.convertStatement(stmnt);
-		conn.add(statement);
-
-		stmnt = statement(CAPITAL_BUILDING);
-		statement = RyaToRdfConversions.convertStatement(stmnt);
-		conn.add(statement);
-
-		stmnt = statement(WHITE_HOUSE);
-		statement = RyaToRdfConversions.convertStatement(stmnt);
-		conn.add(statement);
-		conn.commit();
-	}
-
-	private static Geometry bindingToGeo(final BindingSet bs) throws ParseException {
-		final WKTReader w = new WKTReader();
-		return w.read(bs.getValue("point").stringValue());
-	}
-
-	private static RyaStatement statement(final Geometry geo) {
-		final ValueFactory vf = new ValueFactoryImpl();
-		final Resource subject = vf.createURI("urn:geo");
-		final URI predicate = GeoConstants.GEO_AS_WKT;
-		final WKTWriter w = new WKTWriter();
-		final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
-		return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
-	}
+    private static final GeometryFactory GF = new GeometryFactory();
+    private static final Geometry WASHINGTON_MONUMENT = GF.createPoint(new Coordinate(38.8895, 77.0353));
+    private static final Geometry LINCOLN_MEMORIAL = GF.createPoint(new Coordinate(38.8893, 77.0502));
+    private static final Geometry CAPITAL_BUILDING = GF.createPoint(new Coordinate(38.8899, 77.0091));
+    private static final Geometry WHITE_HOUSE = GF.createPoint(new Coordinate(38.8977, 77.0365));
+
+    @Override
+    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.setBoolean(OptionalConfigUtils.USE_GEO, true);
+        conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
+        conf.setBoolean(ConfigUtils.USE_MONGO, true);
+    }
+
+    @Test
+    public void nearHappyUsesTest() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        try {
+            populateRya(conn);
+
+            //Only captial
+            String query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 0.0, 2000))"
+                            + "}";
+
+            TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            final List<BindingSet> results = new ArrayList<>();
+            while (rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            assertEquals(1, results.size());
+            assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(0)));
+
+            //all but capital
+            query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000))"
+                            + "}";
+
+            rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            results.clear();
+            while (rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            assertEquals(3, results.size());
+            assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+            assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+            assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+
+            // all of them
+            query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 6000, 000))"
+                            + "}";
+
+            rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            results.clear();
+            while (rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            assertEquals(4, results.size());
+            assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+            assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+            assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+            assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+
+            // donut, only 2
+            query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 2000, 100))"
+                            + "}";
+
+            rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            results.clear();
+            while (rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            assertEquals(2, results.size());
+            assertEquals(WHITE_HOUSE, bindingToGeo(results.get(0)));
+            assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(1)));
+
+            // all of them
+            query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral))"
+                            + "}";
+            rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            results.clear();
+            while (rez.hasNext()) {
+                final BindingSet bs = rez.next();
+                results.add(bs);
+            }
+            assertEquals(4, results.size());
+            assertEquals(WASHINGTON_MONUMENT, bindingToGeo(results.get(0)));
+            assertEquals(WHITE_HOUSE, bindingToGeo(results.get(1)));
+            assertEquals(LINCOLN_MEMORIAL, bindingToGeo(results.get(2)));
+            assertEquals(CAPITAL_BUILDING, bindingToGeo(results.get(3)));
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    @Test(expected = MalformedQueryException.class)
+    public void near_invalidDistance() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        try {
+            populateRya(conn);
+
+            //Only captial
+            final String query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, distance))"
+                            + "}";
+
+            conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void near_negativeDistance() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        try {
+            populateRya(conn);
+
+            //Only captial
+            final String query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, -100))"
+                            + "}";
+
+            final TupleQueryResult rez = conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+            while(rez.hasNext()) {
+                rez.next();
+            }
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    @Test(expected = QueryEvaluationException.class)
+    public void tooManyArgumentsTest() throws Exception {
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        try {
+            populateRya(conn);
+
+            // Only captial
+            final String query =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "SELECT * \n" //
+                            + "WHERE { \n" + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfNear(?point, \"POINT(38.8895 77.0353)\"^^geo:wktLiteral, 100, 1000, 10))"
+                            + "}";
+
+            conn.prepareTupleQuery(QueryLanguage.SPARQL, query).evaluate();
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
+    }
+
+    private void populateRya(final SailRepositoryConnection conn) throws Exception {
+        // geo 2x2 points
+        conn.begin();
+        RyaStatement stmnt = statement(WASHINGTON_MONUMENT);
+        Statement statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(LINCOLN_MEMORIAL);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(CAPITAL_BUILDING);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+
+        stmnt = statement(WHITE_HOUSE);
+        statement = RyaToRdfConversions.convertStatement(stmnt);
+        conn.add(statement);
+        conn.commit();
+    }
+
+    private static Geometry bindingToGeo(final BindingSet bs) throws ParseException {
+        final WKTReader w = new WKTReader();
+        return w.read(bs.getValue("point").stringValue());
+    }
+
+    private static RyaStatement statement(final Geometry geo) {
+        final ValueFactory vf = new ValueFactoryImpl();
+        final Resource subject = vf.createURI("urn:geo");
+        final URI predicate = GeoConstants.GEO_AS_WKT;
+        final WKTWriter w = new WKTWriter();
+        final Value object = vf.createLiteral(w.write(geo), GeoConstants.XMLSCHEMA_OGC_WKT);
+        return RdfToRyaConversions.convertStatement(new StatementImpl(subject, predicate, object));
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
index 2d09a6c..bc4d870 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerSfTest.java
@@ -126,28 +126,28 @@ public class MongoGeoIndexerSfTest extends MongoTestBase {
 
     @Test
     public void testEquals() throws Exception {
-    	try(final MongoGeoIndexer g = new MongoGeoIndexer()) {
-    		g.setConf(conf);
-    		g.init();
-
-    		g.storeStatement(statement(A));
-    		g.storeStatement(statement(B));
-    		g.storeStatement(statement(C));
-    		g.storeStatement(statement(D));
-    		g.storeStatement(statement(F));
-    		g.storeStatement(statement(E));
-    		// point
-    		compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
-    		compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-    		// line
-    		compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
-    		compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-
-    		// poly
-    		compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
-    		compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
-    	}
+        try(final MongoGeoIndexer g = new MongoGeoIndexer()) {
+            g.setConf(conf);
+            g.init();
+
+            g.storeStatement(statement(A));
+            g.storeStatement(statement(B));
+            g.storeStatement(statement(C));
+            g.storeStatement(statement(D));
+            g.storeStatement(statement(F));
+            g.storeStatement(statement(E));
+            // point
+            compare(g.queryEquals(F, EMPTY_CONSTRAINTS), F);
+            compare(g.queryEquals(point(2, 2), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+            // line
+            compare(g.queryEquals(E, EMPTY_CONSTRAINTS), E);
+            compare(g.queryEquals(line(2, 2, 3, 3), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+
+            // poly
+            compare(g.queryEquals(A, EMPTY_CONSTRAINTS), A);
+            compare(g.queryEquals(poly(bbox(1, 1, 4, 5)), EMPTY_CONSTRAINTS), EMPTY_RESULTS);
+        }
     }
 
 //    @Test

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
index 0e0fff2..5953132 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoGeoIndexerTest.java
@@ -57,7 +57,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     GeometryFactory gf = new GeometryFactory(new PrecisionModel(), 4326);
 
     @Override
-	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.set(ConfigUtils.GEO_PREDICATES_LIST, "http://www.opengis.net/ont/geosparql#asWKT");
         conf.set(OptionalConfigUtils.USE_GEO, "true");
     }
@@ -65,8 +65,8 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     @Test
     public void testRestrictPredicatesSearch() throws Exception {
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
-        	f.setConf(conf);
+            conf.setStrings(ConfigUtils.GEO_PREDICATES_LIST, "pred:1,pred:2");
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -106,7 +106,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     @Test
     public void testPrimeMeridianSearch() throws Exception {
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -151,7 +151,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDcSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -181,7 +181,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDeleteSearch() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -221,7 +221,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDcSearchWithContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -251,7 +251,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDcSearchWithSubject() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -280,7 +280,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDcSearchWithSubjectAndContext() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -314,7 +314,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     public void testDcSearchWithPredicate() throws Exception {
         // test a ring around dc
         try (final MongoGeoIndexer f = new MongoGeoIndexer()) {
-        	f.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();
@@ -345,7 +345,7 @@ public class MongoGeoIndexerTest extends MongoTestBase {
     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.setConf(conf);
+            f.setConf(conf);
             f.init();
 
             final ValueFactory vf = new ValueFactoryImpl();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
----------------------------------------------------------------------
diff --git a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
index cbb53e5..1c2a339 100644
--- a/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
+++ b/extras/rya.geoindexing/geo.mongo/src/test/java/org/apache/rya/indexing/mongo/MongoIndexerDeleteIT.java
@@ -55,7 +55,7 @@ import com.vividsolutions.jts.io.WKTWriter;
 
 public class MongoIndexerDeleteIT extends MongoTestBase {
     @Override
-	public void updateConfiguration(final MongoDBRdfConfiguration conf) {
+    public void updateConfiguration(final MongoDBRdfConfiguration conf) {
         conf.setStrings(ConfigUtils.FREETEXT_PREDICATES_LIST, new String[] {RDFS.LABEL.stringValue()});
         conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, new String[] {"Property:atTime"});
         conf.setBoolean(ConfigUtils.USE_FREETEXT, true);
@@ -66,59 +66,59 @@ public class MongoIndexerDeleteIT extends MongoTestBase {
 
     @Test
     public void deleteTest() throws Exception {
-    	final Sail sail = GeoRyaSailFactory.getInstance(conf);
-    	final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
-    	try {
-    		populateRya(conn);
-    		final MongoClient client = conf.getMongoClient();
-    		
-    		//The extra 1 is from the person type defined in freetext
-    		assertEquals(8, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
-    		assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
-    		assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
-    		assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
-
-    		//free text -- remove one from many
-    		String delete = "DELETE DATA \n" //
-    				+ "{\n"
-    				+ "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
-    				+ "}";
-    		Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-    		update.execute();
-
-    		// temporal -- remove one from one
-    		delete = "DELETE DATA \n" //
-    				+ "{\n"
-    				+ "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
-    				+ "}";
-
-    		update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-    		update.execute();
-
-    		//geo -- remove many from many
-    		delete =
-    				"PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
-    						+ "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
-    						+ "DELETE \n" //
-    						+ "{\n"
-    						+ "  <urn:geo> geo:asWKT ?point \n"
-    						+ "}"
-    						+ "WHERE { \n"
-    						+ "  <urn:geo> geo:asWKT ?point .\n"
-    						+ "  FILTER(geof:sfWithin(?point, \"POLYGON((0 0, 2 0, 2 1, 0 1, 0 0))\"^^geo:wktLiteral))"
-    						+ "}";
-
-    		update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
-    		update.execute();
-
-    		assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
-    		assertEquals(0, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
-    		assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
-    		assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
-    	} finally {
-    		conn.close();
-    		sail.shutDown();
-    	}
+        final Sail sail = GeoRyaSailFactory.getInstance(conf);
+        final SailRepositoryConnection conn = new SailRepository(sail).getConnection();
+        try {
+            populateRya(conn);
+            final MongoClient client = conf.getMongoClient();
+
+            //The extra 1 is from the person type defined in freetext
+            assertEquals(8, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
+            assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
+            assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
+            assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
+
+            //free text -- remove one from many
+            String delete = "DELETE DATA \n" //
+                    + "{\n"
+                    + "  <urn:people> <http://www.w3.org/2000/01/rdf-schema#label> \"Alice Palace Hose\" "
+                    + "}";
+            Update update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+            update.execute();
+
+            // temporal -- remove one from one
+            delete = "DELETE DATA \n" //
+                    + "{\n"
+                    + "  <foo:time> <Property:atTime> \"0001-02-03T04:05:06Z\" "
+                    + "}";
+
+            update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+            update.execute();
+
+            //geo -- remove many from many
+            delete =
+                    "PREFIX geo: <http://www.opengis.net/ont/geosparql#>\n"
+                            + "PREFIX geof: <http://www.opengis.net/def/function/geosparql/>\n"
+                            + "DELETE \n" //
+                            + "{\n"
+                            + "  <urn:geo> geo:asWKT ?point \n"
+                            + "}"
+                            + "WHERE { \n"
+                            + "  <urn:geo> geo:asWKT ?point .\n"
+                            + "  FILTER(geof:sfWithin(?point, \"POLYGON((0 0, 2 0, 2 1, 0 1, 0 0))\"^^geo:wktLiteral))"
+                            + "}";
+
+            update = conn.prepareUpdate(QueryLanguage.SPARQL, delete);
+            update.execute();
+
+            assertEquals(2, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_geo").count());
+            assertEquals(0, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_temporal").count());
+            assertEquals(1, client.getDatabase(conf.getMongoDBName()).getCollection("ryatest_freetext").count());
+            assertEquals(4, client.getDatabase(conf.getMongoDBName()).getCollection(conf.getTriplesCollectionName()).count());
+        } finally {
+            conn.close();
+            sail.shutDown();
+        }
     }
 
     private void populateRya(final SailRepositoryConnection conn) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
----------------------------------------------------------------------
diff --git a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
index eaf3033..921acaa 100644
--- a/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
+++ b/sail/src/main/java/org/apache/rya/rdftriplestore/RdfCloudTripleStoreConnection.java
@@ -284,8 +284,8 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
         }
 
         try {
-            final List<Class<QueryOptimizer>> optimizers = conf.getOptimizers();
-            final Class<QueryOptimizer> pcjOptimizer = conf.getPcjOptimizer();
+            final List<Class<QueryOptimizer>> optimizers = queryConf.getOptimizers();
+            final Class<QueryOptimizer> pcjOptimizer = queryConf.getPcjOptimizer();
 
             if(pcjOptimizer != null) {
                 QueryOptimizer opt = null;
@@ -304,7 +304,7 @@ public class RdfCloudTripleStoreConnection extends SailConnectionBase {
             }
 
             final ParallelEvaluationStrategyImpl strategy = new ParallelEvaluationStrategyImpl(
-                    new StoreTripleSource(conf, ryaDAO), inferenceEngine, dataset, queryConf);
+                    new StoreTripleSource(queryConf, ryaDAO), inferenceEngine, dataset, queryConf);
 
                 (new BindingAssigner()).optimize(tupleExpr, dataset, bindings);
                 (new ConstantOptimizer(strategy)).optimize(tupleExpr, dataset,

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/157c0649/sail/src/test/java/org/apache/rya/RdfCloudTripleStoreConnectionTest.java
----------------------------------------------------------------------
diff --git a/sail/src/test/java/org/apache/rya/RdfCloudTripleStoreConnectionTest.java b/sail/src/test/java/org/apache/rya/RdfCloudTripleStoreConnectionTest.java
index f34129e..8c926db 100644
--- a/sail/src/test/java/org/apache/rya/RdfCloudTripleStoreConnectionTest.java
+++ b/sail/src/test/java/org/apache/rya/RdfCloudTripleStoreConnectionTest.java
@@ -23,7 +23,9 @@ import static org.apache.rya.api.RdfCloudTripleStoreConstants.NAMESPACE;
 import java.io.InputStream;
 import java.util.List;
 
-import junit.framework.TestCase;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.rya.accumulo.AccumuloRdfConfiguration;
 import org.apache.rya.accumulo.AccumuloRyaDAO;
 import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
@@ -32,10 +34,6 @@ import org.apache.rya.rdftriplestore.RdfCloudTripleStore;
 import org.apache.rya.rdftriplestore.RyaSailRepository;
 import org.apache.rya.rdftriplestore.inference.InferenceEngine;
 import org.apache.rya.rdftriplestore.namespace.NamespaceManager;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.openrdf.model.Literal;
 import org.openrdf.model.Model;
 import org.openrdf.model.Resource;
@@ -62,6 +60,8 @@ import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.rio.RDFFormat;
 import org.openrdf.rio.Rio;
 
+import junit.framework.TestCase;
+
 /**
  * Class RdfCloudTripleStoreConnectionTest
  * Date: Mar 3, 2011
@@ -1086,13 +1086,14 @@ public class RdfCloudTripleStoreConnectionTest extends TestCase {
         Update u = conn.prepareUpdate(QueryLanguage.SPARQL, sparqlUpdate);
         u.execute();
 
-        String query = "PREFIX  ex:  <http://www.example.org/exampleDocument#>\n" +
+        String query =
+                "PREFIX  ex:  <http://www.example.org/exampleDocument#>\n" +
                 "PREFIX  voc:  <http://www.example.org/vocabulary#>\n" +
                 "PREFIX  foaf:  <http://xmlns.com/foaf/0.1/>\n" +
                 "PREFIX  rdfs:  <http://www.w3.org/2000/01/rdf-schema#>\n" +
                 "\n" +
                 "SELECT * \n" +
-//                "FROM NAMED <http://www.example.org/exampleDocument#G1>\n" +
+//              "FROM NAMED <http://www.example.org/exampleDocument#G1>\n" +
                 "WHERE\n" +
                 "{\n" +
                 "  GRAPH ex:G1\n" +



[15/17] incubator-rya git commit: RYA-414 Code review. closes #256

Posted by dl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/4576f556/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
deleted file mode 100644
index 0f8497e..0000000
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/mongo/MongoTemporalIndexerTest.java
+++ /dev/null
@@ -1,722 +0,0 @@
-package org.apache.rya.indexing.mongo;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-import static org.apache.rya.api.resolver.RdfToRyaConversions.convertStatement;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-import java.util.HashSet;
-
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.rya.indexing.StatementConstraints;
-import org.apache.rya.indexing.TemporalInstant;
-import org.apache.rya.indexing.TemporalInstantRfc3339;
-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.MongoTestBase;
-import org.junit.Test;
-import org.openrdf.model.Statement;
-import org.openrdf.model.URI;
-import org.openrdf.model.ValueFactory;
-import org.openrdf.model.impl.StatementImpl;
-import org.openrdf.model.impl.ValueFactoryImpl;
-import org.openrdf.model.vocabulary.RDFS;
-import org.openrdf.query.QueryEvaluationException;
-
-import com.mongodb.DB;
-import com.mongodb.DBCollection;
-import com.mongodb.DBCursor;
-import com.mongodb.DBObject;
-import com.mongodb.MongoException;
-import com.mongodb.MongoSecurityException;
-
-import info.aduna.iteration.CloseableIteration;
-
-/**
- * JUnit tests for TemporalIndexer and it's implementation MongoTemporalIndexer
- *
- * If you enjoy this test, please read RyaTemporalIndexerTest and YagoKBTest, which contain
- * many example SPARQL queries and updates and attempts to test independently of Mongo:
- *
- *     extras/indexingSail/src/test/java/org.apache/rya/indexing/Mongo/RyaTemporalIndexerTest.java
- *     {@link org.apache.rya.indexing.Mongo.RyaTemporalIndexerTest}
- *     {@link org.apache.rya.indexing.Mongo.YagoKBTest.java}
- *
- * Remember, this class in instantiated fresh for each @test method.
- * so fields are reset, unless they are static.
- *
- * These are covered:
- *   Instance {before, equals, after} given Instance
- *   Instance {before, after, inside} given Interval
- *   Instance {hasBeginning, hasEnd} given Interval
- * And a few more.
- *
- */
-public final class MongoTemporalIndexerTest extends MongoTestBase {
-
-    private static final String URI_PROPERTY_EVENT_TIME = "Property:event:time";
-    private static final String URI_PROPERTY_CIRCA = "Property:circa";
-    private static final String URI_PROPERTY_AT_TIME = "Property:atTime";
-    private static final StatementConstraints EMPTY_CONSTRAINTS = new StatementConstraints();
-
-    // Assign this in setUpBeforeClass, store them in each test.
-    // setup() deletes table before each test.
-    static final Statement spo_B00_E01;
-    static final Statement spo_B03_E20;
-    static final Statement spo_B02_E29;
-    static final Statement spo_B02_E30;
-    static final Statement spo_B02_E40;
-    static final Statement spo_B02_E31;
-    static final Statement spo_B29_E30;
-    static final Statement spo_B30_E32;
-
-    // Instants:
-    static final Statement spo_B02;
-    static final int SERIES_OF_SECONDS = 41;
-    static final Statement seriesSpo[] = new Statement[SERIES_OF_SECONDS];
-
-    // These are shared for several tests. Only the seconds are different.
-    // tvB03_E20 read as: interval Begins 3 seconds, ends at 20 seconds
-    static final TemporalInterval tvB00_E01 = new TemporalInterval(makeInstant(00), makeInstant(01));
-    static final TemporalInterval tvB29_E30= new TemporalInterval(makeInstant(29), makeInstant(30));
-    static final TemporalInterval tvB30_E32= new TemporalInterval(makeInstant(30), makeInstant(32));
-    static final TemporalInterval tvB03_E20 = new TemporalInterval(makeInstant(03), makeInstant(20));
-    // 30 seconds, Begins earlier, ends later
-    static final TemporalInterval tvB02_E30= new TemporalInterval(makeInstant(02), makeInstant(30));
-    // use for interval after
-    static final TemporalInterval tvB02_E29= new TemporalInterval(makeInstant(02), makeInstant(29));
-    // same as above, but ends in the middle
-    static final TemporalInterval tvB02_E31 = new TemporalInterval(makeInstant(02), makeInstant(31));
-    // same as above, but ends even later
-    static final TemporalInterval tvB02_E40 = new TemporalInterval(makeInstant(02), makeInstant(40));
-    // instant, match beginnings of several above, before tiB03_E20
-    static final TemporalInstant tsB02 = makeInstant(02);
-    // instant, after all above
-    static final TemporalInstant tsB04 = makeInstant(04);
-
-    // Create a series of instants about times 0 - 40 seconds
-    static final TemporalInstant seriesTs[];
-    static {
-        seriesTs = new TemporalInstant[SERIES_OF_SECONDS];
-        for (int i = 0; i <= 40; i++) {
-            seriesTs[i] = makeInstant(i);
-        }
-    };
-
-    /**
-     * Make an uniform instant with given seconds.
-     */
-    static TemporalInstant makeInstant(final int secondsMakeMeUnique) {
-        return new TemporalInstantRfc3339(2015, 12, 30, 12, 00, secondsMakeMeUnique);
-    }
-
-    static {
-        // Setup the statements only once. Each test will store some of these in there own index table.
-        final ValueFactory vf = new ValueFactoryImpl();
-        final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-        // tiB03_E20 read as: time interval that Begins 3 seconds, ends at 20 seconds,
-        // Each time element the same, except seconds. year, month, .... minute are the same for each statement below.
-        spo_B00_E01 = new StatementImpl(vf.createURI("foo:event0"), pred1_atTime, vf.createLiteral(tvB00_E01.toString()));
-        spo_B02_E29 = new StatementImpl(vf.createURI("foo:event2"), pred1_atTime, vf.createLiteral(tvB02_E29.toString()));
-        spo_B02_E30 = new StatementImpl(vf.createURI("foo:event2"), pred1_atTime, vf.createLiteral(tvB02_E30.toString()));
-        spo_B02_E31 = new StatementImpl(vf.createURI("foo:event3"), pred1_atTime, vf.createLiteral(tvB02_E31.toString()));
-        spo_B02_E40 = new StatementImpl(vf.createURI("foo:event4"), pred1_atTime, vf.createLiteral(tvB02_E40.toString()));
-        spo_B03_E20 = new StatementImpl(vf.createURI("foo:event5"), pred1_atTime, vf.createLiteral(tvB03_E20.toString()));
-        spo_B29_E30 = new StatementImpl(vf.createURI("foo:event1"), pred1_atTime, vf.createLiteral(tvB29_E30.toString()));
-        spo_B30_E32 = new StatementImpl(vf.createURI("foo:event1"), pred1_atTime, vf.createLiteral(tvB30_E32.toString()));
-        spo_B02 = new StatementImpl(vf.createURI("foo:event6"), pred1_atTime, vf.createLiteral(tsB02.getAsReadable()));
-
-        // Create statements about time instants 0 - 40 seconds
-        for (int i = 0; i < seriesTs.length; i++) {
-            seriesSpo[i] = new StatementImpl(vf.createURI("foo:event0" + i), pred1_atTime, vf.createLiteral(seriesTs[i].getAsReadable()));
-        }
-    }
-
-    @Override
-    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
-        // This is from http://linkedevents.org/ontology
-        // and http://motools.sourceforge.net/event/event.html
-        conf.setStrings(ConfigUtils.TEMPORAL_PREDICATES_LIST, ""
-                + URI_PROPERTY_AT_TIME + ","
-                + URI_PROPERTY_CIRCA + ","
-                + URI_PROPERTY_EVENT_TIME);
-    }
-
-    /**
-     * Test method for {@link MongoTemporalIndexer#storeStatement(convertStatement(org.openrdf.model.Statement)}
-     */
-    @Test
-    public void testStoreStatement() throws IOException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
-
-            // Should not be stored because they are not in the predicate list
-            final String validDateStringWithThirteens = "1313-12-13T13:13:13Z";
-            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj1"), RDFS.LABEL, vf.createLiteral(validDateStringWithThirteens))));
-
-            final String invalidDateString = "ThisIsAnInvalidDate";
-            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj2"), pred1_atTime, vf.createLiteral(invalidDateString))));
-
-            // These are different datetimes instant but from different time zones.
-            // This is an arbitrary zone, BRST=Brazil, better if not local.
-            // same as "2015-01-01T01:59:59Z"
-            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
-            // next year, same as "2017-01-01T01:59:59Z"
-            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
-
-            // These should be stored because they are in the predicate list.
-            // BUT they will get converted to the same exact datetime in UTC.
-            final Statement s3 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
-            final Statement s4 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
-            tIndexer.storeStatement(convertStatement(s3));
-            tIndexer.storeStatement(convertStatement(s4));
-
-            // This should not be stored because the object is not a literal
-            tIndexer.storeStatement(convertStatement(new StatementImpl(vf.createURI("foo:subj5"), pred1_atTime, vf.createURI("in:valid"))));
-
-            printTables(tIndexer, "junit testing: Temporal entities stored in testStoreStatement");
-            assertEquals(2, tIndexer.getCollection().find().count());
-        }
-    }
-
-    @Test
-    public void testDelete() throws IOException, MongoException, TableNotFoundException, TableExistsException, NoSuchAlgorithmException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            final ValueFactory vf = new ValueFactoryImpl();
-
-            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-            final URI pred2_circa = vf.createURI(URI_PROPERTY_CIRCA);
-
-            final String testDate2014InBRST = "2014-12-31T23:59:59-02:00";
-            final String testDate2016InET = "2016-12-31T20:59:59-05:00";
-
-            // These should be stored because they are in the predicate list.
-            // BUT they will get converted to the same exact datetime in UTC.
-            final Statement s1 = new StatementImpl(vf.createURI("foo:subj3"), pred1_atTime, vf.createLiteral(testDate2014InBRST));
-            final Statement s2 = new StatementImpl(vf.createURI("foo:subj4"), pred2_circa, vf.createLiteral(testDate2016InET));
-            tIndexer.storeStatement(convertStatement(s1));
-            tIndexer.storeStatement(convertStatement(s2));
-
-            final String dbName = conf.getMongoDBName();
-            final DB db = super.getMongoClient().getDB(dbName);
-            DBCollection collection = db.getCollection(conf.get(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya") + tIndexer.getCollectionName());
-
-            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete before delete");
-            assertEquals("Number of rows stored.", 2, collection.count()); // 4 index entries per statement
-
-            tIndexer.deleteStatement(convertStatement(s1));
-            tIndexer.deleteStatement(convertStatement(s2));
-
-            printTables(tIndexer, "junit testing: Temporal entities stored in testDelete after delete");
-            assertEquals("Number of rows stored after delete.", 0, collection.count());
-        }
-    }
-
-    /**
-     * Test instant after a given instant.
-     * From the series: instant {equal, before, after} instant
-     * @throws MongoSecurityException
-     * @throws MongoException
-     * @throws TableNotFoundException
-     */
-    @Test
-    public void testQueryInstantAfterInstant() throws IOException, QueryEvaluationException, TableNotFoundException, MongoException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-            final int searchForSeconds = 4;
-            final int expectedResultCount = 9;
-            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count + 1]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-    /**
-     * Test instant before a given instant.
-     * From the series: instant {equal, before, after} instant
-     */
-    @Test
-    public void testQueryInstantBeforeInstant() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-            final int searchForSeconds = 4;
-            final int expectedResultCount = 4;
-            for (int s = 0; s <= searchForSeconds + 15; s++) { // <== logic here
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-
-            iter = tIndexer.queryInstantBeforeInstant(seriesTs[searchForSeconds], EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test instant before given interval.
-     * From the series:  Instance {before, after, inside} given Interval
-     */
-    @Test
-    public void testQueryInstantBeforeInterval() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-            final TemporalInterval searchForSeconds = tvB02_E31;
-            final int expectedResultCount = 2; // 00 and 01 seconds.
-            for (int s = 0; s <= 40; s++) { // <== logic here
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantBeforeInterval(searchForSeconds, EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[count]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test instant after given interval.
-     * Instance {before, after, inside} given Interval
-     */
-    @Test
-    public void testQueryInstantAfterInterval() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-            final TemporalInterval searchAfterInterval = tvB02_E31; // from 2 to 31 seconds
-            final int endingSeconds = 31;
-            final int expectedResultCount = 9; // 32,33,...,40 seconds.
-            for (int s = 0; s <= endingSeconds + expectedResultCount; s++) { // <== logic here
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantAfterInterval(searchAfterInterval, EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[count + endingSeconds + 1]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test instant inside given interval.
-     * Instance {before, after, inside} given Interval
-     */
-    @Test
-    public void testQueryInstantInsideInterval() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-            final int beginningSeconds = 2; // <== logic here, and next few lines.
-            final int endingSeconds = 31;
-            final int expectedResultCount = endingSeconds - beginningSeconds - 1; // 3,4,...,30 seconds.
-            for (int s = 0; s <= 40; s++) {
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantInsideInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[count + beginningSeconds + 1]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test instant is the Beginning of the given interval.
-     * from the series: Instance {hasBeginning, hasEnd} Interval
-     */
-    @Test
-    public void testQueryInstantHasBeginningInterval() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-            final int searchSeconds = 2; // <== logic here, and next few lines.
-            final int expectedResultCount = 1; // 2 seconds.
-            for (int s = 0; s <= 10; s++) {
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantHasBeginningInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test instant is the end of the given interval.
-     * from the series: Instance {hasBeginning, hasEnd} Interval
-     */
-    @Test
-    public void testQueryInstantHasEndInterval()  throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instants for s seconds after the uniform time.
-            final TemporalInterval searchInsideInterval = tvB02_E31; // from 2 to 31 seconds
-            final int searchSeconds = 31; // <== logic here, and next few lines.
-            final int expectedResultCount = 1; // 31 seconds.
-            for (int s = 0; s <= 40; s++) {
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryInstantHasEndInterval(searchInsideInterval, EMPTY_CONSTRAINTS);
-            int count = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[searchSeconds]; // <== logic here
-                assertTrue("Should match: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                count++;
-            }
-            assertEquals("Should find count of rows.", expectedResultCount, count);
-        }
-    }
-
-    /**
-     * Test method for
-     * {@link org.apache.rya.indexing.Mongo.temporal.MongoTemporalIndexer#queryIntervalEquals(TemporalInterval, StatementConstraints)}
-     * .
-     * @throws IOException
-     * @throws QueryEvaluationException
-     *
-     */
-    @Test
-    public void testQueryIntervalEquals() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-            tIndexer.storeStatement(convertStatement(seriesSpo[4])); // instance at 4 seconds
-
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryIntervalEquals(tvB02_E40, EMPTY_CONSTRAINTS);
-            // Should be found twice:
-            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but actually returned empty results. spo_B02_E40=" + spo_B02_E40, iter.hasNext());
-            assertTrue("queryIntervalEquals: spo_B02_E40 should be found, but does not match.", spo_B02_E40.equals(iter.next()));
-            assertFalse("queryIntervalEquals: Find no more than one, but actually has more.", iter.hasNext());
-        }
-    }
-
-    /**
-     * Test interval before a given interval, for method:
-     * {@link MongoTemporalIndexer#queryIntervalBefore(TemporalInterval, StatementConstraints)}.
-     *
-     * @throws IOException
-     * @throws QueryEvaluationException
-     */
-    @Test
-    public void testQueryIntervalBefore() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            tIndexer.storeStatement(convertStatement(spo_B00_E01));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            // instants should be ignored.
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
-            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
-            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
-
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryIntervalBefore(tvB02_E31, EMPTY_CONSTRAINTS);
-            // Should be found twice:
-            assertTrue("spo_B00_E01 should be found, but actually returned empty results. spo_B00_E01=" + spo_B00_E01, iter.hasNext());
-            assertTrue("spo_B00_E01 should be found, but found another.", spo_B00_E01.equals(iter.next()));
-            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
-        }
-    }
-
-    /**
-     * interval is after the given interval.  Find interval beginnings after the endings of the given interval.
-     * {@link MongoTemporalIndexer#queryIntervalAfter(TemporalInterval, StatementContraints).
-     *
-     * @throws IOException
-     * @throws QueryEvaluationException
-     */
-    @Test
-    public void testQueryIntervalAfter() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            tIndexer.storeStatement(convertStatement(spo_B00_E01));
-            tIndexer.storeStatement(convertStatement(spo_B02_E29)); //<- after this one.
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B29_E30));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-            // instants should be ignored.
-            tIndexer.storeStatement(convertStatement(spo_B02));
-            tIndexer.storeStatement(convertStatement(seriesSpo[1])); // instance at 1 seconds
-            tIndexer.storeStatement(convertStatement(seriesSpo[2]));
-            tIndexer.storeStatement(convertStatement(seriesSpo[31]));
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            iter = tIndexer.queryIntervalAfter(tvB02_E29, EMPTY_CONSTRAINTS);
-            // Should be found twice:
-            assertTrue("spo_B30_E32 should be found, but actually returned empty results. spo_B30_E32=" + spo_B30_E32, iter.hasNext());
-            final Statement s = iter.next();
-            assertTrue("spo_B30_E32 should be found, but found another. spo_B30_E32="+spo_B30_E32+", but found="+s, spo_B30_E32.equals(s));
-            assertFalse("Find no more than one, but actually has more.", iter.hasNext());
-        }
-    }
-
-    /**
-     * Test instant after a given instant WITH two different predicates as constraints.
-     */
-    @Test
-    public void testQueryWithMultiplePredicates() throws IOException, QueryEvaluationException {
-        try(MongoTemporalIndexer tIndexer = new MongoTemporalIndexer()) {
-            tIndexer.setConf(conf);
-            tIndexer.init();
-
-            // tiB02_E30 read as: Begins 2 seconds, ends at 30 seconds
-            // these should not match as they are not instances.
-            tIndexer.storeStatement(convertStatement(spo_B03_E20));
-            tIndexer.storeStatement(convertStatement(spo_B02_E30));
-            tIndexer.storeStatement(convertStatement(spo_B02_E40));
-            tIndexer.storeStatement(convertStatement(spo_B02_E31));
-            tIndexer.storeStatement(convertStatement(spo_B30_E32));
-
-            // seriesSpo[s] and seriesTs[s] are statements and instant for s seconds after the uniform time.
-            final int searchForSeconds = 4;
-            final int expectedResultCount = 9;
-            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-                tIndexer.storeStatement(convertStatement(seriesSpo[s]));
-            }
-            final ValueFactory vf = new ValueFactoryImpl();
-            final URI pred3_CIRCA_ = vf.createURI(URI_PROPERTY_CIRCA);  // this one to ignore.
-            final URI pred2_eventTime = vf.createURI(URI_PROPERTY_EVENT_TIME);
-            final URI pred1_atTime = vf.createURI(URI_PROPERTY_AT_TIME);
-
-            // add the predicate = EventTime ; Store in an array for verification.
-            final Statement[] SeriesTs_EventTime = new Statement[expectedResultCount+1];
-            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-                final Statement statement = new StatementImpl(vf.createURI("foo:EventTimeSubj0" + s), pred2_eventTime, vf.createLiteral(seriesTs[s].getAsReadable()));
-                tIndexer.storeStatement(convertStatement(statement));
-                if (s>searchForSeconds) {
-                    SeriesTs_EventTime[s - searchForSeconds -1 ] = statement;
-                }
-            }
-            // add the predicate = CIRCA ; to be ignored because it is not in the constraints.
-            for (int s = 0; s <= searchForSeconds + expectedResultCount; s++) { // <== logic here
-                final Statement statement = new StatementImpl(vf.createURI("foo:CircaEventSubj0" + s), pred3_CIRCA_, vf.createLiteral(seriesTs[s].getAsReadable()));
-                tIndexer.storeStatement(convertStatement(statement));
-            }
-
-            CloseableIteration<Statement, QueryEvaluationException> iter;
-            final StatementConstraints constraints = new StatementConstraints();
-            constraints.setPredicates(new HashSet<>(Arrays.asList( pred2_eventTime,  pred1_atTime )));
-
-            iter = tIndexer.queryInstantAfterInstant(seriesTs[searchForSeconds], constraints); // EMPTY_CONSTRAINTS);//
-            int count_AtTime = 0;
-            int count_EventTime = 0;
-            while (iter.hasNext()) {
-                final Statement s = iter.next();
-                final Statement nextExpectedStatement = seriesSpo[searchForSeconds + count_AtTime + 1]; // <== logic here
-                if (s.getPredicate().equals(pred1_atTime)) {
-                    assertTrue("Should match atTime: " + nextExpectedStatement + " == " + s, nextExpectedStatement.equals(s));
-                    count_AtTime++;
-                }
-                else if (s.getPredicate().equals(pred2_eventTime)) {
-                    assertTrue("Should match eventTime: " + SeriesTs_EventTime[count_EventTime] + " == " + s, SeriesTs_EventTime[count_EventTime].equals(s));
-                    count_EventTime++;
-                } else {
-                    assertTrue("This predicate should not be returned: "+s, false);
-                }
-
-            }
-
-            assertEquals("Should find count of atTime    rows.", expectedResultCount, count_AtTime);
-            assertEquals("Should find count of eventTime rows.", expectedResultCount, count_EventTime);
-        }
-    }
-
-    /**
-     * Print and gather statistics on the entire index table.
-     *
-     * @param description
-     *            Printed to the console to find the test case.
-     * @param out
-     *            null or System.out or other output to send a listing.
-     * @param statistics
-     *            Hashes, sums, and counts for assertions.
-     * @return Count of entries in the index table.
-     * @throws IOException
-     */
-    public void printTables(MongoTemporalIndexer tIndexer, final String description) throws IOException {
-        System.out.println("-- start printTables() -- " + description);
-        System.out.println("Reading : " + tIndexer.getCollection().getFullName());
-        final DBCursor cursor = tIndexer.getCollection().find();
-        while(cursor.hasNext()) {
-            final DBObject dbo = cursor.next();
-            System.out.println(dbo.toString());
-        }
-        System.out.println();
-    }
-}