http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 0bd9261..7da9918 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
@@ -32,34 +32,34 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
-import javax.annotation.Nullable;
-import javax.annotation.ParametersAreNonnullByDefault;
-
 import org.apache.hadoop.conf.Configuration;
+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.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.CloseableIterator;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
-import org.apache.rya.indexing.entity.storage.TypeStorage;
 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.ConvertingCursor;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoSecondaryIndex;
 import org.openrdf.model.URI;
 import org.openrdf.model.vocabulary.RDF;
 
 import com.google.common.base.Objects;
 
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.mongodb.MongoDBRdfConfiguration;
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
  * A base class that may be used to update an {@link EntityStorage} as new
  * {@link RyaStatement}s are added to/removed from the Rya instance.
  */
-@ParametersAreNonnullByDefault
-public abstract class BaseEntityIndexer implements EntityIndexer {
+@DefaultAnnotation(NonNull.class)
+public abstract class BaseEntityIndexer implements EntityIndexer, 
MongoSecondaryIndex {
 
     /**
      * When this URI is the Predicate of a Statement, it indicates a {@link 
Type} for an {@link Entity}.
@@ -70,24 +70,8 @@ public abstract class BaseEntityIndexer implements 
EntityIndexer {
     private final AtomicReference<EntityStorage> entities = new 
AtomicReference<>();
     private final AtomicReference<TypeStorage> types = new AtomicReference<>();
 
-    /**
-     * 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.
-     */
-    public abstract @Nullable EntityStorage getEntityStorage(Configuration 
conf);
-
-    /**
-     * 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 abstract @Nullable TypeStorage getTypeStorage(Configuration conf);
-
     @Override
-    public void setConf(Configuration conf) {
+    public void setConf(final Configuration conf) {
         requireNonNull(conf);
         entities.set( getEntityStorage(conf) );
         types.set( getTypeStorage(conf) );
@@ -99,13 +83,13 @@ public abstract class BaseEntityIndexer implements 
EntityIndexer {
     }
 
     @Override
-    public void storeStatement(RyaStatement statement) throws IOException {
+    public void storeStatement(final RyaStatement statement) throws 
IOException {
         requireNonNull(statement);
         storeStatements( singleton(statement) );
     }
 
     @Override
-    public void storeStatements(Collection<RyaStatement> statements) throws 
IOException {
+    public void storeStatements(final Collection<RyaStatement> statements) 
throws IOException {
         requireNonNull(statements);
 
         final Map<RyaURI,List<RyaStatement>> groupedBySubject = 
statements.stream()
@@ -162,7 +146,7 @@ public abstract class BaseEntityIndexer implements 
EntityIndexer {
                     final RyaURI propertyName = statement.getPredicate();
                     final RyaType propertyValue = statement.getObject();
 
-                    try(final CloseableIterator<Type> typesIt = 
types.search(propertyName)) {
+                    try(final ConvertingCursor<Type> typesIt = 
types.search(propertyName)) {
                         // Set the Property for each type that includes the 
Statement's predicate.
                         while(typesIt.hasNext()) {
                             final RyaURI typeId = typesIt.next().getId();
@@ -180,7 +164,7 @@ public abstract class BaseEntityIndexer implements 
EntityIndexer {
     }
 
     @Override
-    public void deleteStatement(RyaStatement statement) throws IOException {
+    public void deleteStatement(final RyaStatement statement) throws 
IOException {
         requireNonNull(statement);
 
         final EntityStorage entities = this.entities.get();
@@ -254,7 +238,7 @@ public abstract class BaseEntityIndexer implements 
EntityIndexer {
     }
 
     @Override
-    public void dropGraph(RyaURI... graphs) {
+    public void dropGraph(final RyaURI... graphs) {
         // We do not support graphs when performing entity centric indexing.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 9ef5dfd..48cb0b1 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,10 +18,13 @@
  */
 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;
+import org.apache.rya.indexing.entity.storage.TypeStorage;
 
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.persist.index.RyaSecondaryIndexer;
+import edu.umd.cs.findbugs.annotations.Nullable;
 
 /**
  * Updates the {@link Entity}s that are in a {@link EntityStorage} when new
@@ -29,4 +32,19 @@ import mvm.rya.api.persist.index.RyaSecondaryIndexer;
  */
 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.
+     */
+    public @Nullable EntityStorage getEntityStorage(Configuration conf);
+
+    /**
+     * 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);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityUpdater.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityUpdater.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityUpdater.java
index 4bb003f..fb5e957 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityUpdater.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/entity/update/EntityUpdater.java
@@ -23,20 +23,20 @@ import static java.util.Objects.requireNonNull;
 import java.util.Optional;
 import java.util.function.Function;
 
-import javax.annotation.ParametersAreNonnullByDefault;
-
+import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Entity;
 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 mvm.rya.api.domain.RyaURI;
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
  * Performs update operations over an {@link EntityStorage}.
  */
-@ParametersAreNonnullByDefault
+@DefaultAnnotation(NonNull.class)
 public class EntityUpdater {
 
     private final EntityStorage storage;
@@ -46,7 +46,7 @@ public class EntityUpdater {
      *
      * @param storage - The storage this updater operates over. (not null)
      */
-    public EntityUpdater(EntityStorage storage) {
+    public EntityUpdater(final EntityStorage storage) {
         this.storage = requireNonNull(storage);
     }
 
@@ -59,7 +59,7 @@ public class EntityUpdater {
      *   the new state of the Entity. (not null)
      * @throws EntityStorageException A non-recoverable error has caused the 
update to fail.
      */
-    public void update(RyaURI subject, EntityMutator mutator) throws 
EntityStorageException {
+    public void update(final RyaURI subject, final EntityMutator mutator) 
throws EntityStorageException {
         requireNonNull(subject);
         requireNonNull(mutator);
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 71b5198..84eebaa 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,8 +18,6 @@
  */
 package org.apache.rya.indexing.entity.update.mongo;
 
-import javax.annotation.ParametersAreNonnullByDefault;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
 import org.apache.rya.indexing.entity.storage.TypeStorage;
@@ -27,29 +25,57 @@ 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 com.mongodb.MongoClient;
 
-import mvm.rya.mongodb.MongoConnectorFactory;
-import mvm.rya.mongodb.MongoDBRdfConfiguration;
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
 
 /**
  * A Mongo DB implementation of {@link EntityIndexer}.
  */
-@ParametersAreNonnullByDefault
+@DefaultAnnotation(NonNull.class)
 public class MongoEntityIndexer extends BaseEntityIndexer {
+    private MongoClient client;
 
     @Override
-    public EntityStorage getEntityStorage(Configuration conf) {
-        final MongoClient mongoClient = 
MongoConnectorFactory.getMongoClient(conf);
+    public EntityStorage getEntityStorage(final Configuration conf) {
+        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) 
conf;
+        if (client == null) {
+            if(mongoConf.getMongoClient() != null) {
+                client = mongoConf.getMongoClient();
+            } else {
+                client = MongoConnectorFactory.getMongoClient(conf);
+            }
+        }
         final String ryaInstanceName = new 
MongoDBRdfConfiguration(conf).getMongoDBName();
-        return new MongoEntityStorage(mongoClient, ryaInstanceName);
+        return new MongoEntityStorage(client, ryaInstanceName);
     }
 
     @Override
-    public TypeStorage getTypeStorage(Configuration conf) {
-        final MongoClient mongoClient = 
MongoConnectorFactory.getMongoClient(conf);
+    public TypeStorage getTypeStorage(final Configuration conf) {
+        final MongoDBRdfConfiguration mongoConf = (MongoDBRdfConfiguration) 
conf;
+        if (client == null) {
+            if(mongoConf.getMongoClient() != null) {
+                client = mongoConf.getMongoClient();
+            } else {
+                client = MongoConnectorFactory.getMongoClient(conf);
+            }
+        }
         final String ryaInstanceName = new 
MongoDBRdfConfiguration(conf).getMongoDBName();
-        return new MongoTypeStorage(mongoClient, ryaInstanceName);
+        return new MongoTypeStorage(client, ryaInstanceName);
+    }
+
+
+    @Override
+    public void init() {
+        //nothing to init.
+    }
+
+    @Override
+    public void setClient(final MongoClient client) {
+        this.client = client;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcher.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcher.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcher.java
index 65c1c9d..0f34030 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcher.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcher.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -64,7 +82,7 @@ public abstract class AbstractExternalSetMatcher<T extends 
ExternalSet> implemen
     protected TupleExpr tuple;
     protected Set<TupleExpr> unmatched;
     protected Set<Filter> filters;
-    private QuerySegmentFactory<T> factory = new QuerySegmentFactory<T>();
+    private final QuerySegmentFactory<T> factory = new 
QuerySegmentFactory<T>();
 
     /**
      * Matches {@link QuerySegment} with underlying QuerySegment. If match
@@ -85,7 +103,7 @@ public abstract class AbstractExternalSetMatcher<T extends 
ExternalSet> implemen
     /**
      * In following method, order is determined by the order in which the node
      * appear in the query.
-     * 
+     *
      * @return - an ordered view of the QueryModelNodes appearing tuple
      *
      */
@@ -110,14 +128,14 @@ public abstract class AbstractExternalSetMatcher<T 
extends ExternalSet> implemen
     }
 
     @Override
-    public QuerySegment<T> nodeToQuerySegment(QueryModelNode node) {
+    public QuerySegment<T> nodeToQuerySegment(final QueryModelNode node) {
         return factory.getQuerySegment(node);
     }
 
     @Override
     public List<QueryModelNode> getAllUnmatchedNodes() {
-        List<QueryModelNode> unmatched = new ArrayList<>();
-        for (QueryModelNode node : segmentNodeList) {
+        final List<QueryModelNode> unmatched = new ArrayList<>();
+        for (final QueryModelNode node : segmentNodeList) {
             if (!(node instanceof ExternalSet)) {
                 unmatched.add(node);
             }
@@ -136,8 +154,8 @@ public abstract class AbstractExternalSetMatcher<T extends 
ExternalSet> implemen
             if (q instanceof UnaryTupleOperator || q instanceof 
BinaryTupleOperator) {
                 unmatched.add((TupleExpr) q);
             } else if (q instanceof FlattenedOptional) {
-                FlattenedOptional opt = (FlattenedOptional) q;
-                TupleExpr rightArg = opt.getRightArg();
+                final FlattenedOptional opt = (FlattenedOptional) q;
+                final TupleExpr rightArg = opt.getRightArg();
                 if (rightArg instanceof UnaryTupleOperator || rightArg 
instanceof BinaryTupleOperator) {
                     unmatched.add(rightArg);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcherFactory.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcherFactory.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcherFactory.java
index d4cd2e5..ca7ede4 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcherFactory.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/AbstractExternalSetMatcherFactory.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -28,7 +46,7 @@ import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
  * given query.
  */
 public abstract class AbstractExternalSetMatcherFactory<T extends ExternalSet> 
{
-    
+
     public ExternalSetMatcher<T> getMatcher(final QuerySegment<T> segment) {
         if(segment instanceof JoinSegment<?>) {
             return getJoinSegmentMatcher((JoinSegment<T>) segment);
@@ -38,9 +56,9 @@ public abstract class AbstractExternalSetMatcherFactory<T 
extends ExternalSet> {
             throw new IllegalArgumentException("Invalid Segment.");
         }
     }
-    
+
     protected abstract ExternalSetMatcher<T> 
getJoinSegmentMatcher(JoinSegment<T> segment);
-    
+
     protected abstract ExternalSetMatcher<T> 
getOptionalJoinSegmentMatcher(OptionalJoinSegment<T> segment);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/BasicRater.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/BasicRater.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/BasicRater.java
index a26c872..6166d8a 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/BasicRater.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/BasicRater.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetConverter.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetConverter.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetConverter.java
index 2e973e9..900e6a8 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetConverter.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetConverter.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -21,10 +39,10 @@ package org.apache.rya.indexing.external.matching;
 import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
 
 public interface ExternalSetConverter<T extends ExternalSet> {
-    
+
     /**
      * Converts the {@link ExternalSet} to a {@link QuerySegment}
-     * 
+     *
      * @param set - ExternalSet to be converted
      * @return QuerySegment derived from ExternalSet
      */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetProvider.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetProvider.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetProvider.java
index 801bb87..e2d6b34 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetProvider.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/ExternalSetProvider.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -32,19 +50,19 @@ public interface ExternalSetProvider<T extends ExternalSet> 
{
 
     /**
      * Extract all {@link ExternalSet}s from specified QuerySegment.
-     * 
+     *
      * @param segment
      * @return - List of ExternalSets
      */
     public List<T> getExternalSets(QuerySegment<T> segment);
-    
+
     /**
      * Extract an Iterator over Lists of ExternalSets. This allows an 
ExtenalSetProvider to pass back
      * different combinations of ExternalSets for the purposes of query 
optimization.
-     * 
+     *
      * @param segment
      * @return - Iterator over different combinations of ExternalSets
      */
     public Iterator<List<T>> getExternalSetCombos(QuerySegment<T> segment);
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/MatcherUtilities.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/MatcherUtilities.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/MatcherUtilities.java
index b4a48eb..1f8e287 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/MatcherUtilities.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/MatcherUtilities.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -41,5 +59,5 @@ public class MatcherUtilities {
             return false;
         }
     }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QueryNodeListRater.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QueryNodeListRater.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QueryNodeListRater.java
index 115a12b..e8b1553 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QueryNodeListRater.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QueryNodeListRater.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -24,7 +42,7 @@ import org.openrdf.query.algebra.QueryModelNode;
 
 /**
  * Class used for determining an optimal query plan.  It assigns a score
- * between 0 and 1 to a list of QueryModelNodes.  A lower score indicates 
+ * between 0 and 1 to a list of QueryModelNodes.  A lower score indicates
  * that the List represents a better collection of nodes for building a query
  * plan.  Usually, the specified List is compared to a base List (original 
query),
  * and the specified List (mutated query) is compared to the original to 
determine
@@ -33,7 +51,7 @@ import org.openrdf.query.algebra.QueryModelNode;
  *
  */
 public interface QueryNodeListRater {
-    
+
     public double rateQuerySegment(List<QueryModelNode> eNodes);
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegment.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegment.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegment.java
index 8603a68..a8f687e 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegment.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegment.java
@@ -28,11 +28,12 @@ import org.openrdf.query.algebra.QueryModelNode;
 import org.openrdf.query.algebra.evaluation.impl.ExternalSet;
 
 /**
- * A QuerySegment represents a subset of a query to be compared to PCJs for
+ * A QuerySegment represents a subset of a query to be compared to 
ExternalSets for
  * query matching. The QuerySegment is represented as a List, where the order 
of
  * the nodes in the list is determined by a Visitor as it traverses the Segment
  * from top down, visiting right children before left.
  *
+ * @param <T> - The type of ExternalSet that will replace a subset of nodes in 
this segment.
  */
 public interface QuerySegment<T extends ExternalSet> extends Cloneable{
 
@@ -62,14 +63,13 @@ public interface QuerySegment<T extends ExternalSet> 
extends Cloneable{
     /**
      * Sets List of {@link QueryModelNode}s representing this QuerySegment to
      * specified list
-     * 
+     *
      * @param nodes
      *            - nodes to set
      */
     public void setNodes(List<QueryModelNode> nodes);
 
     /**
-     *
      * @param nodeToReplace
      *            - QuerySegment representation of ExternalSet T to match with
      *            subset of this QuerySegment

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegmentFactory.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegmentFactory.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegmentFactory.java
index 4bf1faf..0c1eb7e 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegmentFactory.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/external/matching/QuerySegmentFactory.java
@@ -1,3 +1,21 @@
+/**
+ * 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.external.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -33,17 +51,17 @@ import com.google.common.base.Preconditions;
  */
 public class QuerySegmentFactory<T extends ExternalSet> {
 
-    public QuerySegment<T> getQuerySegment(QueryModelNode node) {
+    public QuerySegment<T> getQuerySegment(final QueryModelNode node) {
         Preconditions.checkNotNull(node);
         if(node instanceof Filter) {
-            Filter filter = (Filter)node;
+            final Filter filter = (Filter)node;
             if(MatcherUtilities.segmentContainsLeftJoins(filter)) {
                 return new OptionalJoinSegment<T>(filter);
             } else {
                 return new JoinSegment<T>(filter);
             }
         } else if(node instanceof Join) {
-            Join join = (Join) node;
+            final Join join = (Join) node;
             if(MatcherUtilities.segmentContainsLeftJoins(join)) {
                 return new OptionalJoinSegment<T>(join);
             } else {
@@ -54,7 +72,7 @@ public class QuerySegmentFactory<T extends ExternalSet> {
         } else {
             throw new IllegalArgumentException("Node must be a Join, Filter, 
or LeftJoin");
         }
-        
+
     }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java
index 641830b..828ee4b 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/AccumuloIndexSetProvider.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.rya.indexing.pcj.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -62,7 +80,7 @@ import jline.internal.Preconditions;
  * This provider uses either user specified Accumulo configuration information 
or user a specified
  * List of ExternalTupleSets to populate an internal cache of 
ExternalTupleSets.  If Accumulo configuration
  * is provided, the provider connects to an instance of RyaDetails and 
populates the cache with
- * PCJs registered in RyaDetails.  
+ * PCJs registered in RyaDetails.
  *
  */
 public class AccumuloIndexSetProvider implements 
ExternalSetProvider<ExternalTupleSet> {
@@ -70,25 +88,25 @@ public class AccumuloIndexSetProvider implements 
ExternalSetProvider<ExternalTup
     private static final Logger log = 
Logger.getLogger(ExternalSetProvider.class);
     private static final PCJToSegmentConverter converter = new 
PCJToSegmentConverter();
     private List<ExternalTupleSet> indexCache;
-    private Configuration conf;
+    private final Configuration conf;
     private boolean init = false;
 
-    public AccumuloIndexSetProvider(Configuration conf) {
+    public AccumuloIndexSetProvider(final Configuration conf) {
         Preconditions.checkNotNull(conf);
         this.conf = conf;
     }
-    
-    public AccumuloIndexSetProvider(Configuration conf, List<ExternalTupleSet> 
indices) {
+
+    public AccumuloIndexSetProvider(final Configuration conf, final 
List<ExternalTupleSet> indices) {
         Preconditions.checkNotNull(conf);
         this.conf = conf;
-        this.indexCache = indices;
+        indexCache = indices;
         init = true;
     }
-    
+
     /**
-     * 
+     *
      * @return - size of underlying PCJ cache
-     * @throws Exception 
+     * @throws Exception
      */
     public int size() throws Exception {
         if(!init) {
@@ -103,37 +121,37 @@ public class AccumuloIndexSetProvider implements 
ExternalSetProvider<ExternalTup
      * @return List of PCJs for matching
      */
     @Override
-    public List<ExternalTupleSet> 
getExternalSets(QuerySegment<ExternalTupleSet> segment) {
+    public List<ExternalTupleSet> getExternalSets(final 
QuerySegment<ExternalTupleSet> segment) {
         try {
             if(!init) {
                 indexCache = 
PCJOptimizerUtilities.getValidPCJs(getAccIndices());
                 init = true;
             }
-            TupleExpr query = segment.getQuery().getTupleExpr();
-            IndexedExecutionPlanGenerator iep = new 
IndexedExecutionPlanGenerator(query, indexCache);
-            List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
-            List<ExternalTupleSet> tuples = new ArrayList<>();
-            for (ExternalTupleSet tuple: pcjs) {
-                QuerySegment<ExternalTupleSet> pcj = 
converter.setToSegment(tuple);
+            final TupleExpr query = segment.getQuery().getTupleExpr();
+            final IndexedExecutionPlanGenerator iep = new 
IndexedExecutionPlanGenerator(query, indexCache);
+            final List<ExternalTupleSet> pcjs = iep.getNormalizedIndices();
+            final List<ExternalTupleSet> tuples = new ArrayList<>();
+            for (final ExternalTupleSet tuple: pcjs) {
+                final QuerySegment<ExternalTupleSet> pcj = 
converter.setToSegment(tuple);
                 if (segment.containsQuerySegment(pcj)) {
                     tuples.add(tuple);
                 }
             }
             return tuples;
 
-        } catch (Exception e) {
+        } catch (final Exception e) {
             throw new RuntimeException(e);
         }
     }
-    
+
     /**
-     * @param segment - QuerySegment used to get relevant queries form index 
cache for matching 
-     * 
+     * @param segment - QuerySegment used to get relevant queries form index 
cache for matching
+     *
      * @return Iterator of Lists (combos) of PCJs used to build an optimal 
query plan
      */
     @Override
-    public Iterator<List<ExternalTupleSet>> 
getExternalSetCombos(QuerySegment<ExternalTupleSet> segment) {
-        ValidIndexCombinationGenerator comboGen = new 
ValidIndexCombinationGenerator(segment.getOrderedNodes());
+    public Iterator<List<ExternalTupleSet>> getExternalSetCombos(final 
QuerySegment<ExternalTupleSet> segment) {
+        final ValidIndexCombinationGenerator comboGen = new 
ValidIndexCombinationGenerator(segment.getOrderedNodes());
         return comboGen.getValidIndexCombos(getExternalSets(segment));
     }
 
@@ -204,8 +222,8 @@ public class AccumuloIndexSetProvider implements 
ExternalSetProvider<ExternalTup
                 index.add(new AccumuloIndexSet(indexSparqlString, conf, 
table));
             }
         }
-        
-        
+
+
         return index;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJExternalSetMatcherFactory.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJExternalSetMatcherFactory.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJExternalSetMatcherFactory.java
index e9bfbbe..c7449f5 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJExternalSetMatcherFactory.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJExternalSetMatcherFactory.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.rya.indexing.pcj.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -33,12 +51,12 @@ import 
org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 public class PCJExternalSetMatcherFactory extends 
AbstractExternalSetMatcherFactory<ExternalTupleSet> {
 
     @Override
-    protected ExternalSetMatcher<ExternalTupleSet> 
getJoinSegmentMatcher(JoinSegment<ExternalTupleSet> segment) {
+    protected ExternalSetMatcher<ExternalTupleSet> getJoinSegmentMatcher(final 
JoinSegment<ExternalTupleSet> segment) {
         return new JoinSegmentMatcher<ExternalTupleSet>(segment, new 
PCJToSegmentConverter());
     }
 
     @Override
-    protected ExternalSetMatcher<ExternalTupleSet> 
getOptionalJoinSegmentMatcher(OptionalJoinSegment<ExternalTupleSet> segment) {
+    protected ExternalSetMatcher<ExternalTupleSet> 
getOptionalJoinSegmentMatcher(final OptionalJoinSegment<ExternalTupleSet> 
segment) {
         return new OptionalJoinSegmentMatcher<ExternalTupleSet>(segment, new 
PCJToSegmentConverter());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJToSegmentConverter.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJToSegmentConverter.java
 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJToSegmentConverter.java
index 61d6204..5744f41 100644
--- 
a/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJToSegmentConverter.java
+++ 
b/extras/indexing/src/main/java/org/apache/rya/indexing/pcj/matching/PCJToSegmentConverter.java
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
 package org.apache.rya.indexing.pcj.matching;
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -33,17 +51,17 @@ import com.google.common.base.Preconditions;
 
 /**
  * Implementation of {@link ExternalSetConverter} to convert {@link 
ExternalTupleSet}s
- * to {@link QuerySegment}s. 
+ * to {@link QuerySegment}s.
  *
  */
 public class PCJToSegmentConverter implements 
ExternalSetConverter<ExternalTupleSet> {
 
     private static final PCJToOptionalJoinSegment optional = new 
PCJToOptionalJoinSegment();
     private static final PCJToJoinSegment join = new PCJToJoinSegment();
-    
-    
+
+
     @Override
-    public QuerySegment<ExternalTupleSet> setToSegment(ExternalTupleSet set) {
+    public QuerySegment<ExternalTupleSet> setToSegment(final ExternalTupleSet 
set) {
         Preconditions.checkNotNull(set);
         if (PCJOptimizerUtilities.tupleContainsLeftJoins(set.getTupleExpr())) {
             return optional.getSegment(set);
@@ -60,10 +78,10 @@ public class PCJToSegmentConverter implements 
ExternalSetConverter<ExternalTuple
     static class PCJToJoinSegment extends 
QueryModelVisitorBase<RuntimeException> {
 
         private JoinSegment<ExternalTupleSet> segment;
-        
+
         private PCJToJoinSegment(){};
 
-        public QuerySegment<ExternalTupleSet> getSegment(ExternalTupleSet pcj) 
{
+        public QuerySegment<ExternalTupleSet> getSegment(final 
ExternalTupleSet pcj) {
             segment = null;
             pcj.getTupleExpr().visit(this);
             return segment;
@@ -88,10 +106,10 @@ public class PCJToSegmentConverter implements 
ExternalSetConverter<ExternalTuple
     static class PCJToOptionalJoinSegment extends 
QueryModelVisitorBase<RuntimeException> {
 
         private OptionalJoinSegment<ExternalTupleSet> segment;
-        
+
         private PCJToOptionalJoinSegment(){};
 
-        public QuerySegment<ExternalTupleSet> getSegment(ExternalTupleSet pcj) 
{
+        public QuerySegment<ExternalTupleSet> getSegment(final 
ExternalTupleSet pcj) {
             segment = null;
             pcj.getTupleExpr().visit(this);
             return segment;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 8b1ee15..68f1394 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
@@ -29,13 +29,6 @@ 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.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;
@@ -53,6 +46,12 @@ 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.openrdf.sail.Sail;
+import org.openrdf.sail.SailException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mongodb.MongoClient;
 
 public class RyaSailFactory {
     private static final Logger LOG = 
LoggerFactory.getLogger(RyaSailFactory.class);
@@ -90,7 +89,8 @@ public class RyaSailFactory {
                 
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);
+           }
+            dao = getMongoDAO((MongoDBRdfConfiguration)rdfConfig, client);
         } else {
             rdfConfig = new AccumuloRdfConfiguration(config);
             user = rdfConfig.get(ConfigUtils.CLOUDBASE_USER);
@@ -139,7 +139,7 @@ public class RyaSailFactory {
         dao.setConnector(connector);
 
         ConfigUtils.setIndexers(config);
-        
+
         dao.setConf(config);
         dao.init();
         return dao;

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
index 9446d85..d2d9a54 100644
--- 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
+++ 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/query/EntityQueryNodeTest.java
@@ -18,23 +18,37 @@
  */
 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.query.EntityQueryNode;
 import org.apache.rya.indexing.entity.storage.EntityStorage;
+import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
+import org.apache.rya.mongodb.MockMongoFactory;
 import org.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 com.mongodb.MongoClient;
 
-import mvm.rya.api.domain.RyaURI;
+import info.aduna.iteration.CloseableIteration;
 
 /**
  * Unit tests the methods of {@link EntityQueryNode}.
@@ -136,16 +150,21 @@ public class EntityQueryNodeTest {
         new EntityQueryNode(EMPLOYEE_TYPE, patterns, 
mock(EntityStorage.class));
     }
 
-    // Happy path test.
-
-    // TODO test for all of the types of preconditions
-    //      test when a binding set can join
-    //      test when a binding set does not join
-    //      test when there are constants that are part of the query
-    //      test when there are variables that are part of the query.
-
     @Test
     public void evaluate_constantSubject() throws Exception {
+        final MongoClient client = 
MockMongoFactory.newFactory().newMongoClient();
+        final EntityStorage storage = new MongoEntityStorage(client, "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 { " +
@@ -155,30 +174,107 @@ public class EntityQueryNodeTest {
                     "<urn:SSN:111-11-1111> <urn:name> ?name . " +
                 "}");
 
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-
-
-        // TODO implement
+        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 {
-        // A set of patterns that matches a variable Entity subject.
+        final MongoClient client = 
MockMongoFactory.newFactory().newMongoClient();
+        final EntityStorage storage = new MongoEntityStorage(client, "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 { " +
-                    "?subject <" + RDF.TYPE + "> <urn:person> ."+
-                    "?subject <urn:age> ?age . " +
-                    "?subject <urn:eye> ?eye . " +
-                    "?subject <urn:name> ?name . " +
+                    "?ssn <" + RDF.TYPE + "> <urn:person> ."+
+                    "?ssn <urn:age> ?age . " +
+                    "?ssn <urn:eye> ?eye . " +
+                    "?ssn <urn:name> ?name . " +
                 "}");
 
-        new EntityQueryNode(PERSON_TYPE, patterns, mock(EntityStorage.class));
-
-
-        // TODO implement
+        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 MongoClient client = 
MockMongoFactory.newFactory().newMongoClient();
+        final EntityStorage storage = new MongoEntityStorage(client, "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
@@ -187,7 +283,7 @@ public class EntityQueryNodeTest {
      * @return
      * @throws MalformedQueryException
      */
-    private static List<StatementPattern> getSPs(String sparql) 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();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/EntityDocumentConverterTest.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/EntityDocumentConverterTest.java
 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/EntityDocumentConverterTest.java
index 9b8cb66..79ea998 100644
--- 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/EntityDocumentConverterTest.java
+++ 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/EntityDocumentConverterTest.java
@@ -20,17 +20,15 @@ package org.apache.rya.indexing.entity.storage.mongo;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Entity;
 import org.apache.rya.indexing.entity.model.Property;
-import org.apache.rya.indexing.entity.storage.mongo.EntityDocumentConverter;
 import 
org.apache.rya.indexing.entity.storage.mongo.DocumentConverter.DocumentConverterException;
 import org.bson.Document;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.XMLSchema;
 
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-
 /**
  * Tests the methods of {@link EntityDocumentConverter}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 1269d20..d271ba0 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
@@ -26,25 +26,22 @@ import java.util.HashSet;
 import java.util.Optional;
 import java.util.Set;
 
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Entity;
 import org.apache.rya.indexing.entity.model.Property;
 import org.apache.rya.indexing.entity.model.Type;
 import org.apache.rya.indexing.entity.model.TypedEntity;
-import org.apache.rya.indexing.entity.storage.CloseableIterator;
 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.indexing.entity.storage.mongo.MongoEntityStorage;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.XMLSchema;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-
 /**
  * Integration tests the methods of {@link MongoEntityStorage}.
  */
@@ -198,7 +195,7 @@ public class MongoEntityStorageIT extends MongoITBase {
 
         // Search for all icecreams.
         final Set<TypedEntity> objects = new HashSet<>();
-        try(final CloseableIterator<TypedEntity> it = 
storage.search(icecreamType, new HashSet<>())) {
+        try(final ConvertingCursor<TypedEntity> it = 
storage.search(Optional.empty(), icecreamType, new HashSet<>())) {
             while(it.hasNext()) {
                 objects.add(it.next());
             }
@@ -295,7 +292,7 @@ public class MongoEntityStorageIT extends MongoITBase {
                 new Property(new RyaURI("urn:eye"), new 
RyaType(XMLSchema.STRING, "blue")),
                 new Property(new RyaURI("urn:age"), new RyaType(XMLSchema.INT, 
"30")));
 
-        try(final CloseableIterator<TypedEntity> it = 
storage.search(personType, searchValues)) {
+        try(final ConvertingCursor<TypedEntity> it = 
storage.search(Optional.empty(), personType, searchValues)) {
             while(it.hasNext()) {
                 objects.add(it.next());
             }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
index 9894ee5..692d1aa 100644
--- 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
+++ 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/MongoITBase.java
@@ -18,20 +18,18 @@
  */
 package org.apache.rya.indexing.entity.storage.mongo;
 
-import java.net.UnknownHostException;
 import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.mongodb.MockMongoFactory;
+import org.apache.rya.mongodb.MongoConnectorFactory;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 
 import com.mongodb.MongoClient;
-import com.mongodb.MongoException;
-
-import mvm.rya.mongodb.MongoConnectorFactory;
-import mvm.rya.mongodb.MongoDBRdfConfiguration;
 
 /**
  * A base class that may be used when implementing Mongo DB integration tests 
that
@@ -41,14 +39,15 @@ public class MongoITBase {
 
     private MongoClient mongoClient = null;
     private Set<String> originalDbNames = null;
+    protected MongoDBRdfConfiguration conf;
 
     @Before
-    public void setupTest() throws UnknownHostException, MongoException {
-        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new 
Configuration() );
-        conf.setUseTestMongo(true);
+    public void setupTest() throws Exception {
+        conf = new MongoDBRdfConfiguration( new Configuration() );
         conf.setMongoDBName("testDB");
+        mongoClient = MockMongoFactory.newFactory().newMongoClient();
+        conf.setMongoClient(mongoClient);
 
-        mongoClient = MongoConnectorFactory.getMongoClient(conf);
 
         // Store the names of the DBs that are present before running the test.
         originalDbNames = new HashSet<>();
@@ -69,7 +68,7 @@ public class MongoITBase {
 
     @AfterClass
     public static void shutdown() {
-        MongoConnectorFactory.shutdown();
+        MongoConnectorFactory.closeMongoClient();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 2b4c849..56c10c0 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
@@ -23,21 +23,18 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.HashSet;
+import java.util.Optional;
 import java.util.Set;
 
+import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Type;
-import org.apache.rya.indexing.entity.storage.CloseableIterator;
 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.MongoTypeStorage;
 import org.junit.Test;
 
-import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
-import mvm.rya.api.domain.RyaURI;
-
 /**
  * Integration tests the methods of {@link MongoTypeStorage}.
  */
@@ -161,7 +158,7 @@ public class MongoTypeStorageIT extends MongoITBase {
         storage.create(icecream);
 
         // Search for all Types that have the 'urn:eye' property.
-        final CloseableIterator<Type> typeIt = storage.search(new 
RyaURI("urn:eye"));
+        final ConvertingCursor<Type> typeIt = storage.search(new 
RyaURI("urn:eye"));
 
         final Set<Type> types = new HashSet<>();
         while(typeIt.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/RyaTypeDocumentConverterTest.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/RyaTypeDocumentConverterTest.java
 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/RyaTypeDocumentConverterTest.java
index 35c7968..3196793 100644
--- 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/RyaTypeDocumentConverterTest.java
+++ 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/RyaTypeDocumentConverterTest.java
@@ -20,16 +20,14 @@ package org.apache.rya.indexing.entity.storage.mongo;
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.rya.indexing.entity.storage.mongo.RyaTypeDocumentConverter;
+import org.apache.rya.api.domain.RyaType;
+import org.apache.rya.api.resolver.RdfToRyaConversions;
 import 
org.apache.rya.indexing.entity.storage.mongo.DocumentConverter.DocumentConverterException;
 import org.bson.Document;
 import org.junit.Test;
 import org.openrdf.model.impl.ValueFactoryImpl;
 import org.openrdf.model.vocabulary.XMLSchema;
 
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.resolver.RdfToRyaConversions;
-
 /**
  * Tests the methods of {@link RyaTypeDocumentConverter}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/TypeDocumentConverterTest.java
----------------------------------------------------------------------
diff --git 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/TypeDocumentConverterTest.java
 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/TypeDocumentConverterTest.java
index 9d854cf..4c795c2 100644
--- 
a/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/TypeDocumentConverterTest.java
+++ 
b/extras/indexing/src/test/java/org/apache/rya/indexing/entity/storage/mongo/TypeDocumentConverterTest.java
@@ -20,8 +20,8 @@ package org.apache.rya.indexing.entity.storage.mongo;
 
 import static org.junit.Assert.assertEquals;
 
+import org.apache.rya.api.domain.RyaURI;
 import org.apache.rya.indexing.entity.model.Type;
-import org.apache.rya.indexing.entity.storage.mongo.TypeDocumentConverter;
 import 
org.apache.rya.indexing.entity.storage.mongo.DocumentConverter.DocumentConverterException;
 import org.bson.Document;
 import org.junit.Test;
@@ -29,8 +29,6 @@ import org.junit.Test;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-import mvm.rya.api.domain.RyaURI;
-
 /**
  * Tests the methods of {@link TypeDocumentConverter}.
  */

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c4d4bfe7/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 1fc3f72..a33ddd4 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
@@ -22,7 +22,9 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
+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.indexing.entity.model.Entity;
 import org.apache.rya.indexing.entity.model.Property;
 import org.apache.rya.indexing.entity.model.Type;
@@ -31,8 +33,7 @@ import org.apache.rya.indexing.entity.storage.TypeStorage;
 import org.apache.rya.indexing.entity.storage.mongo.MongoEntityStorage;
 import org.apache.rya.indexing.entity.storage.mongo.MongoITBase;
 import org.apache.rya.indexing.entity.storage.mongo.MongoTypeStorage;
-import org.apache.rya.indexing.entity.update.EntityIndexer;
-import org.apache.rya.indexing.entity.update.mongo.MongoEntityIndexer;
+import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.vocabulary.RDF;
 import org.openrdf.model.vocabulary.XMLSchema;
@@ -40,17 +41,12 @@ import org.openrdf.model.vocabulary.XMLSchema;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
-import mvm.rya.api.domain.RyaStatement;
-import mvm.rya.api.domain.RyaType;
-import mvm.rya.api.domain.RyaURI;
-import mvm.rya.mongodb.MongoDBRdfConfiguration;
-
 /**
  * Integration tests the methods of {@link MongoEntityIndexer}.
  */
 public class MongoEntityIndexerIT extends MongoITBase {
 
-    private static final String RYA_INSTANCE_NAME = "testInstance";
+    private static final String RYA_INSTANCE_NAME = "testDB";
 
     private static final Type PERSON_TYPE =
             new Type(new RyaURI("urn:person"),
@@ -67,6 +63,16 @@ public class MongoEntityIndexerIT extends MongoITBase {
                     .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.
@@ -74,7 +80,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(PERSON_TYPE);
 
         // Index a RyaStatement that will create an Entity with an explicit 
type.
-        final EntityIndexer indexer = makeTestIndexer();
         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);
 
@@ -98,7 +103,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(EMPLOYEE_TYPE);
 
         // Index a RyaStatement that will create an Entity with two implicit 
types.
-        final EntityIndexer indexer = makeTestIndexer();
         final RyaStatement statement = new RyaStatement(new 
RyaURI("urn:SSN/111-11-1111"), new RyaURI("urn:name"), new 
RyaType(XMLSchema.STRING, "Alice"));
         indexer.storeStatement(statement);
 
@@ -123,8 +127,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(EMPLOYEE_TYPE);
 
         // Index a bunch of RyaStatements.
-        final EntityIndexer indexer = makeTestIndexer();
-
         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")));
@@ -156,8 +158,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(EMPLOYEE_TYPE);
 
         // Index a bunch of RyaStatements.
-        final EntityIndexer indexer = makeTestIndexer();
-
         final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
         final RyaURI bobSSN = new RyaURI("urn:SSN/222-22-2222");
 
@@ -208,8 +208,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(EMPLOYEE_TYPE);
 
         // Index a bunch of RyaStatements.
-        final EntityIndexer indexer = makeTestIndexer();
-
         final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
 
         indexer.storeStatements(Sets.newHashSet(
@@ -245,8 +243,6 @@ public class MongoEntityIndexerIT extends MongoITBase {
         types.create(EMPLOYEE_TYPE);
 
         // Index a bunch of RyaStatements.
-        final EntityIndexer indexer = makeTestIndexer();
-
         final RyaURI aliceSSN = new RyaURI("urn:SSN/111-11-1111");
 
         indexer.storeStatements(Sets.newHashSet(
@@ -272,15 +268,4 @@ public class MongoEntityIndexerIT extends MongoITBase {
 
         assertEquals(expected, entity);
     }
-
-    private static EntityIndexer makeTestIndexer() {
-        final EntityIndexer indexer = new MongoEntityIndexer();
-
-        final MongoDBRdfConfiguration conf = new MongoDBRdfConfiguration( new 
Configuration() );
-        conf.setUseTestMongo(true);
-        conf.setMongoDBName(RYA_INSTANCE_NAME);
-
-        indexer.setConf(conf);
-        return indexer;
-    }
 }
\ No newline at end of file

Reply via email to