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:29 UTC

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

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();
+		}
+	}
 }