You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by mi...@apache.org on 2016/02/07 19:26:08 UTC

[05/16] incubator-rya git commit: RYA-32 Improve how metadata and values are written to Accumulo PCJ tables

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java
new file mode 100644
index 0000000..d76695c
--- /dev/null
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java
@@ -0,0 +1,385 @@
+package mvm.rya.indexing.external;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import mvm.rya.accumulo.AccumuloRdfConfiguration;
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.api.resolver.RyaTypeResolverException;
+import mvm.rya.indexing.RyaSailFactory;
+import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.external.tupleSet.AccumuloPcjSerializer;
+import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
+import mvm.rya.indexing.external.tupleSet.PcjTables;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjMetadata;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjVarOrderFactory;
+import mvm.rya.indexing.external.tupleSet.PcjTables.ShiftVarOrderFactory;
+import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.data.Mutation;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.TupleQuery;
+import org.openrdf.query.TupleQueryResult;
+import org.openrdf.query.algebra.BindingSetAssignment;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
+import org.openrdf.repository.RepositoryConnection;
+import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.Sail;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Sets;
+
+public class PcjIntegrationTestingUtil {
+
+	public static Set<QueryModelNode> getTupleSets(TupleExpr te) {
+		final ExternalTupleVisitor etv = new ExternalTupleVisitor();
+		te.visit(etv);
+		return etv.getExtTup();
+	}
+
+	public static void deleteCoreRyaTables(Connector accCon, String prefix)
+			throws AccumuloException, AccumuloSecurityException,
+			TableNotFoundException {
+		final TableOperations ops = accCon.tableOperations();
+		if (ops.exists(prefix + "spo")) {
+			ops.delete(prefix + "spo");
+		}
+		if (ops.exists(prefix + "po")) {
+			ops.delete(prefix + "po");
+		}
+		if (ops.exists(prefix + "osp")) {
+			ops.delete(prefix + "osp");
+		}
+	}
+
+	public static SailRepository getPcjRepo(String tablePrefix, String instance)
+			throws AccumuloException, AccumuloSecurityException,
+			RyaDAOException, RepositoryException {
+
+		final AccumuloRdfConfiguration pcjConf = new AccumuloRdfConfiguration();
+		pcjConf.set(ConfigUtils.USE_PCJ, "true");
+		pcjConf.set(ConfigUtils.USE_MOCK_INSTANCE, "true");
+		pcjConf.set(ConfigUtils.CLOUDBASE_INSTANCE, instance);
+		pcjConf.setTablePrefix(tablePrefix);
+
+		final Sail pcjSail = RyaSailFactory.getInstance(pcjConf);
+		final SailRepository pcjRepo = new SailRepository(pcjSail);
+		pcjRepo.initialize();
+		return pcjRepo;
+	}
+
+	public static SailRepository getNonPcjRepo(String tablePrefix,
+			String instance) throws AccumuloException,
+			AccumuloSecurityException, RyaDAOException, RepositoryException {
+
+		final AccumuloRdfConfiguration nonPcjConf = new AccumuloRdfConfiguration();
+		nonPcjConf.set(ConfigUtils.USE_MOCK_INSTANCE, "true");
+		nonPcjConf.set(ConfigUtils.CLOUDBASE_INSTANCE, instance);
+		nonPcjConf.setTablePrefix(tablePrefix);
+
+		final Sail nonPcjSail = RyaSailFactory.getInstance(nonPcjConf);
+		final SailRepository nonPcjRepo = new SailRepository(nonPcjSail);
+		nonPcjRepo.initialize();
+		return nonPcjRepo;
+	}
+
+	public static void closeAndShutdown(SailRepositoryConnection connection,
+			SailRepository repo) throws RepositoryException {
+		connection.close();
+		repo.shutDown();
+	}
+
+	public static void deleteIndexTables(Connector accCon, int tableNum,
+			String prefix) throws AccumuloException, AccumuloSecurityException,
+			TableNotFoundException {
+		final TableOperations ops = accCon.tableOperations();
+		final String tablename = prefix + "INDEX_";
+		for (int i = 1; i < tableNum + 1; i++) {
+			if (ops.exists(tablename + i)) {
+				ops.delete(tablename + i);
+			}
+		}
+	}
+
+	public static class BindingSetAssignmentCollector extends
+			QueryModelVisitorBase<RuntimeException> {
+
+		private final Set<QueryModelNode> bindingSetList = Sets.newHashSet();
+
+		public Set<QueryModelNode> getBindingSetAssignments() {
+			return bindingSetList;
+		}
+
+		public boolean containsBSAs() {
+			return bindingSetList.size() > 0;
+		}
+
+		@Override
+		public void meet(BindingSetAssignment node) {
+			bindingSetList.add(node);
+			super.meet(node);
+		}
+
+	}
+
+	public static class ExternalTupleVisitor extends
+			QueryModelVisitorBase<RuntimeException> {
+
+		private final Set<QueryModelNode> eSet = new HashSet<>();
+
+		@Override
+		public void meetNode(QueryModelNode node) throws RuntimeException {
+			if (node instanceof ExternalTupleSet) {
+				eSet.add(node);
+			}
+			super.meetNode(node);
+		}
+
+		public Set<QueryModelNode> getExtTup() {
+			return eSet;
+		}
+
+	}
+
+
+
+
+
+
+//****************************Creation and Population of PcjTables***********************************
+
+
+
+
+
+	  /**
+     * Creates a new PCJ Table in Accumulo and populates it by scanning an
+     * instance of Rya for historic matches.
+     * <p>
+     * If any portion of this operation fails along the way, the partially
+     * create PCJ table will be left in Accumulo.
+     *
+     * @param ryaConn - Connects to the Rya that will be scanned. (not null)
+     * @param accumuloConn - Connects to the accumulo that hosts the PCJ results. (not null)
+     * @param pcjTableName - The name of the PCJ table that will be created. (not null)
+     * @param sparql - The SPARQL query whose results will be loaded into the table. (not null)
+     * @param resultVariables - The variables that are included in the query's resulting binding sets. (not null)
+     * @param pcjVarOrderFactory - An optional factory that indicates the various variable orders
+     *   the results will be stored in. If one is not provided, then {@link ShiftVarOrderFactory}
+     *   is used by default. (not null)
+     * @throws PcjException The PCJ table could not be create or the values from
+     *   Rya were not able to be loaded into it.
+     */
+    public static void createAndPopulatePcj(
+            final RepositoryConnection ryaConn,
+            final Connector accumuloConn,
+            final String pcjTableName,
+            final String sparql,
+            final String[] resultVariables,
+            final Optional<PcjVarOrderFactory> pcjVarOrderFactory) throws PcjException {
+        checkNotNull(ryaConn);
+        checkNotNull(accumuloConn);
+        checkNotNull(pcjTableName);
+        checkNotNull(sparql);
+        checkNotNull(resultVariables);
+        checkNotNull(pcjVarOrderFactory);
+
+        PcjTables pcj = new PcjTables();
+        // Create the PCJ's variable orders.
+        PcjVarOrderFactory varOrderFactory = pcjVarOrderFactory.or(new ShiftVarOrderFactory());
+        Set<VariableOrder> varOrders = varOrderFactory.makeVarOrders( new VariableOrder(resultVariables) );
+
+        // Create the PCJ table in Accumulo.
+        pcj.createPcjTable(accumuloConn, pcjTableName, varOrders, sparql);
+
+        // Load historic matches from Rya into the PCJ table.
+        populatePcj(accumuloConn, pcjTableName, ryaConn);
+    }
+
+
+	/**
+	 * Scan Rya for results that solve the PCJ's query and store them in the PCJ
+	 * table.
+	 * <p>
+	 * This method assumes the PCJ table has already been created.
+	 *
+	 * @param accumuloConn
+	 *            - A connection to the Accumulo that hosts the PCJ table. (not
+	 *            null)
+	 * @param pcjTableName
+	 *            - The name of the PCJ table that will receive the results.
+	 *            (not null)
+	 * @param ryaConn
+	 *            - A connection to the Rya store that will be queried to find
+	 *            results. (not null)
+	 * @throws PcjException
+	 *             If results could not be written to the PCJ table, the PCJ
+	 *             table does not exist, or the query that is being execute was
+	 *             malformed.
+	 */
+	public static void populatePcj(final Connector accumuloConn,
+			final String pcjTableName, final RepositoryConnection ryaConn)
+			throws PcjException {
+		checkNotNull(accumuloConn);
+		checkNotNull(pcjTableName);
+		checkNotNull(ryaConn);
+
+		try {
+			// Fetch the query that needs to be executed from the PCJ table.
+			PcjMetadata pcjMetadata = new PcjTables().getPcjMetadata(
+					accumuloConn, pcjTableName);
+			String sparql = pcjMetadata.getSparql();
+
+			// Query Rya for results to the SPARQL query.
+			TupleQuery query = ryaConn.prepareTupleQuery(QueryLanguage.SPARQL,
+					sparql);
+			TupleQueryResult results = query.evaluate();
+
+			// Load batches of 1000 of them at a time into the PCJ table
+			Set<BindingSet> batch = new HashSet<>(1000);
+			while (results.hasNext()) {
+				batch.add(results.next());
+
+				if (batch.size() == 1000) {
+					addResults(accumuloConn, pcjTableName, batch);
+					batch.clear();
+				}
+			}
+
+			if (!batch.isEmpty()) {
+				addResults(accumuloConn, pcjTableName, batch);
+			}
+
+		} catch (RepositoryException | MalformedQueryException
+				| QueryEvaluationException e) {
+			throw new PcjException(
+					"Could not populate a PCJ table with Rya results for the table named: "
+							+ pcjTableName, e);
+		}
+	}
+
+	public static void addResults(final Connector accumuloConn,
+			final String pcjTableName, final Collection<BindingSet> results)
+			throws PcjException {
+		checkNotNull(accumuloConn);
+		checkNotNull(pcjTableName);
+		checkNotNull(results);
+
+		// Write a result to each of the variable orders that are in the table.
+		writeResults(accumuloConn, pcjTableName, results);
+	}
+
+	/**
+	 * Add a collection of results to a specific PCJ table.
+	 *
+	 * @param accumuloConn
+	 *            - A connection to the Accumulo that hosts the PCJ table. (not
+	 *            null)
+	 * @param pcjTableName
+	 *            - The name of the PCJ table that will receive the results.
+	 *            (not null)
+	 * @param results
+	 *            - Binding sets that will be written to the PCJ table. (not
+	 *            null)
+	 * @throws PcjException
+	 *             The provided PCJ table doesn't exist, is missing the PCJ
+	 *             metadata, or the result could not be written to it.
+	 */
+	private static void writeResults(final Connector accumuloConn,
+			final String pcjTableName, final Collection<BindingSet> results)
+			throws PcjException {
+		checkNotNull(accumuloConn);
+		checkNotNull(pcjTableName);
+		checkNotNull(results);
+
+		// Fetch the variable orders from the PCJ table.
+		PcjMetadata metadata = new PcjTables().getPcjMetadata(accumuloConn,
+				pcjTableName);
+
+		// Write each result formatted using each of the variable orders.
+		BatchWriter writer = null;
+		try {
+			writer = accumuloConn.createBatchWriter(pcjTableName,
+					new BatchWriterConfig());
+			for (BindingSet result : results) {
+				Set<Mutation> addResultMutations = makeWriteResultMutations(
+						metadata.getVarOrders(), result);
+				writer.addMutations(addResultMutations);
+			}
+		} catch (TableNotFoundException | MutationsRejectedException e) {
+			throw new PcjException(
+					"Could not add results to the PCJ table named: "
+							+ pcjTableName, e);
+		} finally {
+			if (writer != null) {
+				try {
+					writer.close();
+				} catch (MutationsRejectedException e) {
+					throw new PcjException(
+							"Could not add results to a PCJ table because some of the mutations were rejected.",
+							e);
+				}
+			}
+		}
+	}
+
+	/**
+	 * Create the {@link Mutations} required to write a new {@link BindingSet}
+	 * to a PCJ table for each {@link VariableOrder} that is provided.
+	 *
+	 * @param varOrders
+	 *            - The variables orders the result will be written to. (not
+	 *            null)
+	 * @param result
+	 *            - A new PCJ result. (not null)
+	 * @return Mutation that will write the result to a PCJ table.
+	 * @throws PcjException
+	 *             The binding set could not be encoded.
+	 */
+	private static Set<Mutation> makeWriteResultMutations(
+			final Set<VariableOrder> varOrders, final BindingSet result)
+			throws PcjException {
+		checkNotNull(varOrders);
+		checkNotNull(result);
+
+		Set<Mutation> mutations = new HashSet<>();
+
+		for (final VariableOrder varOrder : varOrders) {
+			try {
+				// Serialize the result to the variable order.
+				byte[] serializedResult = AccumuloPcjSerializer.serialize(
+						result, varOrder.toArray());
+
+				// Row ID = binding set values, Column Family = variable order
+				// of the binding set.
+				Mutation addResult = new Mutation(serializedResult);
+				addResult.put(varOrder.toString(), "", "");
+				mutations.add(addResult);
+			} catch (RyaTypeResolverException e) {
+				throw new PcjException("Could not serialize a result.", e);
+			}
+		}
+
+		return mutations;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
index f7f3cbf..4eaa279 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerIntegrationTest.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.external;
  * 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
@@ -19,31 +19,21 @@ package mvm.rya.indexing.external;
  * under the License.
  */
 
-
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 
-import junit.framework.Assert;
-import mvm.rya.accumulo.AccumuloRdfConfiguration;
 import mvm.rya.api.persist.RyaDAOException;
-import mvm.rya.indexing.RyaSailFactory;
-import mvm.rya.indexing.accumulo.ConfigUtils;
-import mvm.rya.indexing.external.tupleSet.AccumuloIndexSet;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjVarOrderFactory;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.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.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.conf.Configuration;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.openrdf.model.URI;
@@ -61,490 +51,445 @@ import org.openrdf.query.TupleQueryResultHandlerException;
 import org.openrdf.repository.RepositoryException;
 import org.openrdf.repository.sail.SailRepository;
 import org.openrdf.repository.sail.SailRepositoryConnection;
-import org.openrdf.sail.Sail;
 import org.openrdf.sail.SailException;
-import org.openrdf.sail.memory.MemoryStore;
+
+import com.google.common.base.Optional;
 
 public class PrecompJoinOptimizerIntegrationTest {
 
-    private SailRepositoryConnection conn;
-    private SailRepository repo;
-    private Connector accCon;
-    String tablePrefix = "table_";
-    AccumuloRdfConfiguration conf;
-    URI sub, sub2, obj,obj2,subclass, subclass2, talksTo;
-   
-    
-    
-    
-    @Before
-    public void init() throws RepositoryException, TupleQueryResultHandlerException, QueryEvaluationException, MalformedQueryException, 
-    AccumuloException, AccumuloSecurityException, TableExistsException, RyaDAOException {
-
-        conf = new AccumuloRdfConfiguration();
-        conf.set(ConfigUtils.USE_PCJ, "true");
-        conf.set(ConfigUtils.USE_MOCK_INSTANCE,"true");
-        conf.set(ConfigUtils.CLOUDBASE_INSTANCE, "instance");
-        conf.setTablePrefix(tablePrefix);
-        
-        Sail sail = RyaSailFactory.getInstance(conf);
-        repo = new SailRepository(sail);
-        repo.initialize();
-        conn = repo.getConnection();
-
-        sub = new URIImpl("uri:entity");
-        subclass = new URIImpl("uri:class");
-        obj = new URIImpl("uri:obj");
-        talksTo = new URIImpl("uri:talksTo");
-
-        conn.add(sub, RDF.TYPE, subclass);
-        conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
-        conn.add(sub, talksTo, obj);
-
-        sub2 = new URIImpl("uri:entity2");
-        subclass2 = new URIImpl("uri:class2");
-        obj2 = new URIImpl("uri:obj2");
-
-        conn.add(sub2, RDF.TYPE, subclass2);
-        conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
-        conn.add(sub2, talksTo, obj2);
-
-        accCon = new MockInstance("instance").getConnector("root",new PasswordToken("".getBytes()));
-
-    }
-    
-    
-   @After
-   public void close() throws RepositoryException, AccumuloException, AccumuloSecurityException, TableNotFoundException {
-       
-       conf = null;
-       conn.close();
-       accCon.tableOperations().delete(tablePrefix + "spo");
-       accCon.tableOperations().delete(tablePrefix + "po");
-       accCon.tableOperations().delete(tablePrefix + "osp");
-   }
-    
-    
-    
-    @Test
-    public void testEvaluateSingeIndex() throws TupleQueryResultHandlerException, QueryEvaluationException,
-    MalformedQueryException, RepositoryException, AccumuloException, 
-    AccumuloSecurityException, TableExistsException, RyaDAOException, SailException, TableNotFoundException {
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        
-        String indexSparqlString = ""//
-                + "SELECT ?e ?l ?c " //
-                + "{" //
-                + "  ?e a ?c . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-      
-        AccumuloIndexSet ais = new AccumuloIndexSet(indexSparqlString, conn, accCon, tablePrefix + "INDEX1");
-         
-       
-        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 . "//
-                + "}";//
-
-        CountingResultHandler crh = new CountingResultHandler();       
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-        
-//        Scanner scan = accCon.createScanner(tablePrefix + "spo", new Authorizations("U"));
-//        
-//        for(Entry<Key,Value> e: scan) {
-//            System.out.println(e.getKey().getRow());
-//        }
-        
-        Assert.assertEquals(2, crh.getCount());
-        
-         
-    }
-    
-    
-    
-    
-    
-    
-    @Test
-    public void testEvaluateTwoIndexTwoVarOrder1() throws AccumuloException, AccumuloSecurityException, 
-    TableExistsException, RepositoryException, MalformedQueryException, SailException, QueryEvaluationException, 
-    TableNotFoundException, TupleQueryResultHandlerException, RyaDAOException {
-        
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX2")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX2");
-        }
-
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        accCon.tableOperations().create(tablePrefix + "INDEX2");
-        
-        
-        
-        conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-        conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-
- 
-        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 "//
-                + "}";//
-
-        AccumuloIndexSet ais1 = new AccumuloIndexSet(indexSparqlString, conn, accCon, tablePrefix + "INDEX1");
-        AccumuloIndexSet ais2 = new AccumuloIndexSet(indexSparqlString2, conn, accCon, tablePrefix + "INDEX2");
-
-        CountingResultHandler crh = new CountingResultHandler();
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-
-        Assert.assertEquals(2, crh.getCount());
-     
-
-        
-        
-    }
-    
-    
-    @Test
-    public void testEvaluateSingeFilterIndex() throws TupleQueryResultHandlerException, QueryEvaluationException,
-    MalformedQueryException, RepositoryException, AccumuloException, 
-    AccumuloSecurityException, TableExistsException, RyaDAOException, SailException, TableNotFoundException {
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        
-        String indexSparqlString = ""//
-                + "SELECT ?e ?l ?c " //
-                + "{" //
-                + "  Filter(?e = <uri:entity>) " //
-                + "  ?e a ?c . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-      
-        AccumuloIndexSet ais = new AccumuloIndexSet(indexSparqlString, conn, accCon, tablePrefix + "INDEX1");
-         
-       
-        String queryString = ""//
-                + "SELECT ?e ?c ?l ?o " //
-                + "{" //
-                + "   Filter(?e = <uri:entity>) " //
-                + "  ?e a ?c . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
-                + "  ?e <uri:talksTo> ?o . "//
-                + "}";//
-
-        CountingResultHandler crh = new CountingResultHandler();       
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-        
-        Assert.assertEquals(1, crh.getCount());
-        
-         
-    }
-    
-    
-    
-    
-    @Test
-    public void testEvaluateSingeFilterWithUnion() throws TupleQueryResultHandlerException, QueryEvaluationException,
-    MalformedQueryException, RepositoryException, AccumuloException, 
-    AccumuloSecurityException, TableExistsException, RyaDAOException, SailException, TableNotFoundException {
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX2")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX2");
-        }
-        accCon.tableOperations().create(tablePrefix + "INDEX2");
-        
-        String indexSparqlString2 = ""//
-                + "SELECT ?e ?l ?c " //
-                + "{" //
-                + "  Filter(?l = \"label2\") " //
-                + "  ?e a ?c . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-      
-        AccumuloIndexSet ais2 = new AccumuloIndexSet(indexSparqlString2, conn, accCon, tablePrefix + "INDEX2");
-         
-       
-        String queryString = ""//
-                + "SELECT ?e ?c ?o ?m ?l" //
-                + "{" //
-                + "   Filter(?l = \"label2\") " //
-                + "  ?e <uri:talksTo> ?o . "//
-                + " { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m  }"//
-                + " UNION { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-                + "}";//
-
-        CountingResultHandler crh = new CountingResultHandler();       
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-        
-        Assert.assertEquals(1, crh.getCount());
-        
-         
-    }
-    
-    
-    
-    @Test
-    public void testEvaluateSingeFilterWithLeftJoin() throws TupleQueryResultHandlerException, QueryEvaluationException,
-    MalformedQueryException, RepositoryException, AccumuloException, 
-    AccumuloSecurityException, TableExistsException, RyaDAOException, SailException, TableNotFoundException {
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        
-        String indexSparqlString1 = ""//
-                + "SELECT ?e ?l ?c " //
-                + "{" //
-                + "  Filter(?l = \"label3\") " //
-                + "  ?e a ?c . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-        
-   
-        URI sub3 = new URIImpl("uri:entity3");
-        URI subclass3 = new URIImpl("uri:class3");
-        conn.add(sub3, RDF.TYPE, subclass3);
-        conn.add(sub3,RDFS.LABEL, new LiteralImpl("label3"));
-        AccumuloIndexSet ais1 = new AccumuloIndexSet(indexSparqlString1, conn, accCon, tablePrefix + "INDEX1");
-        
-        String queryString = ""//
-                + "SELECT ?e ?c ?o ?m ?l" //
-                + "{" //
-                + "  Filter(?l = \"label3\") " //
-                + "  ?e a ?c . " //  
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . " //
-                + "  OPTIONAL { ?e <uri:talksTo> ?o . ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m }"//
-                + "}";//
-
-        CountingResultHandler crh = new CountingResultHandler();       
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-        
-        Assert.assertEquals(1, crh.getCount());
-        
-         
-    }
-    
-    
-    
-    
-    
-    
-    
-    @Test
-    public void testEvaluateTwoIndexUnionFilter() throws AccumuloException, AccumuloSecurityException, 
-    TableExistsException, RepositoryException, MalformedQueryException, SailException, QueryEvaluationException, 
-    TableNotFoundException, TupleQueryResultHandlerException, RyaDAOException {
-        
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX2")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX2");
-        }
-
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        accCon.tableOperations().create(tablePrefix + "INDEX2");
-           
-        conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-        conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-        conn.add(sub, RDF.TYPE, obj);
-        conn.add(sub2, RDF.TYPE, obj2);
-     
-     
-        String indexSparqlString = ""//
-                + "SELECT ?e ?l ?o " //
-                + "{" //
-                + "   Filter(?l = \"label2\") " //
-                + "  ?e a ?o . "//
-                + "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-
-        String indexSparqlString2 = ""//
-                + "SELECT ?e ?l ?o " //
-                + "{" //
-                + "   Filter(?l = \"label2\") " //
-                + "  ?e <uri:talksTo> ?o . "//
-                + "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
-                + "}";//
-
-        String queryString = ""//
-                + "SELECT ?c ?e ?l ?o " //
-                + "{" //
-                + "   Filter(?l = \"label2\") " //
-                + "  ?e a ?c . "//
-                + " { ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-                + " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-                + "}";//
-
-        AccumuloIndexSet ais1 = new AccumuloIndexSet(indexSparqlString, conn, accCon, tablePrefix + "INDEX1");
-        AccumuloIndexSet ais2 = new AccumuloIndexSet(indexSparqlString2, conn, accCon, tablePrefix + "INDEX2");
-
-        CountingResultHandler crh = new CountingResultHandler();
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-          
-        Assert.assertEquals(6, crh.getCount());
-     
-
-     
-    }
-    
-    
-    
-    
-    
-    @Test
-    public void testEvaluateTwoIndexLeftJoinUnionFilter() throws AccumuloException, AccumuloSecurityException, 
-    TableExistsException, RepositoryException, MalformedQueryException, SailException, QueryEvaluationException, 
-    TableNotFoundException, TupleQueryResultHandlerException, RyaDAOException {
-        
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX1")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX1");
-        }
-
-        if (accCon.tableOperations().exists(tablePrefix + "INDEX2")) {
-            accCon.tableOperations().delete(tablePrefix + "INDEX2");
-        }
-
-        accCon.tableOperations().create(tablePrefix + "INDEX1");
-        accCon.tableOperations().create(tablePrefix + "INDEX2");
-           
-        conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
-        conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
-        conn.add(sub, RDF.TYPE, obj);
-        conn.add(sub2, RDF.TYPE, obj2);
-        
-        URI livesIn = new URIImpl("uri:livesIn");
-        URI city = new URIImpl("uri:city");
-        URI city2 = new URIImpl("uri:city2");
-        URI city3 = new URIImpl("uri:city3");
-        conn.add(sub,livesIn,city);
-        conn.add(sub2,livesIn,city2);
-        conn.add(sub2,livesIn,city3);
-        conn.add(sub,livesIn,city3);
-       
-     
-        String indexSparqlString = ""//
-                + "SELECT ?e ?l ?o " //
-                + "{" //
-                + "  ?e a ?o . "//
-                + "  ?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 ?c ?e ?l ?o " //
-                + "{" //
-                + " Filter(?c = <uri:city3>) " //
-                + " ?e <uri:livesIn> ?c . "//
-                + " OPTIONAL{{ ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
-                + " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }}"//
-                + "}";//
-
-        AccumuloIndexSet ais1 = new AccumuloIndexSet(indexSparqlString, conn, accCon, tablePrefix + "INDEX1");
-        AccumuloIndexSet ais2 = new AccumuloIndexSet(indexSparqlString2, conn, accCon, tablePrefix + "INDEX2");
-
-        CountingResultHandler crh = new CountingResultHandler();
-        conn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
-
-//        Scanner scan = accCon.createScanner(tablePrefix + "spo", new Authorizations("U"));
-//        
-//        for(Entry<Key,Value> e: scan) {
-//            System.out.println(e.getKey().getRow());
-//        }
-        
-        Assert.assertEquals(6, crh.getCount());
-     
-
-     
-    }
-    
-    
-    
-    
-    public static class CountingResultHandler implements TupleQueryResultHandler {
-        private int count = 0;
-
-        public int getCount() {
-            return count;
-        }
-
-        public void resetCount() {
-            this.count = 0;
-        }
-
-        @Override
-        public void startQueryResult(List<String> arg0) throws TupleQueryResultHandlerException {
-        }
-
-
-        @Override
-        public void handleSolution(BindingSet arg0) throws TupleQueryResultHandlerException {
-            System.out.println(arg0);
-            count++;
-            System.out.println("Count is " + count);
-        }
-
-        @Override
-        public void endQueryResult() throws TupleQueryResultHandlerException {
-        }
-
-        @Override
-        public void handleBoolean(boolean arg0) throws QueryResultHandlerException {
-            // TODO Auto-generated method stub
-
-        }
-
-        @Override
-        public void handleLinks(List<String> arg0) throws QueryResultHandlerException {
-            // TODO Auto-generated method stub
-
-        }
-    }
-    
-    
-    
-    
-    
+	private SailRepositoryConnection conn, pcjConn;
+	private SailRepository repo, pcjRepo;
+	private Connector accCon;
+	String tablePrefix = "table_";
+	URI sub, sub2, obj, obj2, subclass, subclass2, talksTo;
+
+	@Before
+	public void init() throws RepositoryException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			TableNotFoundException {
+
+		repo = PcjIntegrationTestingUtil.getNonPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+
+		pcjRepo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		pcjConn = pcjRepo.getConnection();
+
+		sub = new URIImpl("uri:entity");
+		subclass = new URIImpl("uri:class");
+		obj = new URIImpl("uri:obj");
+		talksTo = new URIImpl("uri:talksTo");
+
+		conn.add(sub, RDF.TYPE, subclass);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(sub, talksTo, obj);
+
+		sub2 = new URIImpl("uri:entity2");
+		subclass2 = new URIImpl("uri:class2");
+		obj2 = new URIImpl("uri:obj2");
+
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub2, talksTo, obj2);
+
+		accCon = new MockInstance("instance").getConnector("root",
+				new PasswordToken(""));
+
+	}
+
+	@After
+	public void close() throws RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableNotFoundException {
+
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		PcjIntegrationTestingUtil.closeAndShutdown(pcjConn, pcjRepo);
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.deleteIndexTables(accCon, 2, tablePrefix);
+
+	}
+
+	@Test
+	public void testEvaluateSingeIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException {
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(crh);
+
+		Assert.assertEquals(2, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexTwoVarOrder1() throws AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			RepositoryException, MalformedQueryException, SailException,
+			QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?o ?l " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(2, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException {
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?e = <uri:entity>) " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "   Filter(?e = <uri:entity>) " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterWithUnion()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException {
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?l = \"label2\") " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?o ?m ?l" //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ " { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m  }"//
+				+ " UNION { ?e a ?c .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub2, talksTo, obj2);
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateSingeFilterWithLeftJoin()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException {
+
+		final String indexSparqlString1 = ""//
+				+ "SELECT ?e ?l ?c " //
+				+ "{" //
+				+ "  Filter(?l = \"label3\") " //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final URI sub3 = new URIImpl("uri:entity3");
+		final URI subclass3 = new URIImpl("uri:class3");
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString1, new String[] { "e", "l", "c" },
+				Optional.<PcjVarOrderFactory> absent());
+		final String queryString = ""//
+				+ "SELECT ?e ?c ?o ?m ?l" //
+				+ "{" //
+				+ "  Filter(?l = \"label3\") " //
+				+ "  ?e a ?c . " //
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . " //
+				+ "  OPTIONAL { ?e <uri:talksTo> ?o . ?e <http://www.w3.org/2000/01/rdf-schema#label> ?m }"//
+				+ "}";//
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, talksTo, obj);
+		conn.add(sub, RDFS.LABEL, new LiteralImpl("label"));
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(1, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexUnionFilter() throws AccumuloException,
+			AccumuloSecurityException, TableExistsException,
+			RepositoryException, MalformedQueryException, SailException,
+			QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, RDF.TYPE, obj);
+		conn.add(sub2, RDF.TYPE, obj2);
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e a ?o . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?e ?l ?o " //
+				+ "{" //
+				+ "   Filter(?l = \"label2\") " //
+				+ "  ?e a ?c . "//
+				+ " { ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDF.TYPE, obj2);
+		final CountingResultHandler crh = new CountingResultHandler();
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(6, crh.getCount());
+
+	}
+
+	@Test
+	public void testEvaluateTwoIndexLeftJoinUnionFilter()
+			throws AccumuloException, AccumuloSecurityException,
+			TableExistsException, RepositoryException, MalformedQueryException,
+			SailException, QueryEvaluationException, TableNotFoundException,
+			TupleQueryResultHandlerException, RyaDAOException, PcjException {
+
+		conn.add(obj, RDFS.LABEL, new LiteralImpl("label"));
+		conn.add(obj2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub, RDF.TYPE, obj);
+		conn.add(sub2, RDF.TYPE, obj2);
+
+		final URI livesIn = new URIImpl("uri:livesIn");
+		final URI city = new URIImpl("uri:city");
+		final URI city2 = new URIImpl("uri:city2");
+		final URI city3 = new URIImpl("uri:city3");
+		conn.add(sub, livesIn, city);
+		conn.add(sub2, livesIn, city2);
+		conn.add(sub2, livesIn, city3);
+		conn.add(sub, livesIn, city3);
+
+		final String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?o . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String indexSparqlString2 = ""//
+				+ "SELECT ?e ?l ?o " //
+				+ "{" //
+				+ "  ?e <uri:talksTo> ?o . "//
+				+ "  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "}";//
+
+		final String queryString = ""//
+				+ "SELECT ?c ?e ?l ?o " //
+				+ "{" //
+				+ " Filter(?c = <uri:city3>) " //
+				+ " ?e <uri:livesIn> ?c . "//
+				+ " OPTIONAL{{ ?e a ?o .  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l  }"//
+				+ " UNION { ?e <uri:talksTo> ?o .  ?o <http://www.w3.org/2000/01/rdf-schema#label> ?l  }}"//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_2", indexSparqlString2, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub2, livesIn, city3);
+		conn.add(sub, livesIn, city3);
+
+		final CountingResultHandler crh = new CountingResultHandler();
+		pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString).evaluate(
+				crh);
+
+		Assert.assertEquals(6, crh.getCount());
+
+	}
+
+	public static class CountingResultHandler implements
+			TupleQueryResultHandler {
+		private int count = 0;
+
+		public int getCount() {
+			return count;
+		}
+
+		public void resetCount() {
+			this.count = 0;
+		}
+
+		@Override
+		public void startQueryResult(List<String> arg0)
+				throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleSolution(BindingSet arg0)
+				throws TupleQueryResultHandlerException {
+			System.out.println(arg0);
+			count++;
+			System.out.println("Count is " + count);
+		}
+
+		@Override
+		public void endQueryResult() throws TupleQueryResultHandlerException {
+		}
+
+		@Override
+		public void handleBoolean(boolean arg0)
+				throws QueryResultHandlerException {
+			// TODO Auto-generated method stub
+
+		}
+
+		@Override
+		public void handleLinks(List<String> arg0)
+				throws QueryResultHandlerException {
+			// TODO Auto-generated method stub
+
+		}
+	}
+
 }
