You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by ca...@apache.org on 2018/01/12 21:25:34 UTC

[6/8] incubator-rya git commit: RYA-303 Mongo PCJ Support. Closes #172.

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJIT.java
new file mode 100644
index 0000000..7933374
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoBatchUpdatePCJIT.java
@@ -0,0 +1,148 @@
+/*
+ * 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.api.client.mongo;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.rya.api.client.Install.InstallConfiguration;
+import org.apache.rya.api.client.RyaClient;
+import org.apache.rya.api.client.accumulo.AccumuloBatchUpdatePCJ;
+import org.apache.rya.api.utils.CloseableIterator;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
+import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinUpdaterType;
+import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
+import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
+import org.apache.rya.mongodb.MongoDBRdfConfiguration;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+import org.openrdf.model.Statement;
+import org.openrdf.model.ValueFactory;
+import org.openrdf.model.impl.ValueFactoryImpl;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.impl.MapBindingSet;
+
+/**
+ * Integration tests the methods of {@link AccumuloBatchUpdatePCJ}.
+ */
+public class MongoBatchUpdatePCJIT extends MongoITBase {
+
+    @Override
+    protected void updateConfiguration(final MongoDBRdfConfiguration conf) {
+        conf.setBoolean(ConfigUtils.USE_PCJ, true);
+        conf.set(ConfigUtils.PCJ_STORAGE_TYPE, PrecomputedJoinStorageType.MONGO.toString());
+        conf.set(ConfigUtils.PCJ_UPDATER_TYPE, PrecomputedJoinUpdaterType.NO_UPDATE.toString());
+    }
+
+
+    @Test
+    public void batchUpdate() throws Exception {
+        // Setup a Rya Client.
+        final MongoConnectionDetails connectionDetails = getConnectionDetails();
+        final RyaClient ryaClient = MongoRyaClientFactory.build(connectionDetails, getMongoClient());
+
+        // Install an instance of Rya on the mini accumulo cluster.
+        ryaClient.getInstall().install(conf.getRyaInstanceName(), InstallConfiguration.builder()
+                .setEnablePcjIndex(true)
+                .build());
+
+        // Load some statements into the Rya instance.
+        final ValueFactory vf = ValueFactoryImpl.getInstance();
+        final Collection<Statement> statements = new ArrayList<>();
+        statements.add(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:Frank"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:George"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+        statements.add(vf.createStatement(vf.createURI("urn:Hillary"), vf.createURI("urn:likes"), vf.createURI("urn:icecream")));
+
+        statements.add(vf.createStatement(vf.createURI("urn:Alice"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:Bob"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:Charlie"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:David"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:Eve"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:Frank"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:blue")));
+        statements.add(vf.createStatement(vf.createURI("urn:George"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:green")));
+        statements.add(vf.createStatement(vf.createURI("urn:Hillary"), vf.createURI("urn:hasEyeColor"), vf.createURI("urn:brown")));
+        ryaClient.getLoadStatements().loadStatements(conf.getRyaInstanceName(), statements);
+
+        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(getMongoClient(), conf.getRyaInstanceName())) {
+            // Create a PCJ for a SPARQL query.
+            final String sparql = "SELECT ?name WHERE { ?name <urn:likes> <urn:icecream> . ?name <urn:hasEyeColor> <urn:blue> . }";
+            final String pcjId = pcjStorage.createPcj(sparql);
+
+            // Run the test.
+            ryaClient.getBatchUpdatePCJ().batchUpdate(conf.getRyaInstanceName(), pcjId);
+
+            // Verify the correct results were loaded into the PCJ table.
+            final Set<BindingSet> expectedResults = new HashSet<>();
+
+            MapBindingSet bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:Alice"));
+            expectedResults.add(bs);
+
+            bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:Bob"));
+            expectedResults.add(bs);
+
+            bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:Charlie"));
+            expectedResults.add(bs);
+
+            bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:David"));
+            expectedResults.add(bs);
+
+            bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:Eve"));
+            expectedResults.add(bs);
+
+            bs = new MapBindingSet();
+            bs.addBinding("name", vf.createURI("urn:Frank"));
+            expectedResults.add(bs);
+
+            final Set<BindingSet> results = new HashSet<>();
+            try(CloseableIterator<BindingSet> resultsIt = pcjStorage.listResults(pcjId)) {
+                while(resultsIt.hasNext()) {
+                    results.add( resultsIt.next() );
+                }
+            }
+            assertEquals(expectedResults, results);
+        }
+    }
+
+    private MongoConnectionDetails getConnectionDetails() {
+        final java.util.Optional<char[]> password = conf.getMongoPassword() != null ?
+                java.util.Optional.of(conf.getMongoPassword().toCharArray()) :
+                    java.util.Optional.empty();
+
+                return new MongoConnectionDetails(
+                        conf.getMongoHostname(),
+                        Integer.parseInt(conf.getMongoPort()),
+                        java.util.Optional.ofNullable(conf.getMongoUser()),
+                        password);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoCreatePCJIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoCreatePCJIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoCreatePCJIT.java
new file mode 100644
index 0000000..183c195
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoCreatePCJIT.java
@@ -0,0 +1,122 @@
+/**
+ * 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.api.client.mongo;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import org.apache.rya.api.client.CreatePCJ;
+import org.apache.rya.api.client.Install;
+import org.apache.rya.api.client.Install.DuplicateInstanceNameException;
+import org.apache.rya.api.client.Install.InstallConfiguration;
+import org.apache.rya.api.client.InstanceDoesNotExistException;
+import org.apache.rya.api.client.RyaClient;
+import org.apache.rya.api.client.RyaClientException;
+import org.apache.rya.api.client.accumulo.AccumuloCreatePCJ;
+import org.apache.rya.api.instance.RyaDetails;
+import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
+import org.apache.rya.indexing.pcj.storage.PcjMetadata;
+import org.apache.rya.indexing.pcj.storage.PrecomputedJoinStorage;
+import org.apache.rya.indexing.pcj.storage.mongo.MongoPcjStorage;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Integration tests the methods of {@link AccumuloCreatePCJ}.
+ */
+public class MongoCreatePCJIT extends MongoITBase {
+    @Test(expected = InstanceDoesNotExistException.class)
+    public void instanceDoesNotExist() throws Exception {
+        final RyaClient ryaClient = MongoRyaClientFactory.build(getConnectionDetails(), getMongoClient());
+        // Skip the install step to create error causing situation.
+        ryaClient.getCreatePCJ().createPCJ(conf.getRyaInstanceName(), "");
+    }
+
+    @Test
+    public void createPCJ() throws Exception {
+        final MongoConnectionDetails connectionDetails = getConnectionDetails();
+        final RyaClient ryaClient = MongoRyaClientFactory.build(connectionDetails, getMongoClient());
+        // Initialize the commands that will be used by this test.
+        final CreatePCJ createPCJ = ryaClient.getCreatePCJ();
+        final Install installRya = ryaClient.getInstall();
+        final InstallConfiguration installConf = InstallConfiguration.builder()
+                .setEnablePcjIndex(true)
+                .build();
+        installRya.install(conf.getRyaInstanceName(), installConf);
+
+        System.out.println(getMongoClient().getDatabase(conf.getRyaInstanceName()).getCollection("instance_details").find().first().toJson());
+        // Create a PCJ.
+        final String sparql =
+                "SELECT ?x " +
+                        "WHERE { " +
+                        "?x <http://talksTo> <http://Eve>. " +
+                        "?x <http://worksAt> <http://TacoJoint>." +
+                        "}";
+        final String pcjId = createPCJ.createPCJ(conf.getRyaInstanceName(), sparql);
+
+        // Verify the RyaDetails were updated to include the new PCJ.
+        final Optional<RyaDetails> ryaDetails = ryaClient.getGetInstanceDetails().getDetails(conf.getRyaInstanceName());
+        final ImmutableMap<String, PCJDetails> details = ryaDetails.get().getPCJIndexDetails().getPCJDetails();
+        final PCJDetails pcjDetails = details.get(pcjId);
+
+        assertEquals(pcjId, pcjDetails.getId());
+        assertFalse( pcjDetails.getLastUpdateTime().isPresent() );
+
+        // Verify the PCJ's metadata was initialized.
+
+        try(final PrecomputedJoinStorage pcjStorage = new MongoPcjStorage(getMongoClient(), conf.getRyaInstanceName())) {
+            final PcjMetadata pcjMetadata = pcjStorage.getPcjMetadata(pcjId);
+            //confirm that the pcj was added to the pcj store.
+            assertEquals(sparql, pcjMetadata.getSparql());
+            assertEquals(0L, pcjMetadata.getCardinality());
+        }
+    }
+
+    private MongoConnectionDetails getConnectionDetails() {
+        final java.util.Optional<char[]> password = conf.getMongoPassword() != null ?
+                java.util.Optional.of(conf.getMongoPassword().toCharArray()) :
+                    java.util.Optional.empty();
+
+                return new MongoConnectionDetails(
+                        conf.getMongoHostname(),
+                        Integer.parseInt(conf.getMongoPort()),
+                        java.util.Optional.ofNullable(conf.getMongoUser()),
+                        password);
+    }
+
+    @Test(expected = RyaClientException.class)
+    public void createPCJ_invalidSparql() throws DuplicateInstanceNameException, RyaClientException {
+        final MongoConnectionDetails connectionDetails = getConnectionDetails();
+        final RyaClient ryaClient = MongoRyaClientFactory.build(connectionDetails, getMongoClient());
+        // Initialize the commands that will be used by this test.
+        final CreatePCJ createPCJ = ryaClient.getCreatePCJ();
+        final Install installRya = ryaClient.getInstall();
+        final InstallConfiguration installConf = InstallConfiguration.builder()
+                .setEnablePcjIndex(true)
+                .build();
+        installRya.install(conf.getRyaInstanceName(), installConf);
+
+        // Create a PCJ.
+        final String sparql = "not valid sparql";
+        createPCJ.createPCJ(conf.getRyaInstanceName(), sparql);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoDeletePCJIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoDeletePCJIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoDeletePCJIT.java
new file mode 100644
index 0000000..d8137bc
--- /dev/null
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoDeletePCJIT.java
@@ -0,0 +1,93 @@
+/**
+ * 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.api.client.mongo;
+
+import static org.junit.Assert.assertNull;
+
+import org.apache.rya.api.client.CreatePCJ;
+import org.apache.rya.api.client.DeletePCJ;
+import org.apache.rya.api.client.Install;
+import org.apache.rya.api.client.Install.InstallConfiguration;
+import org.apache.rya.api.client.InstanceDoesNotExistException;
+import org.apache.rya.api.client.RyaClient;
+import org.apache.rya.api.client.accumulo.AccumuloCreatePCJ;
+import org.apache.rya.api.instance.RyaDetails;
+import org.apache.rya.api.instance.RyaDetails.PCJIndexDetails.PCJDetails;
+import org.apache.rya.mongodb.MongoITBase;
+import org.junit.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Integration tests the methods of {@link AccumuloCreatePCJ}.
+ */
+public class MongoDeletePCJIT extends MongoITBase {
+    @Test(expected = InstanceDoesNotExistException.class)
+    public void instanceDoesNotExist() throws Exception {
+        final RyaClient ryaClient = MongoRyaClientFactory.build(getConnectionDetails(), getMongoClient());
+        // Skip the install step to create error causing situation.
+        ryaClient.getDeletePCJ().deletePCJ(conf.getRyaInstanceName(), "doesn't matter, should fail before the pcjID is needed");
+    }
+
+    @Test
+    public void deletePCJ() throws Exception {
+        final MongoConnectionDetails connectionDetails = getConnectionDetails();
+        final RyaClient ryaClient = MongoRyaClientFactory.build(connectionDetails, getMongoClient());
+        // Initialize the commands that will be used by this test.
+        final CreatePCJ createPCJ = ryaClient.getCreatePCJ();
+        final Install installRya = ryaClient.getInstall();
+        final InstallConfiguration installConf = InstallConfiguration.builder()
+                .setEnablePcjIndex(true)
+                .build();
+        installRya.install(conf.getRyaInstanceName(), installConf);
+
+        System.out.println(getMongoClient().getDatabase(conf.getRyaInstanceName()).getCollection("instance_details").find().first().toJson());
+        // Create a PCJ.
+        final String sparql =
+                "SELECT ?x " +
+                        "WHERE { " +
+                        "?x <http://talksTo> <http://Eve>. " +
+                        "?x <http://worksAt> <http://TacoJoint>." +
+                        "}";
+        final String pcjId = createPCJ.createPCJ(conf.getRyaInstanceName(), sparql);
+
+        final DeletePCJ deletePCJ = ryaClient.getDeletePCJ();
+        deletePCJ.deletePCJ(conf.getRyaInstanceName(), pcjId);
+
+        // Verify the RyaDetails were updated to include the new PCJ.
+        final Optional<RyaDetails> ryaDetails = ryaClient.getGetInstanceDetails().getDetails(conf.getRyaInstanceName());
+        final ImmutableMap<String, PCJDetails> details = ryaDetails.get().getPCJIndexDetails().getPCJDetails();
+        final PCJDetails pcjDetails = details.get(pcjId);
+
+        assertNull(pcjDetails);
+    }
+
+    private MongoConnectionDetails getConnectionDetails() {
+        final java.util.Optional<char[]> password = conf.getMongoPassword() != null ?
+                java.util.Optional.of(conf.getMongoPassword().toCharArray()) :
+                    java.util.Optional.empty();
+
+                return new MongoConnectionDetails(
+                        conf.getMongoHostname(),
+                        Integer.parseInt(conf.getMongoPort()),
+                        java.util.Optional.ofNullable(conf.getMongoUser()),
+                        password);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoGetInstanceDetailsIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoGetInstanceDetailsIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoGetInstanceDetailsIT.java
index b3d96f6..5f1d222 100644
--- a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoGetInstanceDetailsIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoGetInstanceDetailsIT.java
@@ -84,7 +84,7 @@ public class MongoGetInstanceDetailsIT extends MongoITBase {
                 // PCJJ Index is not supported, so it flips to false.
                 .setPCJIndexDetails(
                         PCJIndexDetails.builder()
-                        .setEnabled(false))
+                        .setEnabled(true))
 
                 .setProspectorDetails( new ProspectorDetails(Optional.<Date>absent()) )
                 .setJoinSelectivityDetails( new JoinSelectivityDetails(Optional.<Date>absent()) )
@@ -119,10 +119,10 @@ public class MongoGetInstanceDetailsIT extends MongoITBase {
                 java.util.Optional.of(conf.getMongoPassword().toCharArray()) :
                     java.util.Optional.empty();
 
-        return new MongoConnectionDetails(
-                conf.getMongoHostname(),
-                Integer.parseInt(conf.getMongoPort()),
-                java.util.Optional.ofNullable(conf.getMongoUser()),
-                password);
+                return new MongoConnectionDetails(
+                        conf.getMongoHostname(),
+                        Integer.parseInt(conf.getMongoPort()),
+                        java.util.Optional.ofNullable(conf.getMongoUser()),
+                        password);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java
index e3df916..ecca646 100644
--- a/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/api/client/mongo/MongoInstallIT.java
@@ -79,6 +79,47 @@ public class MongoInstallIT extends MongoITBase {
         assertTrue("Instance should exist.", instanceExists.exists(ryaInstance));
     }
 
+    @Test
+    public void install_withAllIndexers() throws DuplicateInstanceNameException, RyaClientException {
+        // Install an instance of Rya.
+        final String ryaInstance = conf.getMongoDBName();
+
+        // Setup the connection details that were used for the embedded Mongo DB instance we are testing with.
+        final MongoConnectionDetails connectionDetails = getConnectionDetails();
+
+        // Check that the instance does not exist.
+        final InstanceExists instanceExists = new MongoInstanceExists(getMongoClient());
+        assertFalse(instanceExists.exists(ryaInstance));
+
+        // Install an instance of Rya with all the valid options turned on.
+        final InstallConfiguration installConfig = InstallConfiguration.builder()
+                .setEnableTableHashPrefix(true)
+                .setEnableFreeTextIndex(true)
+                .setEnableTemporalIndex(true)
+                .setEnableEntityCentricIndex(true)
+                .setEnableGeoIndex(true)
+                .setEnablePcjIndex(true)
+                .build();
+
+        final RyaClient ryaClient = MongoRyaClientFactory.build(connectionDetails, getMongoClient());
+        final Install install = ryaClient.getInstall();
+        install.install(ryaInstance, installConfig);
+
+        // Check that the instance exists.
+        assertTrue(instanceExists.exists(ryaInstance));
+
+        // Show that the expected collections were created within the database.
+        final List<String> expected = Arrays.asList(INSTANCE_DETAILS_COLLECTION_NAME, "rya_triples");
+        int count = 0;
+        final List<String> found = new ArrayList<>();
+        for (final String collection : getMongoClient().getDatabase(conf.getMongoDBName()).listCollectionNames()) {
+            count += expected.contains(collection) ? 1 : 0;
+            found.add( collection );
+        }
+        assertTrue("Tables missing from:" + expected + " actual:" + found, expected.size() == count);
+        assertTrue("Instance should exist.", instanceExists.exists(ryaInstance));
+    }
+
     @Test(expected = DuplicateInstanceNameException.class)
     public void install_alreadyExists() throws DuplicateInstanceNameException, RyaClientException {
         // Install an instance of Rya.
@@ -100,10 +141,10 @@ public class MongoInstallIT extends MongoITBase {
                 Optional.of(conf.getMongoPassword().toCharArray()) :
                     Optional.empty();
 
-        return new MongoConnectionDetails(
-                conf.getMongoHostname(),
-                Integer.parseInt(conf.getMongoPort()),
-                Optional.ofNullable(conf.getMongoUser()),
-                password);
+                return new MongoConnectionDetails(
+                        conf.getMongoHostname(),
+                        Integer.parseInt(conf.getMongoPort()),
+                        Optional.ofNullable(conf.getMongoUser()),
+                        password);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/IndexPlanValidatorTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/IndexPlanValidatorTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/IndexPlanValidatorTest.java
index ffd6691..d54523c 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/IndexPlanValidatorTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/IndexPlanValidatorTest.java
@@ -1,6 +1,4 @@
-package org.apache.rya.indexing.IndexPlanValidator;
-
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,18 +16,26 @@ package org.apache.rya.indexing.IndexPlanValidator;
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.rya.indexing.IndexPlanValidator;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.apache.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
+import org.apache.rya.indexing.mongodb.pcj.MongoPcjIndexSetProvider;
 import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
-
+import org.apache.rya.indexing.pcj.matching.provider.AbstractPcjIndexSetProvider;
+import org.apache.rya.indexing.pcj.matching.provider.AccumuloIndexSetProvider;
+import org.apache.rya.mongodb.EmbeddedMongoSingleton;
+import org.apache.rya.mongodb.StatefulMongoDBRdfConfiguration;
 import org.junit.Assert;
 import org.junit.Test;
-import org.openrdf.query.MalformedQueryException;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.TupleExpr;
 import org.openrdf.query.parser.ParsedQuery;
@@ -37,716 +43,743 @@ import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import com.google.common.collect.Lists;
 
+@RunWith(Parameterized.class)
 public class IndexPlanValidatorTest {
+    private final AbstractPcjIndexSetProvider provider;
+
+    @Parameterized.Parameters
+    public static Collection providers() throws Exception {
+        final StatefulMongoDBRdfConfiguration conf = new StatefulMongoDBRdfConfiguration(new Configuration(), EmbeddedMongoSingleton.getNewMongoClient());
+        return Lists.<AbstractPcjIndexSetProvider> newArrayList(
+                new AccumuloIndexSetProvider(new Configuration()),
+                new MongoPcjIndexSetProvider(conf)
+                );
+    }
+
+    public IndexPlanValidatorTest(final AbstractPcjIndexSetProvider provider) {
+        this.provider = provider;
+    }
+
+    @Test
+    public void testEvaluateTwoIndexTwoVarOrder1()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?c ?e ?l  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?o ?l " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexTwoVarOrder2()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?l ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?o ?l " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexTwoVarOrder3()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?l ?e ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?o ?l " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
 
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder1()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?c ?e ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder2()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder3()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?l ?e ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder4()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?c ?l  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?o ?l " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
 
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexTwoVarOrder4()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?c ?l  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?o ?l " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
 
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-
-	@Test
-	public void testEvaluateTwoIndexTwoVarOrder6()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
 
-		String indexSparqlString2 = ""//
-				+ "SELECT ?l ?e ?o " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+
+    @Test
+    public void testEvaluateTwoIndexTwoVarOrder6()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?l ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?l ?e ?o " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
 
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais2);
-		index.add(ais1);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexCrossProduct1()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais2);
-		index.add(ais1);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(true);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexCrossProduct2()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(true);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexCrossProduct3()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?e ?l ?c  " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?e ?l ?o " //
-				+ "{" //
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexDiffVars() throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?chicken ?dog ?pig  " //
-				+ "{" //
-				+ "  ?dog a ?chicken . "//
-				+ "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?fish ?ant ?turkey " //
-				+ "{" //
-				+ "  ?fish <uri:talksTo> ?turkey . "//
-				+ "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexDiffVars2() throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?dog ?pig ?chicken  " //
-				+ "{" //
-				+ "  ?dog a ?chicken . "//
-				+ "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?fish ?ant ?turkey " //
-				+ "{" //
-				+ "  ?fish <uri:talksTo> ?turkey . "//
-				+ "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexDiffVars3() throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?pig ?dog ?chicken  " //
-				+ "{" //
-				+ "  ?dog a ?chicken . "//
-				+ "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?fish ?ant ?turkey " //
-				+ "{" //
-				+ "  ?fish <uri:talksTo> ?turkey . "//
-				+ "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(false);
-		Assert.assertEquals(true, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testEvaluateTwoIndexDiffVarsDirProd()
-			throws Exception {
-
-
-		String indexSparqlString = ""//
-				+ "SELECT ?pig ?dog ?chicken  " //
-				+ "{" //
-				+ "  ?dog a ?chicken . "//
-				+ "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
-				+ "}";//
-
-		String indexSparqlString2 = ""//
-				+ "SELECT ?fish ?ant ?turkey " //
-				+ "{" //
-				+ "  ?fish <uri:talksTo> ?turkey . "//
-				+ "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
-				+ "}";//
-
-		String queryString = ""//
-				+ "SELECT ?e ?c ?l ?o ?f ?g " //
-				+ "{" //
-				+ "  ?e a ?c . "//
-				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?e <uri:talksTo> ?o . "//
-				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-				+ "  ?f <uri:talksTo> ?g . " //
-				+ "}";//
-
-		SPARQLParser sp = new SPARQLParser();
-		ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
-		ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
-
-		List<ExternalTupleSet> index = Lists.newArrayList();
-
-		SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
-				(Projection) index1.getTupleExpr());
-		SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
-				(Projection) index2.getTupleExpr());
-
-		index.add(ais1);
-		index.add(ais2);
-
-		ParsedQuery pq = sp.parseQuery(queryString, null);
-		TupleExpr tup = pq.getTupleExpr().clone();
-		PCJOptimizer pcj = new PCJOptimizer(index, false);
-		pcj.optimize(tup, null, null);
-
-		IndexPlanValidator ipv = new IndexPlanValidator(true);
-		Assert.assertEquals(false, ipv.isValid(tup));
-
-	}
-
-	@Test
-	public void testValidTupleIterator() throws Exception {
-
-		String q1 = ""//
-				+ "SELECT ?f ?m ?d ?h ?i " //
-				+ "{" //
-				+ "  ?f a ?m ."//
-				+ "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
-				+ "  ?d <uri:talksTo> ?f . "//
-				+ "  ?d <uri:hangOutWith> ?f ." //
-				+ "  ?f <uri:hangOutWith> ?h ." //
-				+ "  ?f <uri:associatesWith> ?i ." //
-				+ "  ?i <uri:associatesWith> ?h ." //
-				+ "}";//
-
-		String q2 = ""//
-				+ "SELECT ?t ?s ?u " //
-				+ "{" //
-				+ "  ?s a ?t ."//
-				+ "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
-				+ "  ?u <uri:talksTo> ?s . "//
-				+ "}";//
-
-		String q3 = ""//
-				+ "SELECT ?s ?t ?u " //
-				+ "{" //
-				+ "  ?s <uri:hangOutWith> ?t ." //
-				+ "  ?t <uri:hangOutWith> ?u ." //
-				+ "}";//
-
-		String q4 = ""//
-				+ "SELECT ?s ?t ?u " //
-				+ "{" //
-				+ "  ?s <uri:associatesWith> ?t ." //
-				+ "  ?t <uri:associatesWith> ?u ." //
-				+ "}";//
-
-		SPARQLParser parser = new SPARQLParser();
-
-		ParsedQuery pq1 = parser.parseQuery(q1, null);
-		ParsedQuery pq2 = parser.parseQuery(q2, null);
-		ParsedQuery pq3 = parser.parseQuery(q3, null);
-		ParsedQuery pq4 = parser.parseQuery(q4, null);
-
-		SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet(
-				(Projection) pq2.getTupleExpr());
-		SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet(
-				(Projection) pq3.getTupleExpr());
-		SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet(
-				(Projection) pq4.getTupleExpr());
-
-		List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
-
-		list.add(extTup2);
-		list.add(extTup1);
-		list.add(extTup3);
-
-		IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(
-				pq1.getTupleExpr(), list);
-
-		Iterator<TupleExpr> plans = new TupleExecutionPlanGenerator()
-				.getPlans(iep.getIndexedTuples());
-		IndexPlanValidator ipv = new IndexPlanValidator(true);
-		Iterator<TupleExpr> validPlans = ipv.getValidTuples(plans);
-
-		int size = 0;
-
-		while (validPlans.hasNext()) {
-			Assert.assertTrue(validPlans.hasNext());
-			validPlans.next();
-			size++;
-		}
-
-		Assert.assertTrue(!validPlans.hasNext());
-		Assert.assertEquals(732, size);
-
-	}
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais2);
+        index.add(ais1);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexCrossProduct1()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?l ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?l ?o " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais2);
+        index.add(ais1);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(true);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexCrossProduct2()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?l ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?l ?o " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(true);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexCrossProduct3()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?e ?l ?c  " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?e ?l ?o " //
+                + "{" //
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexDiffVars() throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?chicken ?dog ?pig  " //
+                + "{" //
+                + "  ?dog a ?chicken . "//
+                + "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?fish ?ant ?turkey " //
+                + "{" //
+                + "  ?fish <uri:talksTo> ?turkey . "//
+                + "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexDiffVars2() throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?dog ?pig ?chicken  " //
+                + "{" //
+                + "  ?dog a ?chicken . "//
+                + "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?fish ?ant ?turkey " //
+                + "{" //
+                + "  ?fish <uri:talksTo> ?turkey . "//
+                + "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexDiffVars3() throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?pig ?dog ?chicken  " //
+                + "{" //
+                + "  ?dog a ?chicken . "//
+                + "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?fish ?ant ?turkey " //
+                + "{" //
+                + "  ?fish <uri:talksTo> ?turkey . "//
+                + "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(false);
+        Assert.assertEquals(true, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testEvaluateTwoIndexDiffVarsDirProd()
+            throws Exception {
+
+
+        final String indexSparqlString = ""//
+                + "SELECT ?pig ?dog ?chicken  " //
+                + "{" //
+                + "  ?dog a ?chicken . "//
+                + "  ?dog <http://www.w3.org/2000/01/rdf-schema#label> ?pig "//
+                + "}";//
+
+        final String indexSparqlString2 = ""//
+                + "SELECT ?fish ?ant ?turkey " //
+                + "{" //
+                + "  ?fish <uri:talksTo> ?turkey . "//
+                + "  ?turkey <http://www.w3.org/2000/01/rdf-schema#label> ?ant "//
+                + "}";//
+
+        final String queryString = ""//
+                + "SELECT ?e ?c ?l ?o ?f ?g " //
+                + "{" //
+                + "  ?e a ?c . "//
+                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?e <uri:talksTo> ?o . "//
+                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+                + "  ?f <uri:talksTo> ?g . " //
+                + "}";//
+
+        final SPARQLParser sp = new SPARQLParser();
+        final ParsedQuery index1 = sp.parseQuery(indexSparqlString, null);
+        final ParsedQuery index2 = sp.parseQuery(indexSparqlString2, null);
+
+        final List<ExternalTupleSet> index = Lists.newArrayList();
+
+        final SimpleExternalTupleSet ais1 = new SimpleExternalTupleSet(
+                (Projection) index1.getTupleExpr());
+        final SimpleExternalTupleSet ais2 = new SimpleExternalTupleSet(
+                (Projection) index2.getTupleExpr());
+
+        index.add(ais1);
+        index.add(ais2);
+
+        final ParsedQuery pq = sp.parseQuery(queryString, null);
+        final TupleExpr tup = pq.getTupleExpr().clone();
+        provider.setIndices(index);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, provider);
+        pcj.optimize(tup, null, null);
+
+        final IndexPlanValidator ipv = new IndexPlanValidator(true);
+        Assert.assertEquals(false, ipv.isValid(tup));
+
+    }
+
+    @Test
+    public void testValidTupleIterator() throws Exception {
+
+        final String q1 = ""//
+                + "SELECT ?f ?m ?d ?h ?i " //
+                + "{" //
+                + "  ?f a ?m ."//
+                + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
+                + "  ?d <uri:talksTo> ?f . "//
+                + "  ?d <uri:hangOutWith> ?f ." //
+                + "  ?f <uri:hangOutWith> ?h ." //
+                + "  ?f <uri:associatesWith> ?i ." //
+                + "  ?i <uri:associatesWith> ?h ." //
+                + "}";//
+
+        final String q2 = ""//
+                + "SELECT ?t ?s ?u " //
+                + "{" //
+                + "  ?s a ?t ."//
+                + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
+                + "  ?u <uri:talksTo> ?s . "//
+                + "}";//
+
+        final String q3 = ""//
+                + "SELECT ?s ?t ?u " //
+                + "{" //
+                + "  ?s <uri:hangOutWith> ?t ." //
+                + "  ?t <uri:hangOutWith> ?u ." //
+                + "}";//
+
+        final String q4 = ""//
+                + "SELECT ?s ?t ?u " //
+                + "{" //
+                + "  ?s <uri:associatesWith> ?t ." //
+                + "  ?t <uri:associatesWith> ?u ." //
+                + "}";//
+
+        final SPARQLParser parser = new SPARQLParser();
+
+        final ParsedQuery pq1 = parser.parseQuery(q1, null);
+        final ParsedQuery pq2 = parser.parseQuery(q2, null);
+        final ParsedQuery pq3 = parser.parseQuery(q3, null);
+        final ParsedQuery pq4 = parser.parseQuery(q4, null);
+
+        final SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet(
+                (Projection) pq2.getTupleExpr());
+        final SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet(
+                (Projection) pq3.getTupleExpr());
+        final SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet(
+                (Projection) pq4.getTupleExpr());
+
+        final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+
+        list.add(extTup2);
+        list.add(extTup1);
+        list.add(extTup3);
+
+        final IndexedExecutionPlanGenerator iep = new IndexedExecutionPlanGenerator(
+                pq1.getTupleExpr(), list);
+
+        final Iterator<TupleExpr> plans = new TupleExecutionPlanGenerator()
+                .getPlans(iep.getIndexedTuples());
+        final IndexPlanValidator ipv = new IndexPlanValidator(true);
+        final Iterator<TupleExpr> validPlans = ipv.getValidTuples(plans);
+
+        int size = 0;
+
+        while (validPlans.hasNext()) {
+            Assert.assertTrue(validPlans.hasNext());
+            validPlans.next();
+            size++;
+        }
+
+        Assert.assertTrue(!validPlans.hasNext());
+        Assert.assertEquals(732, size);
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/ThreshholdPlanSelectorTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/ThreshholdPlanSelectorTest.java b/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/ThreshholdPlanSelectorTest.java
index b244a60..106288e 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/ThreshholdPlanSelectorTest.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/IndexPlanValidator/ThreshholdPlanSelectorTest.java
@@ -23,10 +23,11 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.apache.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
 import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
-
+import org.apache.rya.indexing.pcj.matching.provider.AccumuloIndexSetProvider;
 import org.junit.Assert;
 import org.junit.Test;
 import org.openrdf.query.algebra.Projection;
@@ -679,7 +680,7 @@ public class ThreshholdPlanSelectorTest {
 		eList.add(sep);
 
 		final TupleExpr te = pq1.getTupleExpr().clone();
-		final PCJOptimizer pcj = new PCJOptimizer(eList, false);
+		final PCJOptimizer pcj = new PCJOptimizer(eList, false, new AccumuloIndexSetProvider(new Configuration(), eList));
         pcj.optimize(te, null, null);
 
 		ThreshholdPlanSelector tps = new ThreshholdPlanSelector(

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
index 8c55e98..5d53737 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloConstantPcjIT.java
@@ -75,10 +75,10 @@ public class AccumuloConstantPcjIT {
 			TableNotFoundException, RyaDAOException, InferenceEngineException,
 			NumberFormatException, UnknownHostException, SailException {
 
-		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
+		repo = PcjIntegrationTestingUtil.getAccumuloNonPcjRepo(prefix, "instance");
 		conn = repo.getConnection();
 
-		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
+		pcjRepo = PcjIntegrationTestingUtil.getAccumuloPcjRepo(prefix, "instance");
 		pcjConn = pcjRepo.getConnection();
 
 		final URI sub = new URIImpl("uri:entity");

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
index 76560f8..d8a70aa 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/AccumuloPcjIT.java
@@ -34,8 +34,19 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.accumulo.AccumuloRdfConfiguration;
+import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
+import org.apache.rya.api.persist.RyaDAOException;
+import org.apache.rya.indexing.IndexPlanValidator.IndexPlanValidator;
+import org.apache.rya.indexing.accumulo.ConfigUtils;
+import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
+import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
+import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
+import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
+import org.apache.rya.indexing.pcj.matching.provider.AccumuloIndexSetProvider;
 import org.apache.rya.indexing.pcj.storage.PcjException;
 import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -64,17 +75,6 @@ import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-import org.apache.rya.accumulo.AccumuloRdfConfiguration;
-import org.apache.rya.api.RdfCloudTripleStoreConfiguration;
-import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.IndexPlanValidator.IndexPlanValidator;
-import org.apache.rya.indexing.accumulo.ConfigUtils;
-import org.apache.rya.indexing.external.PrecomputedJoinIndexerConfig.PrecomputedJoinStorageType;
-import org.apache.rya.indexing.external.tupleSet.AccumuloIndexSet;
-import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
-import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
-import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
-
 public class AccumuloPcjIT {
 
 	private SailRepositoryConnection conn, pcjConn;
@@ -93,10 +93,10 @@ public class AccumuloPcjIT {
 			TableNotFoundException, InferenceEngineException,
 			NumberFormatException, UnknownHostException, SailException {
 
-		repo = PcjIntegrationTestingUtil.getNonPcjRepo(prefix, "instance");
+		repo = PcjIntegrationTestingUtil.getAccumuloNonPcjRepo(prefix, "instance");
 		conn = repo.getConnection();
 
-		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(prefix, "instance");
+		pcjRepo = PcjIntegrationTestingUtil.getAccumuloPcjRepo(prefix, "instance");
 		pcjConn = pcjRepo.getConnection();
 
 		final URI sub = new URIImpl("uri:entity");
@@ -1274,7 +1274,7 @@ public class AccumuloPcjIT {
 		final List<TupleExpr> teList = Lists.newArrayList();
 		final TupleExpr te = pq.getTupleExpr();
 
-		final PCJOptimizer pcj = new PCJOptimizer(index, false);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, new AccumuloIndexSetProvider(conf));
         pcj.optimize(te, null, null);
 		teList.add(te);
 
@@ -1372,7 +1372,7 @@ public class AccumuloPcjIT {
 		final List<TupleExpr> teList = Lists.newArrayList();
 		final TupleExpr te = pq.getTupleExpr();
 
-		final PCJOptimizer pcj = new PCJOptimizer(index, false);
+        final PCJOptimizer pcj = new PCJOptimizer(index, false, new AccumuloIndexSetProvider(conf));
         pcj.optimize(te, null, null);
 
 		teList.add(te);

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c826ffea/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
index 732bc12..026fa34 100644
--- a/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
+++ b/extras/indexing/src/test/java/org/apache/rya/indexing/external/PCJOptionalTestIT.java
@@ -29,8 +29,17 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.rya.api.persist.RyaDAOException;
+import org.apache.rya.indexing.external.PrecompJoinOptimizerIT.CountingResultHandler;
+import org.apache.rya.indexing.external.PrecompJoinOptimizerTest.NodeCollector;
+import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
+import org.apache.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
+import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
+import org.apache.rya.indexing.pcj.matching.provider.AccumuloIndexSetProvider;
 import org.apache.rya.indexing.pcj.storage.PcjException;
 import org.apache.rya.indexing.pcj.storage.accumulo.PcjVarOrderFactory;
+import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -58,14 +67,6 @@ import org.openrdf.sail.SailException;
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 
-import org.apache.rya.api.persist.RyaDAOException;
-import org.apache.rya.indexing.external.PrecompJoinOptimizerIT.CountingResultHandler;
-import org.apache.rya.indexing.external.PrecompJoinOptimizerTest.NodeCollector;
-import org.apache.rya.indexing.external.tupleSet.ExternalTupleSet;
-import org.apache.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
-import org.apache.rya.indexing.pcj.matching.PCJOptimizer;
-import org.apache.rya.rdftriplestore.inference.InferenceEngineException;
-
 public class PCJOptionalTestIT {
 
 
@@ -83,10 +84,10 @@ public class PCJOptionalTestIT {
             TableNotFoundException, InferenceEngineException, NumberFormatException,
             UnknownHostException, SailException {
 
-        repo = PcjIntegrationTestingUtil.getNonPcjRepo(tablePrefix, "instance");
+        repo = PcjIntegrationTestingUtil.getAccumuloNonPcjRepo(tablePrefix, "instance");
         conn = repo.getConnection();
 
-        pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+        pcjRepo = PcjIntegrationTestingUtil.getAccumuloPcjRepo(tablePrefix, "instance");
         pcjConn = pcjRepo.getConnection();
 
         sub = new URIImpl("uri:entity");
@@ -196,7 +197,7 @@ public class PCJOptionalTestIT {
         PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
         PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
 
-        repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+        repo = PcjIntegrationTestingUtil.getAccumuloPcjRepo(tablePrefix, "instance");
         conn = repo.getConnection();
         conn.add(sub, RDF.TYPE, subclass);
         conn.add(sub2, RDF.TYPE, subclass2);
@@ -240,7 +241,7 @@ public class PCJOptionalTestIT {
         final List<QueryModelNode> optTupNodes = Lists.newArrayList();
         optTupNodes.add(extTup1);
 
-        final PCJOptimizer pcj = new PCJOptimizer(list, true);
+        final PCJOptimizer pcj = new PCJOptimizer(list, true, new AccumuloIndexSetProvider(new Configuration(), list));
         final TupleExpr te = pq1.getTupleExpr();
         pcj.optimize(te, null, null);
 
@@ -306,7 +307,7 @@ public class PCJOptionalTestIT {
         final List<QueryModelNode> optTupNodes = Lists.newArrayList();
         optTupNodes.add(extTup2);
 
-        final PCJOptimizer opt = new PCJOptimizer(list, true);
+        final PCJOptimizer opt = new PCJOptimizer(list, true, new AccumuloIndexSetProvider(new Configuration(), list));
         final TupleExpr te = pq1.getTupleExpr();
         opt.optimize(te, null, null);