-    
-    
-    
-    

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerTest.java
index 396224f..6e8c721 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerTest.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/PrecompJoinOptimizerTest.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.external;
  * 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
@@ -23,10 +23,10 @@ package mvm.rya.indexing.external;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.Assert;
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
 import mvm.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
 
+import org.junit.Assert;
 import org.junit.Test;
 import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.QueryModelNode;
@@ -41,34 +41,34 @@ import com.beust.jcommander.internal.Lists;
 
 public class PrecompJoinOptimizerTest {
 
-    private String q7 = ""//
+    private final String q7 = ""//
             + "SELECT ?s ?t ?u " //
             + "{" //
             + "  ?s a ?t ."//
             + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
             + "  ?u <uri:talksTo> ?s . "//
             + "}";//
-    
-    private String q8 = ""//
+
+    private final String q8 = ""//
             + "SELECT ?e ?l ?c " //
             + "{" //
             + "  ?e a ?l ."//
             + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?c ."//
             + "  ?c <uri:talksTo> ?e . "//
             + "}";//
-    
-    private String q9 = ""//
+
+    private final String q9 = ""//
             + "SELECT ?f ?m ?d " //
             + "{" //
             + "  ?f a ?m ."//
             + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
             + "  ?d <uri:talksTo> ?f . "//
             + "}";//
-    
-    
-    
-    
-    private String q15 = ""//
+
+
+
+
+    private final String q15 = ""//
             + "SELECT ?f ?m ?d ?e ?l ?c " //
             + "{" //
             + "  ?f a ?m ."//
@@ -78,8 +78,8 @@ public class PrecompJoinOptimizerTest {
             + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
             + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?c ."//
             + "}";//
-    
-    private String q16 = ""//
+
+    private final String q16 = ""//
             + "SELECT ?f ?m ?d ?e ?l " //
             + "{" //
             + "  ?d <uri:talksTo> ?f . "//
@@ -87,28 +87,28 @@ public class PrecompJoinOptimizerTest {
             + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
             + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
             + "}";//
-    
-    private String q17 = ""//
+
+    private final String q17 = ""//
             + "SELECT ?chicken ?dog ?cat  " //
             + "{" //
             + "  ?chicken <uri:talksTo> ?dog . "//
             + "  ?cat <http://www.w3.org/2000/01/rdf-schema#label> ?chicken ."//
             + "}";//
-    
-    private String q18 = ""//
+
+    private final String q18 = ""//
             + "SELECT ?dog ?chicken " //
             + "{" //
             + "  ?chicken <uri:talksTo> ?dog . "//
             + "}";//
-    
-    private String q19 = ""//
+
+    private final String q19 = ""//
             + "SELECT ?cat ?chicken " //
             + "{" //
             + "  ?cat <http://www.w3.org/2000/01/rdf-schema#label> ?chicken ."//
             + "}";//
-    
-    
-    private String q20 = ""//
+
+
+    private final String q20 = ""//
             + "SELECT ?f ?m ?d ?e ?l ?c " //
             + "{" //
             + "  ?f a ?m ."//
@@ -119,68 +119,65 @@ public class PrecompJoinOptimizerTest {
             + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
             + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?c ."//
             + "}";//
-    
-    
-    
-    private String q21 = ""//
+
+
+
+    private final String q21 = ""//
             + "SELECT ?u ?s ?t " //
             + "{" //
             + "  ?s a ?t ."//
             + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
             + "  ?u <uri:talksTo> ?s . "//
             + "}";//
-    
-    
-    
+
+
+
     @Test
     public void testSingleIndex() throws Exception {
 
-        SPARQLParser parser = new SPARQLParser();
-        
+        final SPARQLParser parser = new SPARQLParser();
+
 
-        ParsedQuery pq1 = parser.parseQuery(q15, null);
-        ParsedQuery pq2 = parser.parseQuery(q7, null);
-        ParsedQuery pq3 = parser.parseQuery(q8, null);
-        ParsedQuery pq4 = parser.parseQuery(q9, null);
-       
-        SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
-        SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
-        SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
+        final ParsedQuery pq1 = parser.parseQuery(q15, null);
+        final ParsedQuery pq2 = parser.parseQuery(q7, null);
+        final ParsedQuery pq3 = parser.parseQuery(q8, null);
+        final ParsedQuery pq4 = parser.parseQuery(q9, null);
 
-        List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+        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(extTup1);
-        
-        List<QueryModelNode> optTupNodes = Lists.newArrayList();
+
+        final List<QueryModelNode> optTupNodes = Lists.newArrayList();
         optTupNodes.add(extTup2);
         optTupNodes.add(extTup3);
 
-        PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
-        TupleExpr te = pq1.getTupleExpr();
+        final PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
+        final TupleExpr te = pq1.getTupleExpr();
         pcj.optimize(te, null, null);
-        
-        NodeCollector nc = new NodeCollector();
+
+        final NodeCollector nc = new NodeCollector();
         te.visit(nc);
-        
-        List<QueryModelNode> qNodes = nc.getNodes();
-             
-        
+
+        final List<QueryModelNode> qNodes = nc.getNodes();
+
+
         Assert.assertEquals(qNodes.size(), optTupNodes.size());
-        for(QueryModelNode node: qNodes) {
+        for(final QueryModelNode node: qNodes) {
             Assert.assertTrue(optTupNodes.contains(node));
         }
-               
+
 
     }
-    
-    
-    
-    
-    
+
+
     @Test
     public void testSingleIndex2() throws Exception {
 
-        String q1 = ""//
+        final String q1 = ""//
                 + "SELECT ?f ?m ?d ?e ?l ?c " //
                 + "{" //
                 + "  ?f a ?m ."//
@@ -192,7 +189,7 @@ public class PrecompJoinOptimizerTest {
                 + "  ?m <uri:talksTo> ?e . "//
                 + "}";//
 
-        String q2 = ""//
+        final String q2 = ""//
                 + "SELECT ?u ?s ?t " //
                 + "{" //
                 + "  ?s a ?t ."//
@@ -200,7 +197,7 @@ public class PrecompJoinOptimizerTest {
                 + "  ?u <uri:talksTo> ?s . "//
                 + "}";//
 
-        String q3 = ""//
+        final String q3 = ""//
                 + "SELECT ?e ?c ?l " //
                 + "{" //
                 + "  ?c a ?l ."//
@@ -208,7 +205,7 @@ public class PrecompJoinOptimizerTest {
                 + "  ?e <uri:talksTo> ?c . "//
                 + "}";//
 
-        String q4 = ""//
+        final String q4 = ""//
                 + "SELECT ?d ?f ?m " //
                 + "{" //
                 + "  ?f a ?m ."//
@@ -216,54 +213,54 @@ public class PrecompJoinOptimizerTest {
                 + "  ?d <uri:talksTo> ?f . "//
                 + "}";//
 
-        SPARQLParser parser = new SPARQLParser();
+        final 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);
+        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);
 
-        SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
-        SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
-        SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
+        final SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
+        final SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
+        final SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
 
-        List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+        final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
 
         list.add(extTup1);
 
-        List<StatementPattern> spList = StatementPatternCollector.process(pq1.getTupleExpr());
-        List<QueryModelNode> optTupNodes = Lists.newArrayList();
+        final List<StatementPattern> spList = StatementPatternCollector.process(pq1.getTupleExpr());
+        final List<QueryModelNode> optTupNodes = Lists.newArrayList();
         optTupNodes.add(extTup3);
         optTupNodes.add(spList.get(6));
         optTupNodes.add(extTup2);
 
-        PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
-        TupleExpr te = pq1.getTupleExpr();
+        final PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
+        final TupleExpr te = pq1.getTupleExpr();
         pcj.optimize(te, null, null);
 
-        NodeCollector nc = new NodeCollector();
+        final NodeCollector nc = new NodeCollector();
         te.visit(nc);
 
         //System.out.println("Optimal plan is " + optimalTup);
 
-        List<QueryModelNode> qNodes = nc.getNodes();
+        final List<QueryModelNode> qNodes = nc.getNodes();
         //System.out.println("Returned list is " + qNodes + " and comp list is " + optTupNodes);
 
         Assert.assertTrue(qNodes.equals(optTupNodes));
 
     }
-    
-    
-    
-    
-    
-    
-    
-    
+
+
+
+
+
+
+
+
     @Test
     public void testTwoIndex() throws Exception {
 
-        String q1 = ""//
+        final String q1 = ""//
                 + "SELECT ?f ?m ?d ?h ?i " //
                 + "{" //
                 + "  ?f a ?m ."//
@@ -275,7 +272,7 @@ public class PrecompJoinOptimizerTest {
                 + "  ?i <uri:associatesWith> ?h ." //
                 + "}";//
 
-        String q2 = ""//
+        final String q2 = ""//
                 + "SELECT ?t ?s ?u " //
                 + "{" //
                 + "  ?s a ?t ."//
@@ -283,21 +280,21 @@ public class PrecompJoinOptimizerTest {
                 + "  ?u <uri:talksTo> ?s . "//
                 + "}";//
 
-        String q3 = ""//
+        final String q3 = ""//
                 + "SELECT ?s ?t ?u " //
                 + "{" //
                 + "  ?s <uri:hangOutWith> ?t ." //
                 + "  ?t <uri:hangOutWith> ?u ." //
                 + "}";//
 
-        String q4 = ""//
+        final String q4 = ""//
                 + "SELECT ?s ?t ?u " //
                 + "{" //
                 + "  ?s <uri:associatesWith> ?t ." //
                 + "  ?t <uri:associatesWith> ?u ." //
                 + "}";//
 
-        String q5 = ""//
+        final String q5 = ""//
                 + "SELECT ?m ?f ?d " //
                 + "{" //
                 + "  ?f a ?m ."//
@@ -305,73 +302,73 @@ public class PrecompJoinOptimizerTest {
                 + "  ?d <uri:talksTo> ?f . "//
                 + "}";//
 
-        String q6 = ""//
+        final String q6 = ""//
                 + "SELECT ?d ?f ?h " //
                 + "{" //
                 + "  ?d <uri:hangOutWith> ?f ." //
                 + "  ?f <uri:hangOutWith> ?h ." //
                 + "}";//
 
-        String q7 = ""//
+        final String q7 = ""//
                 + "SELECT ?f ?i ?h " //
                 + "{" //
                 + "  ?f <uri:associatesWith> ?i ." //
                 + "  ?i <uri:associatesWith> ?h ." //
                 + "}";//
 
-        SPARQLParser parser = new SPARQLParser();
+        final 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);
-        ParsedQuery pq5 = parser.parseQuery(q5, null);
-        ParsedQuery pq6 = parser.parseQuery(q6, null);
-        ParsedQuery pq7 = parser.parseQuery(q7, null);
+        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 ParsedQuery pq5 = parser.parseQuery(q5, null);
+        final ParsedQuery pq6 = parser.parseQuery(q6, null);
+        final ParsedQuery pq7 = parser.parseQuery(q7, null);
 
-        SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
-        SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
-        SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
-        SimpleExternalTupleSet extTup4 = new SimpleExternalTupleSet((Projection) pq5.getTupleExpr());
-        SimpleExternalTupleSet extTup5 = new SimpleExternalTupleSet((Projection) pq6.getTupleExpr());
-        SimpleExternalTupleSet extTup6 = new SimpleExternalTupleSet((Projection) pq7.getTupleExpr());
+        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 SimpleExternalTupleSet extTup4 = new SimpleExternalTupleSet((Projection) pq5.getTupleExpr());
+        final SimpleExternalTupleSet extTup5 = new SimpleExternalTupleSet((Projection) pq6.getTupleExpr());
+        final SimpleExternalTupleSet extTup6 = new SimpleExternalTupleSet((Projection) pq7.getTupleExpr());
 
-        List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+        final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
 
         list.add(extTup2);
         list.add(extTup1);
         list.add(extTup3);
 
-        List<QueryModelNode> optTupNodes = Lists.newArrayList();
+        final List<QueryModelNode> optTupNodes = Lists.newArrayList();
         optTupNodes.add(extTup4);
         optTupNodes.add(extTup6);
         optTupNodes.add(extTup5);
 
-        PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
-        TupleExpr te = pq1.getTupleExpr();
+        final PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
+        final TupleExpr te = pq1.getTupleExpr();
         pcj.optimize(te, null, null);
 
         System.out.println(te);
 
-        NodeCollector nc = new NodeCollector();
+        final NodeCollector nc = new NodeCollector();
         te.visit(nc);
 
-        List<QueryModelNode> qNodes = nc.getNodes();
+        final List<QueryModelNode> qNodes = nc.getNodes();
 
         Assert.assertTrue(qNodes.equals(optTupNodes));
 
     }
-    
-     
 
-    
-    
-    
+
+
+
+
+
     @Test
     public void twoIndexFilterTest() {
-        
-        
-        String q1 = ""//
+
+
+        final String q1 = ""//
                 + "SELECT ?f ?m ?d ?e ?l ?c " //
                 + "{" //
                 + "  Filter(?f > \"5\")." //
@@ -383,59 +380,59 @@ public class PrecompJoinOptimizerTest {
                 + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
                 + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?c ."//
                 + "}";//
-        
-        
-        String q2 = ""//
+
+
+        final String q2 = ""//
                 + "SELECT ?s ?t ?u " //
                 + "{" //
                 + "  ?s a ?t ."//
                 + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
                 + "  ?u <uri:talksTo> ?s . "//
                 + "}";//
-        
-        
-        String q3 = ""//
+
+
+        final String q3 = ""//
                 + "SELECT ?s ?t ?u " //
                 + "{" //
-                + " Filter(?s > \"5\") ."// 
+                + " Filter(?s > \"5\") ."//
                 + "  ?s a ?t ."//
                 + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
                 + "  ?u <uri:talksTo> ?s . "//
                 + "}";//
-        
-        
-        
-        String q4 = ""//
+
+
+
+        final String q4 = ""//
                 + "SELECT ?f ?m ?d " //
                 + "{" //
-                + " Filter(?f > \"5\") ."// 
+                + " Filter(?f > \"5\") ."//
                 + "  ?f a ?m ."//
                 + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
                 + "  ?d <uri:talksTo> ?f . "//
                 + "}";//
-        
-        
-        String q5 = ""//
+
+
+        final String q5 = ""//
                 + "SELECT ?e ?l ?c " //
                 + "{" //
-                + " Filter(?e > \"5\") ."// 
+                + " Filter(?e > \"5\") ."//
                 + "  ?e a ?l ."//
                 + "  ?l <http://www.w3.org/2000/01/rdf-schema#label> ?c ."//
                 + "  ?c <uri:talksTo> ?e . "//
                 + "}";//
-        
-        
-        
-        
-        SPARQLParser parser = new SPARQLParser();
+
+
+
+
+        final SPARQLParser parser = new SPARQLParser();
 
         ParsedQuery pq1 = null;
         ParsedQuery pq2 = null;
         ParsedQuery pq3 = null;
         ParsedQuery pq4 = null;
         ParsedQuery pq5 = null;
-        
-       
+
+
 
         try {
             pq1 = parser.parseQuery(q1, null);
@@ -443,53 +440,53 @@ public class PrecompJoinOptimizerTest {
             pq3 = parser.parseQuery(q3, null);
             pq4 = parser.parseQuery(q4, null);
             pq5 = parser.parseQuery(q5, null);
-            
-           
-        } catch (Exception e) {
+
+
+        } catch (final Exception e) {
             e.printStackTrace();
         }
 
-        SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
-        SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
-        SimpleExternalTupleSet extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
-        SimpleExternalTupleSet extTup4 = new SimpleExternalTupleSet((Projection) pq5.getTupleExpr());
-     
-        List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+        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 SimpleExternalTupleSet extTup4 = new SimpleExternalTupleSet((Projection) pq5.getTupleExpr());
+
+        final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
 
         list.add(extTup2);
         list.add(extTup1);
-        
-        List<ExternalTupleSet> list2 = new ArrayList<ExternalTupleSet>();
+
+        final List<ExternalTupleSet> list2 = new ArrayList<ExternalTupleSet>();
 
         list2.add(extTup3);
         list2.add(extTup4);
-      
-        PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
-        TupleExpr te = pq1.getTupleExpr();
+
+        final PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
+        final TupleExpr te = pq1.getTupleExpr();
         pcj.optimize(te, null, null);
 
         System.out.println(te);
-        
-        NodeCollector nc = new NodeCollector();
+
+        final NodeCollector nc = new NodeCollector();
         te.visit(nc);
 
         Assert.assertEquals(nc.getNodes().size(), list2.size());
 
-        for(QueryModelNode e: nc.getNodes()) {
-            Assert.assertTrue(list2.contains((ExternalTupleSet)e));
+        for(final QueryModelNode e: nc.getNodes()) {
+            Assert.assertTrue(list2.contains(e));
         }
-        
-        
-        
+
+
+
     }
-    
-
-    
-    
-    
-    
- 
-    
+
+
+
+
+
+
+
+
     public static class NodeCollector extends QueryModelVisitorBase<RuntimeException> {
 
         List<QueryModelNode> qNodes = Lists.newArrayList();
@@ -498,24 +495,24 @@ public class PrecompJoinOptimizerTest {
         public List<QueryModelNode> getNodes() {
             return qNodes;
         }
-        
-       
+
+
 
         @Override
         public void meetNode(QueryModelNode node) {
             if(node instanceof StatementPattern || node instanceof ExternalTupleSet) {
                 qNodes.add(node);
-            } 
+            }
             super.meetNode(node);
-            
+
         }
 
-     
+
     }
-    
-    
-    
-    
+
+
+
+
 
 
 }