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

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

RYA-32 Improve how metadata and values are written to Accumulo PCJ tables


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

Branch: refs/heads/develop
Commit: c12f58f46be0aed3c88eff7504d4bfd4e4f295e9
Parents: 4b7bd4f
Author: Caleb Meier <me...@gmail.com>
Authored: Fri Jan 29 11:40:59 2016 -0500
Committer: Caleb Meier <me...@gmail.com>
Committed: Fri Jan 29 11:40:59 2016 -0500

----------------------------------------------------------------------
 extras/indexing/pom.xml                         |    7 +
 .../accumulo/precompQuery/AccumuloPcjQuery.java |  352 ++
 .../AccumuloPrecompQueryIndexer.java            |  326 --
 .../GeneralizedExternalProcessor.java           |  156 +-
 .../IndexPlanValidator/IndexListPruner.java     |   12 +-
 .../IndexedExecutionPlanGenerator.java          |  109 +-
 .../ValidIndexCombinationGenerator.java         | 1054 ++---
 .../VarConstantIndexListPruner.java             |   61 +-
 .../main/java/mvm/rya/indexing/PcjQuery.java    |   40 +
 .../mvm/rya/indexing/PrecompQueryIndexer.java   |   63 -
 .../accumulo/entity/EntityOptimizer.java        |   48 +-
 .../accumulo/entity/EntityTupleSet.java         |   73 +-
 .../rya/indexing/accumulo/entity/StarQuery.java |  342 +-
 .../rya/indexing/accumulo/geo/GeoTupleSet.java  |   53 +-
 .../indexing/external/ExternalIndexMain.java    |  219 -
 .../indexing/external/ExternalProcessor.java    |  726 ---
 .../mvm/rya/indexing/external/ExternalSail.java |   86 -
 .../indexing/external/ExternalSailExample.java  |  124 -
 .../indexing/external/PrecompJoinOptimizer.java | 1474 +++---
 .../external/tupleSet/AccumuloIndexSet.java     |  775 ++--
 .../tupleSet/AccumuloPcjSerializer.java         |  103 +
 .../external/tupleSet/ExternalTupleSet.java     |  241 +-
 .../indexing/external/tupleSet/PcjTables.java   |  800 ++++
 .../tupleSet/SimpleExternalTupleSet.java        |  118 +-
 .../GeneralizedExternalProcessorTest.java       |  418 +-
 .../IndexPlanValidatorTest.java                 | 1856 +++-----
 .../IndexedExecutionPlanGeneratorTest.java      |  717 ++-
 .../ThreshholdPlanSelectorTest.java             | 1465 +++---
 .../TupleExecutionPlanGeneratorTest.java        |  138 +-
 .../IndexPlanValidator/TupleReArrangerTest.java |   83 +-
 .../ValidIndexCombinationGeneratorTest.java     | 1083 ++---
 .../VarConstantIndexListPrunerTest.java         |  547 ++-
 .../external/AccumuloConstantIndexSetTest.java  |  831 ----
 .../AccumuloConstantPcjIntegrationTest.java     |  410 ++
 .../indexing/external/AccumuloIndexSetTest.java | 4330 ------------------
 .../external/AccumuloIndexSetTest2.java         |  803 ----
 .../external/AccumuloPcjIntegrationTest.java    | 1426 ++++++
 .../external/PcjIntegrationTestingUtil.java     |  385 ++
 .../PrecompJoinOptimizerIntegrationTest.java    |  939 ++--
 .../external/PrecompJoinOptimizerTest.java      |  361 +-
 .../external/PrecompJoinOptimizerTest2.java     | 1130 +++++
 .../PrecompJoinOptimizerVarToConstTest.java     |  430 ++
 .../external/tupleSet/AccumuloIndexSetTest.java |  678 +++
 .../tupleSet/AccumuloPcjSerialzerTest.java      |  114 +
 .../tupleSet/ExternalProcessorTest.java         | 1654 -------
 .../tupleSet/PcjTablesIntegrationTests.java     |  413 ++
 .../external/tupleSet/PcjTablesTests.java       |   65 +
 .../tupleSet/VarConstExternalProcessorTest.java |  490 --
 .../src/main/java/RyaDirectExample.java         | 1521 +++---
 49 files changed, 12568 insertions(+), 17081 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/extras/indexing/pom.xml b/extras/indexing/pom.xml
index f484916..3bfa0ce 100644
--- a/extras/indexing/pom.xml
+++ b/extras/indexing/pom.xml
@@ -81,6 +81,13 @@ under the License.
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
+        
+        <dependency>
+            <groupId>org.apache.accumulo</groupId>
+            <artifactId>accumulo-minicluster</artifactId>
+            <version>${accumulo.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java b/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java
new file mode 100644
index 0000000..08c0f78
--- /dev/null
+++ b/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java
@@ -0,0 +1,352 @@
+package mvm.rya.accumulo.precompQuery;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ *
+ */
+import info.aduna.iteration.CloseableIteration;
+import info.aduna.iteration.Iteration;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import mvm.rya.api.resolver.RyaTypeResolverException;
+import mvm.rya.indexing.PcjQuery;
+import mvm.rya.indexing.external.tupleSet.AccumuloIndexSet;
+import mvm.rya.indexing.external.tupleSet.AccumuloPcjSerializer;
+import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
+
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+import org.openrdf.query.BindingSet;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+/**
+ * This class encapsulates how pre-computed join tables are used during query
+ * evaluation. The method
+ * {@link AccumuloPcjQuery#queryPrecompJoin(List, String, Map, Map, Collection)
+ * is used by {@link AccumuloIndexSet#evaluate(BindingSet)} to evaluate the
+ * {@link AccumuloIndexSet#getTupleExpr()} associated with the Accumulo
+ * pre-computed join table. Given the {@link BindingSet} constraints, it uses
+ * the variables common to the BindingSet constraints and the pre-computed join
+ * table TupleExpr to build a {@Range} prefix to scan the pre-computed
+ * join table to obtain results for the constrained sub-query.
+ *
+ */
+public class AccumuloPcjQuery implements PcjQuery {
+	private final Connector accCon;
+	private final String tableName;
+
+	public AccumuloPcjQuery(Connector accCon, String tableName) {
+		this.accCon = accCon;
+		this.tableName = tableName;
+	}
+
+	/**
+	 * @param commonVars - variables common to bsConstraints and table in terms of query variables
+	 * @param localityGroup - the column family to scan in terms of table variables
+	 * @param valMap - Literal type associated with constant constraints
+	 * @param varMap - map query variables to pre-computed join table variables
+	 * @param bsConstraints - binding set constraints
+	 * @return {@link Iteration} over result BindingSets
+	 */
+	@Override
+	public CloseableIteration<BindingSet, QueryEvaluationException> queryPrecompJoin(
+			List<String> commonVars, String localityGroup,
+			Map<String, org.openrdf.model.Value> valMap,
+			Map<String, String> varMap, Collection<BindingSet> bsConstraints)
+			throws QueryEvaluationException, TableNotFoundException {
+
+		final Iterator<Entry<Key, Value>> accIter;
+		final Map<String, org.openrdf.model.Value> constValMap = valMap;
+		final HashMultimap<Range, BindingSet> map = HashMultimap.create();
+
+		final List<BindingSet> extProdList = Lists.newArrayList();
+		final List<String> prefixVars = commonVars;
+		final BatchScanner bs = accCon.createBatchScanner(tableName,
+				new Authorizations(), 10);
+		final Set<Range> ranges = Sets.newHashSet();
+		final Map<String, String> tableVarMap = varMap;
+		final boolean bsContainsPrefixVar = bindingsContainsPrefixVar(
+				bsConstraints, prefixVars);
+
+		bs.fetchColumnFamily(new Text(localityGroup));
+		// process bindingSet and constant constraints
+		for (final BindingSet bSet : bsConstraints) {
+			// bindings sets and PCJ have common vars
+			if (bsContainsPrefixVar) {
+				byte[] rangePrefix = null;
+				final QueryBindingSet rangeBs = new QueryBindingSet();
+				for (final String var : prefixVars) {
+					if (var.startsWith(ExternalTupleSet.CONST_PREFIX)) {
+						rangeBs.addBinding(var, constValMap.get(var));
+					} else {
+						rangeBs.addBinding(var, bSet.getBinding(var).getValue());
+					}
+				}
+				try {
+					rangePrefix = AccumuloPcjSerializer.serialize(rangeBs,
+							commonVars.toArray(new String[commonVars.size()]));
+				} catch (final RyaTypeResolverException e) {
+					e.printStackTrace();
+				}
+				final Range r = Range.prefix(new Text(rangePrefix));
+				map.put(r, bSet);
+				ranges.add(r);
+				// non-empty binding sets and no common vars with no constant
+				// constraints
+			} else if (bSet.size() > 0 && prefixVars.size() == 0) {
+				extProdList.add(bSet);
+			}
+		}
+
+		// constant constraints and no bindingSet constraints
+		// add range of entire table if no constant constraints and
+		// bsConstraints consists of single, empty set (occurs when AIS is
+		// first node evaluated in query)
+		if (ranges.isEmpty()) {
+			// constant constraints
+			if (prefixVars.size() > 0) {
+				byte[] rangePrefix = null;
+				final QueryBindingSet rangeBs = new QueryBindingSet();
+				for (final String var : prefixVars) {
+					if (var.startsWith(ExternalTupleSet.CONST_PREFIX)) {
+						rangeBs.addBinding(var, constValMap.get(var));
+					}
+				}
+				try {
+					rangePrefix = AccumuloPcjSerializer.serialize(rangeBs,
+							commonVars.toArray(new String[commonVars.size()]));
+				} catch (final RyaTypeResolverException e) {
+					e.printStackTrace();
+				}
+				final Range r = Range.prefix(new Text(rangePrefix));
+				ranges.add(r);
+			}
+			// no constant or bindingSet constraints
+			else {
+				ranges.add(new Range("", true, "~", false));
+			}
+		}
+
+		if (ranges.size() == 0) {
+			accIter = null;
+		} else {
+			bs.setRanges(ranges);
+			accIter = bs.iterator();
+		}
+		return new CloseableIteration<BindingSet, QueryEvaluationException>() {
+			@Override
+			public void remove() throws QueryEvaluationException {
+				throw new UnsupportedOperationException();
+			}
+
+			private Iterator<BindingSet> inputSet = null;
+			private QueryBindingSet currentSolutionBs = null;
+			private boolean hasNextCalled = false;
+			private boolean isEmpty = false;
+
+			@Override
+			public BindingSet next() throws QueryEvaluationException {
+				final QueryBindingSet bs = new QueryBindingSet();
+				if (hasNextCalled) {
+					hasNextCalled = false;
+					if (inputSet != null) {
+						bs.addAll(inputSet.next());
+					}
+					bs.addAll(currentSolutionBs);
+				} else if (isEmpty) {
+					throw new NoSuchElementException();
+				} else {
+					if (this.hasNext()) {
+						hasNextCalled = false;
+						if (inputSet != null) {
+							bs.addAll(inputSet.next());
+						}
+						bs.addAll(currentSolutionBs);
+					} else {
+						throw new NoSuchElementException();
+					}
+				}
+				return bs;
+			}
+
+			@Override
+			public boolean hasNext() throws QueryEvaluationException {
+				if (accIter == null) {
+					isEmpty = true;
+					return false;
+				}
+				if (!hasNextCalled && !isEmpty) {
+					while (accIter.hasNext() || inputSet != null
+							&& inputSet.hasNext()) {
+						if (inputSet != null && inputSet.hasNext()) {
+							hasNextCalled = true;
+							return true;
+						}
+						final Key k = accIter.next().getKey();
+						// get bindings from scan without values associated with
+						// constant constraints
+						BindingSet bs;
+						try {
+							bs = getBindingSetWithoutConstants(k, tableVarMap);
+						} catch (final RyaTypeResolverException e) {
+							throw new QueryEvaluationException(e);
+						}
+						currentSolutionBs = new QueryBindingSet();
+						currentSolutionBs.addAll(bs);
+
+						// check to see if additional bindingSet constraints
+						// exist in map
+						if (map.size() > 0) {
+							// get prefix range to retrieve remainder of
+							// bindingSet from map
+							byte[] rangePrefix;
+							try {
+								rangePrefix = getPrefixByte(bs, constValMap,
+										prefixVars);
+							} catch (final RyaTypeResolverException e) {
+								throw new QueryEvaluationException(e);
+							}
+							final Range r = Range.prefix(new Text(rangePrefix));
+							inputSet = map.get(r).iterator();
+							if (!inputSet.hasNext()) {
+								continue;
+							} else {
+								hasNextCalled = true;
+								return true;
+							}
+							// check to see if binding set constraints exist,
+							// but no common vars
+						} else if (extProdList.size() > 0) {
+							inputSet = extProdList.iterator();
+							hasNextCalled = true;
+							return true;
+						}
+						// no bindingsSet constraints--only constant constraints
+						// or none
+						else {
+							hasNextCalled = true;
+							return true;
+						}
+					}
+					isEmpty = true;
+					return false;
+				} else if (isEmpty) {
+					return false;
+				} else {
+					return true;
+				}
+			}
+
+			@Override
+			public void close() throws QueryEvaluationException {
+				bs.close();
+			}
+		};
+	}
+
+	/**
+	 *
+	 * @param bindingSets - binding set constraints
+	 * @param prefixVars - common prefix variables to table and any constant constraints
+	 * @return true if there are variables common to binding sets and table and false
+	 * if prefixVars only consists of constant constraints
+	 */
+	private boolean bindingsContainsPrefixVar(
+			Collection<BindingSet> bindingSets, List<String> prefixVars) {
+		final Iterator<BindingSet> iter = bindingSets.iterator();
+		if (iter.hasNext()) {
+			final BindingSet tempBindingSet = iter.next();
+			final Set<String> bindings = tempBindingSet.getBindingNames();
+			for (final String var : prefixVars) {
+				if (bindings.contains(var)) {
+					return true;
+				}
+			}
+		}
+		return false;
+	}
+
+	/**
+	 *
+	 * @param bs - binding set from which byte is extracted
+	 * @param valMap - map which associated Value type to constant constraint
+	 * @param prefixVars - prefix of variables common to binding sets and table and constant constraints
+	 * @return - bytes associated with values in bs that are associated with prefixVars
+	 * @throws RyaTypeResolverException
+	 */
+	private static byte[] getPrefixByte(BindingSet bs,
+			Map<String, org.openrdf.model.Value> valMap, List<String> prefixVars)
+			throws RyaTypeResolverException {
+		final QueryBindingSet bSet = new QueryBindingSet();
+		for (final String var : prefixVars) {
+			if (var.startsWith(ExternalTupleSet.CONST_PREFIX)) {
+				bSet.addBinding(var, valMap.get(var));
+			} else if (bs.getBindingNames().size() > 0
+					&& bs.getBinding(var) != null) {
+				bSet.addBinding(var, bs.getBinding(var).getValue());
+			}
+		}
+		return AccumuloPcjSerializer.serialize(bSet,
+				prefixVars.toArray(new String[prefixVars.size()]));
+	}
+
+	/**
+	 *
+	 * @param key - Accumulo key obtained from scan
+	 * @param tableVarMap - map that associated query variables and table variables
+	 * @return - BindingSet without values associated with constant constraints
+	 * @throws RyaTypeResolverException
+	 */
+	private static BindingSet getBindingSetWithoutConstants(Key key,
+			Map<String, String> tableVarMap) throws RyaTypeResolverException {
+		final byte[] row = key.getRow().getBytes();
+		final String[] varOrder = key.getColumnFamily().toString()
+				.split(ExternalTupleSet.VAR_ORDER_DELIM);
+		final QueryBindingSet temp = new QueryBindingSet(
+				AccumuloPcjSerializer.deSerialize(row, varOrder));
+		final QueryBindingSet bs = new QueryBindingSet();
+		for (final String var : temp.getBindingNames()) {
+			if (!tableVarMap.get(var).startsWith(ExternalTupleSet.CONST_PREFIX)) {
+				bs.addBinding(tableVarMap.get(var), temp.getValue(var));
+			}
+		}
+		return bs;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPrecompQueryIndexer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPrecompQueryIndexer.java b/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPrecompQueryIndexer.java
deleted file mode 100644
index 86cb73e..0000000
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPrecompQueryIndexer.java
+++ /dev/null
@@ -1,326 +0,0 @@
-package mvm.rya.accumulo.precompQuery;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-import info.aduna.iteration.CloseableIteration;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.openrdf.query.Binding;
-import org.openrdf.query.BindingSet;
-import org.openrdf.query.QueryEvaluationException;
-import org.openrdf.query.algebra.evaluation.QueryBindingSet;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-import mvm.rya.indexing.PrecompQueryIndexer;
-import mvm.rya.indexing.external.tupleSet.AccumuloIndexSet.AccValueFactory;
-
-public class AccumuloPrecompQueryIndexer implements PrecompQueryIndexer {
-
-    
-    private Connector accCon;
-    private String tableName;
-    private Map<String, AccValueFactory> bindings;
-  
-    
-    
-    public AccumuloPrecompQueryIndexer(Connector accCon, String tableName) {
-        this.accCon = accCon;
-        this.tableName = tableName;
-    }
-    
-    
-    @Override
-    public void storeBindingSet(BindingSet bs) throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void storeBindingSets(Collection<BindingSet> bindingSets) throws IOException, IllegalArgumentException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public CloseableIteration<BindingSet, QueryEvaluationException> queryPrecompJoin(List<String> varOrder,
-            String localityGroup, Map<String, AccValueFactory> bindings, Map<String, org.openrdf.model.Value> valMap, Collection<BindingSet> bsConstraints) 
-                    throws QueryEvaluationException, TableNotFoundException {
-        
-        
-        final int prefixLength = Integer.parseInt(varOrder.remove(varOrder.size()-1));
-        final Iterator<Entry<Key,Value>> accIter;
-        final HashMultimap<Range,BindingSet> map = HashMultimap.create();
-        final List<BindingSet> extProdList = Lists.newArrayList();
-        final Map<String, AccValueFactory> bindingMap = bindings;
-        final List<String> order = varOrder;
-        final BatchScanner bs = accCon.createBatchScanner(tableName, new Authorizations(), 10);
-        final Set<Range> ranges = Sets.newHashSet();
-        
-        
-        
-        bs.fetchColumnFamily(new Text(localityGroup));
-        
-        //process bindingSet and constant constraints
-        for (BindingSet bSet : bsConstraints) {
-            StringBuffer rangePrefix = new StringBuffer();
-            int i = 0;
-
-            for (String b : order) {
-
-                if (i >= prefixLength) {
-                    break;
-                }
-
-                if (b.startsWith("-const-")) {
-                    String val = bindings.get(b).create(valMap.get(b));
-                    rangePrefix.append(val);
-                    rangePrefix.append("\u0000");
-                } else {
-
-                    Binding v = bSet.getBinding(b);
-                    if (v == null) {
-                        throw new IllegalStateException("Binding set can't have null value!");
-                    }
-                    String val = bindings.get(b).create(bSet.getValue(b));
-                    rangePrefix.append(val);
-                    rangePrefix.append("\u0000");
-
-                }
-
-                i++;
-
-            }
-            if (rangePrefix.length() > 0) {
-                String prefixWithOutNull = rangePrefix.deleteCharAt(rangePrefix.length() - 1).toString();
-                String prefixWithNull = prefixWithOutNull + "\u0001";
-                Range r = new Range(new Key(prefixWithOutNull), true, new Key(prefixWithNull), false);
-                map.put(r, bSet);
-                ranges.add(r);
-            } else if (bSet.size() > 0) {
-                extProdList.add(bSet);
-            }
-        }
-        
-        //constant constraints and no bindingSet constraints
-        //add range of entire table if no constant constraints and
-        //bsConstraints consists of single, empty set (occurs when AIS is
-        //first node evaluated in query)
-        if (ranges.isEmpty() && bsConstraints.size() > 0) {
-
-            if (prefixLength > 0) {
-                StringBuffer rangePrefix = new StringBuffer();
-
-                int i = 0;
-                for (String b : order) {
-                    if (i >= prefixLength) {
-                        break;
-                    }
-                    if (b.startsWith("-const-")) {
-                        String val = bindings.get(b).create(valMap.get(b));
-                        rangePrefix.append(val);
-                        rangePrefix.append("\u0000");
-                    } 
-                    i++;
-                }
-
-                String prefixWithOutNull = rangePrefix.deleteCharAt(rangePrefix.length() - 1).toString();
-                String prefixWithNull = prefixWithOutNull + "\u0001";
-                Range r = new Range(new Key(prefixWithOutNull), true, new Key(prefixWithNull), false);
-                ranges.add(r);
-
-            } else { // no constant or bindingSet constraints
-                ranges.add(new Range("", true, "~", false));
-            }
-        }
-        
-        if (ranges.size() == 0) {
-            accIter = null;
-        } else {
-            bs.setRanges(ranges);
-            accIter = bs.iterator();
-        }
-
-   
-        return new CloseableIteration<BindingSet, QueryEvaluationException>() {
-
-            @Override
-            public void remove() throws QueryEvaluationException {
-                throw new UnsupportedOperationException();
-            }
-
-            private Iterator<BindingSet> inputSet = null;
-            private QueryBindingSet currentSolutionBs = null;
-            private boolean hasNextCalled = false;
-            private boolean isEmpty = false;
-           
-
-            
-            @Override
-            public BindingSet next() throws QueryEvaluationException {
-                QueryBindingSet bs = new QueryBindingSet();
-
-                if (hasNextCalled) {
-                    hasNextCalled = false;
-                    if (inputSet != null) {
-                        bs.addAll(inputSet.next());
-                    }
-                    bs.addAll(currentSolutionBs);
-                } else if (isEmpty) {
-                    throw new NoSuchElementException();
-                } else {
-                    if (this.hasNext()) {
-                        hasNextCalled = false;
-                        if (inputSet != null) {
-                            bs.addAll(inputSet.next());
-                        }
-                        bs.addAll(currentSolutionBs);
-                    } else {
-                        throw new NoSuchElementException();
-                    }
-                }
-
-                return bs;
-            }
-
-            @Override
-            public boolean hasNext() throws QueryEvaluationException {
-
-                if(accIter == null ) {
-                    isEmpty = true;
-                    return false;
-                }
-                
-                if (!hasNextCalled && !isEmpty) {
-                    while (accIter.hasNext() || (inputSet != null && inputSet.hasNext())) {
-
-                        if(inputSet != null && inputSet.hasNext()) {
-                            hasNextCalled = true;
-                            return true;
-                        }
-                        
-                        
-                        Key k = accIter.next().getKey();
-                        final String[] s = k.getRow().toString().split("\u0000");
-                       
-                        StringBuilder rangePrefix = new StringBuilder();
-                        // TODO Assuming that order specifies order of variables
-                        // commmon to
-                        // bindingSet passed in and variables in index table
-                        // --size is equal to
-                        
-                        for (int i = 0; i < prefixLength; i++) {
-                            rangePrefix.append(s[i]);
-                            rangePrefix.append("\u0000");
-                        }
-
-                        // TODO I need to remember what the type was!
-                        currentSolutionBs = new QueryBindingSet();
-                        int i = 0;
-                        for (String b : order) {
-                            if (b.startsWith("-const")) {
-                                i++;
-                            } else {
-                                final String v = s[i];
-                                currentSolutionBs.addBinding(b, bindingMap.get(b).create(v));
-                                i++;
-                            }
-
-                        }
-                        //check to see if bindingSet constraints exist
-                        if (map.size() > 0) {
-                            String prefixWithOutNull = rangePrefix.deleteCharAt(rangePrefix.length() - 1).toString();
-                            String prefixWithNull = prefixWithOutNull + "\u0001";
-                            Range r = new Range(new Key(prefixWithOutNull), true, new Key(prefixWithNull), false);
-                            inputSet = map.get(r).iterator();
-                            if (!inputSet.hasNext()) {
-                                continue;
-                            } else {
-                                hasNextCalled = true;
-                                return true;
-                            } // check to see if binding set constraints exist, but no common vars
-                        } else if (extProdList.size() > 0) {
-                            inputSet = extProdList.iterator();
-                            hasNextCalled = true;
-                            return true;
-                        }else {  //no bindingsSet constraints--only constant constraints or none
-                            hasNextCalled = true;
-                            return true;
-                        }
-                    }
-
-                    isEmpty = true;
-                    return false;
-
-                } else if (isEmpty) {
-                    return false;
-                } else {
-                    return true;
-                }
-
-            }
-
-            @Override
-            public void close() throws QueryEvaluationException {
-                bs.close();
-            }
-
-        };
-    }
-
-    
-    
-    @Override
-    public void flush() throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void close() throws IOException {
-        // TODO Auto-generated method stub
-
-    }
-    
-    
-    
-    
-    
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/GeneralizedExternalProcessor.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/GeneralizedExternalProcessor.java b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/GeneralizedExternalProcessor.java
index 27a0d15..58e84d9 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/GeneralizedExternalProcessor.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/GeneralizedExternalProcessor.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.IndexPlanValidator;
  * 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
@@ -53,29 +53,26 @@ public class GeneralizedExternalProcessor {
 
     /**
      * Iterates through list of normalized indexes and replaces all subtrees of query which match index with index.
-     * 
+     *
      * @param query
      * @return TupleExpr
      */
     public static TupleExpr process(TupleExpr query, List<ExternalTupleSet> indexSet) {
-        
+
         boolean indexPlaced = false;
         TupleExpr rtn = query.clone();
-                
-        
-        //TODO optimization: turn on when testing done
         QueryNodeCount qnc = new QueryNodeCount();
         rtn.visit(qnc);
-        
+
         if(qnc.getNodeCount()/2 < indexSet.size()) {
             return null;
         }
 
-        
+
         //move BindingSetAssignment Nodes out of the way
         organizeBSAs(rtn);
-        
-        
+
+
         // test to see if query contains no other nodes
         // than filter, join, projection, and statement pattern and
         // test whether query contains duplicate StatementPatterns and filters
@@ -105,26 +102,18 @@ public class GeneralizedExternalProcessor {
 
             }
             if(indexPlaced) {
-//                if(indexSet.size() == 3) {
-//                    System.out.println("IndexSet is " + indexSet);
-//                    System.out.println("Tuple is " + rtn);
-//                }
                 return rtn;
             } else {
-//                if(indexSet.size() == 3) {
-//                    System.out.println("IndexSet is " + indexSet);
-//                }
-//               
                 return null;
             }
-            
+
         } else {
             throw new IllegalArgumentException("Invalid Query.");
         }
     }
-    
-    
-    
+
+
+
 
 
     // determines whether query is valid, which requires that a
@@ -139,12 +128,12 @@ public class GeneralizedExternalProcessor {
 
         Set<String> spVars = getVarNames(getQNodes("sp", node));
 
-        if (vqv.isValid() && (spVars.size() > 0)) {
+        if (vqv.isValid() && spVars.size() > 0) {
 
             FilterCollector fvis = new FilterCollector();
             node.visit(fvis);
             List<QueryModelNode> fList = fvis.getFilters();
-            return (fList.size() == Sets.newHashSet(fList).size() && getVarNames(fList).size() <= spVars.size());
+            return fList.size() == Sets.newHashSet(fList).size() && getVarNames(fList).size() <= spVars.size();
 
         } else {
             return false;
@@ -211,16 +200,17 @@ public class GeneralizedExternalProcessor {
 
         }
 
-        public void meet(Projection node) {
+        @Override
+		public void meet(Projection node) {
             // moves external tuples above statement patterns before attempting
             // to bubble down index statement patterns found in query tree
-            
+
             organizeExtTuples(node);
-            
             super.meet(node);
         }
 
-        public void meet(Join node) {
+        @Override
+		public void meet(Join node) {
             // if right node contained in index, move it to bottom of query tree
             if (sSet.contains(node.getRightArg())) {
 
@@ -228,7 +218,6 @@ public class GeneralizedExternalProcessor {
                 Set<QueryModelNode> compSet = Sets.difference(eSet, sSet);
 
                 if (eSet.containsAll(sSet)) {
-
                     QNodeExchanger qne = new QNodeExchanger(node.getRightArg(), compSet);
                     node.visit(qne);
                     node.replaceChildNode(node.getRightArg(), qne.getReplaced());
@@ -293,7 +282,8 @@ public class GeneralizedExternalProcessor {
             return toBeReplaced;
         }
 
-        public void meet(Join node) {
+        @Override
+		public void meet(Join node) {
 
             if (compSet.contains(node.getRightArg())) {
                 this.toBeReplaced = node.getRightArg();
@@ -317,7 +307,6 @@ public class GeneralizedExternalProcessor {
     // this method is that
     // SPBubbleDownVisitor has been called to position index StatementPatterns
     // within query tree.
-    //TODO this visitor assumes that all filters are positioned at top of query tree
     //could lead to problems if filter optimizer called before external processor
     private static class FilterBubbleDownVisitor extends QueryModelVisitorBase<RuntimeException> {
 
@@ -335,7 +324,8 @@ public class GeneralizedExternalProcessor {
             return filterPlaced;
         }
 
-        public void meet(Join node) {
+        @Override
+		public void meet(Join node) {
 
             if (!compSet.contains(node.getRightArg())) {
                 // looks for placed to position filter node. if right node is
@@ -343,13 +333,13 @@ public class GeneralizedExternalProcessor {
                 // and left node is statement pattern node contained in index or
                 // is a join, place
                 // filter above join.
-                if (node.getLeftArg() instanceof Join || !(compSet.contains(node.getLeftArg()))) {
-                    
+                if (node.getLeftArg() instanceof Join || !compSet.contains(node.getLeftArg())) {
+
                     QueryModelNode pNode = node.getParentNode();
                     ((Filter) filter).setArg(node);
                     pNode.replaceChildNode(node, filter);
                     filterPlaced = true;
-                    
+
                     return;
                 } // otherwise place filter below join and above right arg
                 else {
@@ -359,12 +349,12 @@ public class GeneralizedExternalProcessor {
                     return;
 
                 }
-            } else if ((node.getLeftArg() instanceof StatementPattern) && !compSet.contains(node.getLeftArg())) {
-                
+            } else if (node.getLeftArg() instanceof StatementPattern && !compSet.contains(node.getLeftArg())) {
+
                 ((Filter) filter).setArg(node.getLeftArg());
                 node.replaceChildNode(node.getLeftArg(), filter);
                 filterPlaced = true;
-                
+
                 return;
             } else {
                 super.meet(node);
@@ -380,8 +370,9 @@ public class GeneralizedExternalProcessor {
 
         for (QueryModelNode s : nodes) {
             tempVars = VarCollector.process(s);
-            for (String t : tempVars)
-                nodeVarNames.add(t);
+            for (String t : tempVars) {
+				nodeVarNames.add(t);
+			}
         }
         return nodeVarNames;
 
@@ -403,7 +394,8 @@ public class GeneralizedExternalProcessor {
 
         }
 
-        public void meet(Filter node) {
+        @Override
+		public void meet(Filter node) {
 
             Set<QueryModelNode> eSet = getQNodes(node);
             Set<QueryModelNode> compSet = Sets.difference(eSet, sSet);
@@ -415,7 +407,7 @@ public class GeneralizedExternalProcessor {
             // and index (assuming that SPBubbleDownVisitor has already been
             // called)
             if (sSet.contains(node.getCondition()) && !bubbledFilters.contains(node.getCondition())) {
-                FilterBubbleDownVisitor fbdv = new FilterBubbleDownVisitor((Filter) node.clone(), compSet);
+                FilterBubbleDownVisitor fbdv = new FilterBubbleDownVisitor(node.clone(), compSet);
                 node.visit(fbdv);
                 bubbledFilters.add(node.getCondition());
                 // checks if filter correctly placed, and if it has been,
@@ -425,7 +417,7 @@ public class GeneralizedExternalProcessor {
                     QueryModelNode pNode = node.getParentNode();
                     TupleExpr cNode = node.getArg();
                     pNode.replaceChildNode(node, cNode);
-                   
+
 
                     super.meetNode(pNode);
                 }
@@ -446,37 +438,35 @@ public class GeneralizedExternalProcessor {
     // to position the StatementPatterns and Filters.
     private static class SubsetEqualsVisitor extends QueryModelVisitorBase<RuntimeException> {
 
-        private TupleExpr query;
         private TupleExpr tuple;
         private QueryModelNode indexQNode;
         private ExternalTupleSet set;
         private Set<QueryModelNode> sSet = Sets.newHashSet();
-        private TupleExpr temp;
         private boolean indexPlaced = false;
-        
+
 
         public SubsetEqualsVisitor(ExternalTupleSet index, TupleExpr query) {
-            this.query = query;
             this.tuple = index.getTupleExpr();
             this.set = index;
             indexQNode = ((Projection) tuple).getArg();
             sSet = getQNodes(indexQNode);
 
         }
-        
+
         public boolean indexPlaced() {
             return indexPlaced;
         }
-        
 
-        public void meet(Join node) {
+
+        @Override
+		public void meet(Join node) {
 
             Set<QueryModelNode> eSet = getQNodes(node);
 
             if (eSet.containsAll(sSet) && !(node.getRightArg() instanceof BindingSetAssignment)) {
 
 //                System.out.println("Eset is " + eSet + " and sSet is " + sSet);
-                
+
                 if (eSet.equals(sSet)) {
                     node.replaceWith(set);
                     indexPlaced = true;
@@ -506,9 +496,9 @@ public class GeneralizedExternalProcessor {
             }
 
         }
-      //TODO might need to include BindingSetAssignment Condition here
       //to account for index consisting of only filter and BindingSetAssignment nodes
-        public void meet(Filter node) {
+        @Override
+		public void meet(Filter node) {
 
             Set<QueryModelNode> eSet = getQNodes(node);
 
@@ -523,9 +513,10 @@ public class GeneralizedExternalProcessor {
                 }
             }
         }
-        
-        
-        public void meet(StatementPattern node) {
+
+
+        @Override
+		public void meet(StatementPattern node) {
             return;
         }
     }
@@ -541,24 +532,27 @@ public class GeneralizedExternalProcessor {
             return isValid;
         }
 
-        public void meet(Projection node) {
+        @Override
+		public void meet(Projection node) {
             node.getArg().visit(this);
         }
 
-        public void meet(Filter node) {
+        @Override
+		public void meet(Filter node) {
             node.getArg().visit(this);
         }
-        
-      
-       
-        
-       
-        public void meetNode(QueryModelNode node) {
 
-            if (!((node instanceof Join) || (node instanceof StatementPattern) || (node instanceof BindingSetAssignment) || (node instanceof Var))) {
+
+
+
+
+        @Override
+		public void meetNode(QueryModelNode node) {
+
+            if (!(node instanceof Join || node instanceof StatementPattern || node instanceof BindingSetAssignment || node instanceof Var)) {
                 isValid = false;
                 return;
-           
+
             } else{
                 super.meetNode(node);
             }
@@ -575,21 +569,22 @@ public class GeneralizedExternalProcessor {
             this.extTuples = extTuples;
         }
 
-        public void meet(Join queryNode) {
+        @Override
+		public void meet(Join queryNode) {
 
             // if query tree contains external tuples and they are not
             // positioned above statement pattern node
             // reposition
             if (this.extTuples.size() > 0 && !(queryNode.getRightArg() instanceof ExternalTupleSet)
                     && !(queryNode.getRightArg() instanceof BindingSetAssignment)) {
-                
+
                 if (queryNode.getLeftArg() instanceof ExternalTupleSet) {
                     QueryModelNode temp = queryNode.getLeftArg();
                     queryNode.setLeftArg(queryNode.getRightArg());
                     queryNode.setRightArg((TupleExpr)temp);
                 } else {
 
-                    QNodeExchanger qnev = new QNodeExchanger((QueryModelNode) queryNode.getRightArg(), this.extTuples);
+                    QNodeExchanger qnev = new QNodeExchanger(queryNode.getRightArg(), this.extTuples);
                     queryNode.visit(qnev);
                     queryNode.replaceChildNode(queryNode.getRightArg(), qnev.getReplaced());
                     super.meet(queryNode);
@@ -657,13 +652,14 @@ public class GeneralizedExternalProcessor {
             this.bsas = bsas;
         }
 
-        public void meet(Join queryNode) {
+        @Override
+		public void meet(Join queryNode) {
 
             // if query tree contains external tuples and they are not
             // positioned above statement pattern node
             // reposition
             if (this.bsas.size() > 0 && !(queryNode.getRightArg() instanceof BindingSetAssignment)) {
-                QNodeExchanger qnev = new QNodeExchanger((QueryModelNode) queryNode.getRightArg(), bsas);
+                QNodeExchanger qnev = new QNodeExchanger(queryNode.getRightArg(), bsas);
                 queryNode.visit(qnev);
                 queryNode.replaceChildNode(queryNode.getRightArg(), qnev.getReplaced());
                 super.meet(queryNode);
@@ -674,8 +670,8 @@ public class GeneralizedExternalProcessor {
         }
 
     }
-        
-        
+
+
     public static class BindingSetAssignmentCollector extends QueryModelVisitorBase<RuntimeException> {
 
         private Set<QueryModelNode> bindingSetList = Sets.newHashSet();
@@ -685,7 +681,7 @@ public class GeneralizedExternalProcessor {
         }
 
         public boolean containsBSAs() {
-            return (bindingSetList.size() > 0);
+            return bindingSetList.size() > 0;
         }
 
         @Override
@@ -695,9 +691,9 @@ public class GeneralizedExternalProcessor {
         }
 
     }
-    
-    
-    
+
+
+
     public static class QueryNodeCount extends QueryModelVisitorBase<RuntimeException> {
 
         private int nodeCount;
@@ -725,6 +721,6 @@ public class GeneralizedExternalProcessor {
 
     }
 
-   
-   
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexListPruner.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexListPruner.java b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexListPruner.java
index fa1dc13..8fbcbe0 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexListPruner.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexListPruner.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.IndexPlanValidator;
  * 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
@@ -22,14 +22,10 @@ package mvm.rya.indexing.IndexPlanValidator;
 
 import java.util.List;
 
-
-
-import java.util.Set;
-
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
 
 public interface IndexListPruner {
 
-    public Set<ExternalTupleSet> getRelevantIndices(List<ExternalTupleSet> indexList);
-        
+    public List<ExternalTupleSet> getRelevantIndices(List<ExternalTupleSet> indexList);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexedExecutionPlanGenerator.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexedExecutionPlanGenerator.java b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexedExecutionPlanGenerator.java
index acf3f6a..a0fca34 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexedExecutionPlanGenerator.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/IndexedExecutionPlanGenerator.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.IndexPlanValidator;
  * 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
@@ -22,9 +22,7 @@ package mvm.rya.indexing.IndexPlanValidator;
 
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.Set;
 
 import mvm.rya.indexing.external.QueryVariableNormalizer;
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
@@ -32,38 +30,29 @@ import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
 import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.TupleExpr;
 
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 public class IndexedExecutionPlanGenerator implements ExternalIndexMatcher {
 
     private final TupleExpr query;
-    private List<ExternalTupleSet> normalizedIndexList;
-    
+    private final List<ExternalTupleSet> normalizedIndexList;
+
     public IndexedExecutionPlanGenerator(TupleExpr query, List<ExternalTupleSet> indexList) {
         this.query = query;
-        VarConstantIndexListPruner vci = new VarConstantIndexListPruner(query);
+        final VarConstantIndexListPruner vci = new VarConstantIndexListPruner(query);
         normalizedIndexList = getNormalizedIndices(vci.getRelevantIndices(indexList));
     }
-    
+
     public List<ExternalTupleSet> getNormalizedIndices() {
         return normalizedIndexList;
     }
-    
-  
-    
-    
+
     @Override
     public Iterator<TupleExpr> getIndexedTuples() {
-        
-        ValidIndexCombinationGenerator vic = new ValidIndexCombinationGenerator(query);
-        final Iterator<List<ExternalTupleSet>> iter = vic.getValidIndexCombos(normalizedIndexList);
 
+        final ValidIndexCombinationGenerator vic = new ValidIndexCombinationGenerator(query);
+        final Iterator<List<ExternalTupleSet>> iter = vic.getValidIndexCombos(normalizedIndexList);
         return new Iterator<TupleExpr>() {
-
             private TupleExpr next = null;
             private boolean hasNextCalled = false;
             private boolean isEmpty = false;
@@ -73,7 +62,7 @@ public class IndexedExecutionPlanGenerator implements ExternalIndexMatcher {
 
                 if (!hasNextCalled && !isEmpty) {
                     while (iter.hasNext()) {
-                        TupleExpr temp = GeneralizedExternalProcessor.process(query, iter.next());
+                        final TupleExpr temp = GeneralizedExternalProcessor.process(query, iter.next());
                         if (temp != null) {
                             next = temp;
                             hasNextCalled = true;
@@ -104,104 +93,36 @@ public class IndexedExecutionPlanGenerator implements ExternalIndexMatcher {
                     } else {
                         throw new NoSuchElementException();
                     }
-
                 }
-
             }
 
             @Override
             public void remove() {
-
                 throw new UnsupportedOperationException("Cannot delete from iterator!");
-
             }
-
         };
     }
 
-    
-    private List<ExternalTupleSet> getNormalizedIndices(Set<ExternalTupleSet> indexSet) {
+    private List<ExternalTupleSet> getNormalizedIndices(List<ExternalTupleSet> indexSet) {
 
         ExternalTupleSet tempIndex;
-        List<ExternalTupleSet> normalizedIndexSet = Lists.newArrayList();
-
-        for (ExternalTupleSet e : indexSet) {
+        final List<ExternalTupleSet> normalizedIndexSet = Lists.newArrayList();
 
+        for (final ExternalTupleSet e : indexSet) {
             List<TupleExpr> tupList = null;
             try {
                 tupList = QueryVariableNormalizer.getNormalizedIndex(query, e.getTupleExpr());
-            } catch (Exception e1) {
+            } catch (final Exception e1) {
                 // TODO Auto-generated catch block
                 e1.printStackTrace();
             }
 
-            for (TupleExpr te : tupList) {
-
+            for (final TupleExpr te : tupList) {
                 tempIndex = (ExternalTupleSet) e.clone();
-                setTableMap(te, tempIndex);
-                setSupportedVarOrderMap(tempIndex);
                 tempIndex.setProjectionExpr((Projection) te);
                 normalizedIndexSet.add(tempIndex);
-
             }
-
         }
-
         return normalizedIndexSet;
     }
-
-    private void setTableMap(TupleExpr tupleMatch, ExternalTupleSet index) {
-
-        List<String> replacementVars = Lists.newArrayList(tupleMatch.getBindingNames());
-        List<String> tableVars = Lists.newArrayList(index.getTupleExpr().getBindingNames());
-
-        Map<String, String> tableMap = Maps.newHashMap();
-
-        for (int i = 0; i < tableVars.size(); i++) {
-            tableMap.put(replacementVars.get(i), tableVars.get(i));
-        }
-        // System.out.println("Table map is " + tableMap);
-        index.setTableVarMap(tableMap);
-
-    }
-    
-    
-    private void setSupportedVarOrderMap(ExternalTupleSet index) {
-
-        Map<String, Set<String>> supportedVarOrders = Maps.newHashMap();
-        BiMap<String, String> biMap = HashBiMap.create(index.getTableVarMap()).inverse();
-        Map<String, Set<String>> oldSupportedVarOrders = index.getSupportedVariableOrderMap();
-
-        Set<String> temp = null;
-        Set<String> keys = oldSupportedVarOrders.keySet();
-
-        for (String s : keys) {
-            temp = oldSupportedVarOrders.get(s);
-            Set<String> newSet = Sets.newHashSet();
-
-            for (String t : temp) {
-                newSet.add(biMap.get(t));
-            }
-            
-            String[] tempStrings = s.split("\u0000");
-            String v = "";
-            for(String u: tempStrings) {
-                if(v.length() == 0){
-                    v = v + biMap.get(u);
-                } else {
-                    v = v + "\u0000" + biMap.get(u);
-                }
-            }
-
-            supportedVarOrders.put(v, newSet);
-
-        }
-
-        index.setSupportedVariableOrderMap(supportedVarOrders);
-
-    }
-    
-    
-    
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/c12f58f4/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/ValidIndexCombinationGenerator.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/ValidIndexCombinationGenerator.java b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/ValidIndexCombinationGenerator.java
index b3c3fcd..483457f 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/ValidIndexCombinationGenerator.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/IndexPlanValidator/ValidIndexCombinationGenerator.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.IndexPlanValidator;
  * 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,7 +19,6 @@ package mvm.rya.indexing.IndexPlanValidator;
  * under the License.
  */
 
-
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -27,645 +26,436 @@ import java.util.NoSuchElementException;
 import java.util.Set;
 
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
-import mvm.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
 
-import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.algebra.Filter;
-import org.openrdf.query.algebra.Projection;
 import org.openrdf.query.algebra.QueryModelNode;
 import org.openrdf.query.algebra.StatementPattern;
 import org.openrdf.query.algebra.TupleExpr;
 import org.openrdf.query.algebra.helpers.QueryModelVisitorBase;
-import org.openrdf.query.parser.ParsedQuery;
-import org.openrdf.query.parser.sparql.SPARQLParser;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public class ValidIndexCombinationGenerator {
-    
-    
-    private TupleExpr query;
-    private Set<String> invalidCombos = Sets.newTreeSet();
-    private Set<QueryModelNode> spFilterSet;
-    
-    
-    public ValidIndexCombinationGenerator(TupleExpr query) {
-        this.query = query;
-        SpFilterCollector sfc = new SpFilterCollector();
-        query.visit(sfc);
-        spFilterSet = sfc.getSpFilterSet();
-    }
-    
-    
-    
-    
-    public Iterator<List<ExternalTupleSet>> getValidIndexCombos(List<ExternalTupleSet> indexSet) {
-
-        Collections.shuffle(indexSet);
-        final List<ExternalTupleSet> list = indexSet;
-        final Iterator<List<Integer>> iter = getValidCombos(list);
-
-        return new Iterator<List<ExternalTupleSet>>() {
-
-            private List<ExternalTupleSet> next = null;
-            private List<Integer> nextCombo = null;
-            private boolean hasNextCalled = false;
-            private boolean isEmpty = false;
-
-            @Override
-            public boolean hasNext() {
-
-                if (!hasNextCalled && !isEmpty) {
-                    if (!iter.hasNext()) {
-                        isEmpty = true;
-                        return false;
-                    } else {
-                        nextCombo = iter.next();
-                        List<ExternalTupleSet> indexCombo = Lists.newArrayList();
-                        for (Integer i : nextCombo) {
-                            indexCombo.add(list.get(i));
-                        }
-                        next = indexCombo;
-                        hasNextCalled = true;
-                        return true;
-
-                    }
-
-                } else if (isEmpty) {
-                    return false;
-                } else {
-                    return true;
-                }
-            }
-
-            @Override
-            public List<ExternalTupleSet> next() {
-
-                if (hasNextCalled) {
-                    hasNextCalled = false;
-                    return next;
-                } else if(isEmpty) {
-                    throw new NoSuchElementException();
-                }else {
-                    if (this.hasNext()) {
-                        hasNextCalled = false;
-                        return next;
-                    } else {
-                        throw new NoSuchElementException();
-                    }
-                }
-            }
-
-            @Override
-            public void remove() {
-
-                throw new UnsupportedOperationException("Cannot delete from iterator!");
-
-            }
-
-        };
-
-    }
-    
-    
-    
-    private Iterator<List<Integer>> getValidCombos(List<ExternalTupleSet> indexList) {
-        
-        
-        final List<ExternalTupleSet> list = indexList;
-        final int indexSize = list.size();
-        final Iterator<List<Integer>> iter = getCombos(indexSize);
-        
-        
-        return new Iterator<List<Integer>>() {
-
-            private List<Integer> next = null;
-            private boolean hasNextCalled = false;
-            private boolean isEmpty = false;
-
-            @Override
-            public boolean hasNext() {
-                if (!hasNextCalled && !isEmpty) {
-
-                    while (iter.hasNext()) {
-                        List<Integer> tempNext = iter.next();
-                        if (isValid(tempNext, list)) {
-                            next = tempNext;
-                            hasNextCalled = true;
-                            return true;
-                        }
-
-                    }
-
-                    isEmpty = true;
-                    return false;
-
-                } else if (isEmpty) {
-                    return false;
-                } else {
-                    return true;
-                }
-            }
-
-            @Override
-            public List<Integer> next() {
-
-                if (hasNextCalled) {
-                    hasNextCalled = false;
-                    return next;
-                } else if (isEmpty) {
-                    throw new NoSuchElementException();
-                } else {
-                    if (this.hasNext()) {
-                        hasNextCalled = false;
-                        return next;
-                    } else {
-                        throw new NoSuchElementException();
-                    }
-
-                }
-
-            }
-
-            @Override
-            public void remove() {
-
-                throw new UnsupportedOperationException("Cannot delete from iterator!");
-
-            }
-
-        };
-    }
-    
-    
-    
-    
-    
-    
-    private Iterator<List<Integer>> getCombos(int indexListSize) {
-
-        final int indexSize = indexListSize;
-        final int maxSubListSize = spFilterSet.size() / 2;
-
-        return new Iterator<List<Integer>>() {
-
-            private List<Integer> next = null;
-            private boolean hasNextCalled = false;
-            private boolean isEmpty = false;
-            private int subListSize = Math.min(maxSubListSize, indexSize) + 1;
-            Iterator<List<Integer>> subList = null;
-
-            @Override
-            public boolean hasNext() {
-
-                if (!hasNextCalled && !isEmpty) {
-                    if (subList != null && subList.hasNext()) {
-                        next = subList.next();
-                        hasNextCalled = true;
-                        return true;
-                    } else {
-                        subListSize--;
-                        if (subListSize == 0) {
-                            isEmpty = true;
-                            return false;
-                        }
-                        subList = getCombos(subListSize, indexSize);
-                        if (subList == null) {
-                            throw new IllegalStateException("Combos cannot be null!");
-                        }
-                        next = subList.next();
-                        hasNextCalled = true;
-                        return true;
-
-                    }
-                } else if (isEmpty) {
-                    return false;
-                } else {
-                    return true;
-                }
-            }
-
-            @Override
-            public List<Integer> next() {
-
-                if (hasNextCalled) {
-                    hasNextCalled = false;
-                    return next;
-                } else if (isEmpty) {
-                    throw new NoSuchElementException();
-                } else {
-                    if (this.hasNext()) {
-                        hasNextCalled = false;
-                        return next;
-                    } else {
-                        throw new NoSuchElementException();
-                    }
-
-                }
-
-            }
-
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException("Cannot delete from iterator!");
-            }
-
-        };
-
-    }
-    
-    
-    
-    private Iterator<List<Integer>> getCombos(int subListSize, int indexListSize) {
-        
-        if(subListSize > indexListSize) {
-            throw new IllegalArgumentException("Sublist size must be less than or equal to list size!");
-        }
-        
-        final int subSize = subListSize;
-        final int indexSize = indexListSize;
-        
-        return new Iterator<List<Integer>>() {
-
-            private List<Integer> next = null;
-            private List<Integer> tempList = Lists.newArrayList();
-            private boolean calledHasNext = false;
-            private boolean isEmpty = false;
-            
-            @Override
-            public boolean hasNext() {
-
-                if (!calledHasNext && !isEmpty) {
-                    if (next == null) {
-                        for (int i = 0; i < subSize; i++) {
-                            tempList.add(i);
-                        }
-                        next = tempList;
-                        calledHasNext = true;
-                        return true;
-                    } else {
-                        next = getNext(next, indexSize - 1);
-                        if (next == null) {
-                            isEmpty = true;
-                            return false;
-                        } else {
-                            calledHasNext = true;
-                            return true;
-                        }
-
-                    }
-                } else if(isEmpty) {  
-                    return false;
-                } else {
-                    return true;
-                }
-
-            }
-
-            @Override
-            public List<Integer> next() {
-
-                if (calledHasNext) {
-                    calledHasNext = false;
-                    return next;
-                } else if (isEmpty) {
-                    throw new NoSuchElementException();
-                } else {
-                    if (this.hasNext()) {
-                        calledHasNext = false;
-                        return next;
-                    } else {
-                        throw new NoSuchElementException();
-                    }
-                }
-            }
-            @Override
-            public void remove() {
-                throw new UnsupportedOperationException();
-                
-            }
-            
-            
-            
-        };
-    }
-    
-
-    
-    
-    
-    
-    private List<Integer> getNext(List<Integer> prev, int maxInt) {
-        
-        List<Integer> returnList = Lists.newArrayList();
-        int size = prev.size();
-        int incrementPos = -1;
-        int incrementVal = 0;
-        
-        for(int i = 0; i < size; i++) {
-            if(prev.get(size-(i+1)) != maxInt - i) {
-                incrementPos = size - (i+1);
-                break;
-            }
-        }
-        
-        if (incrementPos == -1) {
-            return null;
-        } else {
-
-            incrementVal = prev.get(incrementPos);
-            for (int i = 0; i < incrementPos; i++) {
-                returnList.add(prev.get(i));
-            }
-
-            for (int j = incrementPos; j < size; j++) {
-                returnList.add(++incrementVal);
-            }
-
-            return returnList;
-        }
-    }
-    
-    
-    
-    
-    private boolean isValid(List<Integer> combo, List<ExternalTupleSet> indexList) {
-        
-        String s1 = Joiner.on("\u0000").join(combo).trim();
-        
-        if(invalidCombos.contains(s1)) {
-            return false;
-        } else {
-            int valid = indicesDisjoint(combo, indexList);
-            
-            if (valid >= 0) {
-                String s2 = "";
-                for (int i = 0; i < valid + 1; i++) {
-                    if (s2.length() == 0) {
-                        s2 = s2 + combo.get(i);
-                    } else {
-                        s2 = s2 + "\u0000" + combo.get(i);
-                    }
-                }
-                invalidCombos.add(s2);
-
-                for (int i = valid + 1; i < combo.size(); i++) {
-                    s2 = s2 + "\u0000" + combo.get(i);
-                    invalidCombos.add(s2);
-                }
-
-                return false;
-            } else {
-                return true;
-            }
-        }
-        
-        
-    }
-    
-    
-    
-    private int indicesDisjoint(List<Integer> combo, List<ExternalTupleSet> indexList) {
-        
-        Set<QueryModelNode> indexNodes = Sets.newHashSet();
-        Set<QueryModelNode> tempNodes;
-        TupleExpr temp;
-        
-        
-        int j = 0;
-        for(Integer i: combo) {
-            temp = indexList.get(i).getTupleExpr();
-            SpFilterCollector spf = new SpFilterCollector();
-            temp.visit(spf);
-            tempNodes = spf.getSpFilterSet();
-            if(Sets.intersection(indexNodes, tempNodes).size() == 0) {
-                indexNodes = Sets.union(indexNodes, tempNodes);
-                if(indexNodes.size() > spFilterSet.size()) {
-                    return j;
-                }
-            } else {
-                return j;
-            }
-            j++;
-        }
-        
-        return -1;
-    }
-    
-    
-    
-    
-    public static void main(String[] args) {
-        
-        
-        String q1 = ""//
-                + "SELECT ?f ?m ?d " //
-                + "{" //
-                + "  ?f a ?m ."//
-                + "  ?m <http://www.w3.org/2000/01/rdf-schema#label> ?d ."//
-                + "  ?d <uri:talksTo> ?f . "//
-                + "  ?f <uri:hangOutWith> ?m ." //
-                + "  ?m <uri:hangOutWith> ?d ." //
-                + "  ?f <uri:associatesWith> ?m ." //
-                + "  ?m <uri:associatesWith> ?d ." //
-                + "}";//
-        
-        
-        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 ." //
-                + "}";//
-        
-        
-        String q5 = ""//
-                + "SELECT ?t ?s ?u " //
-                + "{" //
-                + "  ?s a ?t ."//
-                + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
-                + "  ?u <uri:talksTo> ?s . "//
-                + "  ?s <uri:hangOutWith> ?t ." //
-                + "  ?t <uri:hangOutWith> ?u ." //
-                + "}";//
-        
-        String q6 = ""//
-                + "SELECT ?s ?t ?u " //
-                + "{" //
-                + "  ?s <uri:associatesWith> ?t ." //
-                + "  ?t <uri:associatesWith> ?u ." //
-                + "  ?s <uri:hangOutWith> ?t ." //
-                + "  ?t <uri:hangOutWith> ?u ." //
-                + "}";//
-        
-        
-        String q7 = ""//
-                + "SELECT ?s ?t ?u " //
-                + "{" //
-                + "  ?s <uri:associatesWith> ?t ." //
-                + "  ?t <uri:associatesWith> ?u ." //
-                + "  ?t <uri:hangOutWith> ?u ." //
-                + "}";//
-        
-        
-        
-        String q8 = ""//
-                + "SELECT ?t ?s ?u " //
-                + "{" //
-                + "  ?s a ?t ."//
-                + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
-                + "  ?u <uri:talksTo> ?s . "//
-                + "  ?s <uri:associatesWith> ?t ." //
-                + "}";//
-        
-        
-        String q9 = ""//
-                + "SELECT ?t ?s ?u " //
-                + "{" //
-                + "  ?s a ?t ."//
-                + "  ?t <http://www.w3.org/2000/01/rdf-schema#label> ?u ."//
-                + "}";//
-        
-        
-        
-        
-        
-        
-        
-        
-
-        SPARQLParser parser = new SPARQLParser();
-        ParsedQuery pq1 = null;
-        ParsedQuery pq2 = null;
-        ParsedQuery pq3 = null;
-        ParsedQuery pq4 = null;
-        ParsedQuery pq5 = null;
-        ParsedQuery pq6 = null;
-        ParsedQuery pq7 = null;
-        ParsedQuery pq8 = null;
-        ParsedQuery pq9 = null;
-        
-        SimpleExternalTupleSet extTup1 = null;
-        SimpleExternalTupleSet extTup2 = null;
-        SimpleExternalTupleSet extTup3 = null;
-        SimpleExternalTupleSet extTup4 = null;
-        SimpleExternalTupleSet extTup5 = null;
-        SimpleExternalTupleSet extTup6 = null;
-        SimpleExternalTupleSet extTup7 = null;
-        SimpleExternalTupleSet extTup8 = null;
-        
-        
-        
-        
-        
-        try {
-            pq1 = parser.parseQuery(q1, null);
-            pq2 = parser.parseQuery(q2, null);
-            pq3 = parser.parseQuery(q3, null);
-            pq4 = parser.parseQuery(q4, null);
-            pq5 = parser.parseQuery(q5, null);
-            pq6 = parser.parseQuery(q6, null);
-            pq7 = parser.parseQuery(q7, null);
-            pq8 = parser.parseQuery(q8, null);
-            pq9 = parser.parseQuery(q9, null);
-           
-
-            extTup1 = new SimpleExternalTupleSet((Projection) pq2.getTupleExpr());
-            extTup2 = new SimpleExternalTupleSet((Projection) pq3.getTupleExpr());
-            extTup3 = new SimpleExternalTupleSet((Projection) pq4.getTupleExpr());
-            extTup4 = new SimpleExternalTupleSet((Projection) pq5.getTupleExpr());
-            extTup5 = new SimpleExternalTupleSet((Projection) pq6.getTupleExpr());
-            extTup6 = new SimpleExternalTupleSet((Projection) pq7.getTupleExpr());
-            extTup7 = new SimpleExternalTupleSet((Projection) pq8.getTupleExpr());
-            extTup8 = new SimpleExternalTupleSet((Projection) pq9.getTupleExpr());
-            
-          
-        } catch (MalformedQueryException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-        }
-        
-        List<ExternalTupleSet> indexList = Lists.newArrayList();
-        indexList.add(extTup1);
-        indexList.add(extTup2);
-        indexList.add(extTup3);
-        indexList.add(extTup4);
-        indexList.add(extTup5);
-        indexList.add(extTup6);
-        indexList.add(extTup7);
-        indexList.add(extTup8);
-        
-        
-        ValidIndexCombinationGenerator vic = new ValidIndexCombinationGenerator(pq1.getTupleExpr());
-        Iterator<List<ExternalTupleSet>> combos = vic.getValidIndexCombos(indexList);
-        int size = 0;
-        while(combos.hasNext()) {
-            combos.hasNext();
-            size++;
-            List<ExternalTupleSet> eSet = combos.next();
-            System.out.println("********************************************");
-            for(ExternalTupleSet e: eSet) {
-                System.out.println(e.getTupleExpr());
-            }
-            System.out.println("********************************************");
-        }
-        
-        System.out.println("size is " + size + " has next " + combos.hasNext());
-    }
-    
-    
-    
-    
-    
-    private static class SpFilterCollector extends QueryModelVisitorBase<RuntimeException> {
-
-        private Set<QueryModelNode> spFilterSet = Sets.newHashSet();
-
-        
-        public int getNodeNumber() {
-            return spFilterSet.size();
-        }
-        
-        
-        public Set<QueryModelNode> getSpFilterSet() {
-            return spFilterSet;
-        }
-        
-        
-        @Override
-        public void meet(StatementPattern node) {
-            
-            spFilterSet.add(node);
-            return;
-            
-        }
-        
-        
-        @Override
-        public void meet(Filter node) {
-            
-            spFilterSet.add(node.getCondition());
-            node.getArg().visit(this);
-        }  
-        
-
-    }
+
+	private TupleExpr query;
+	private Set<String> invalidCombos = Sets.newTreeSet();
+	private Set<QueryModelNode> spFilterSet;
+
+	public ValidIndexCombinationGenerator(TupleExpr query) {
+		this.query = query;
+		SpFilterCollector sfc = new SpFilterCollector();
+		query.visit(sfc);
+		spFilterSet = sfc.getSpFilterSet();
+	}
+
+	public Iterator<List<ExternalTupleSet>> getValidIndexCombos(
+			List<ExternalTupleSet> indexSet) {
+
+		Collections.shuffle(indexSet);
+		final List<ExternalTupleSet> list = indexSet;
+		final Iterator<List<Integer>> iter = getValidCombos(list);
+
+		return new Iterator<List<ExternalTupleSet>>() {
+
+			private List<ExternalTupleSet> next = null;
+			private List<Integer> nextCombo = null;
+			private boolean hasNextCalled = false;
+			private boolean isEmpty = false;
+
+			@Override
+			public boolean hasNext() {
+
+				if (!hasNextCalled && !isEmpty) {
+					if (!iter.hasNext()) {
+						isEmpty = true;
+						return false;
+					} else {
+						nextCombo = iter.next();
+						List<ExternalTupleSet> indexCombo = Lists
+								.newArrayList();
+						for (Integer i : nextCombo) {
+							indexCombo.add(list.get(i));
+						}
+						next = indexCombo;
+						hasNextCalled = true;
+						return true;
+
+					}
+
+				} else if (isEmpty) {
+					return false;
+				} else {
+					return true;
+				}
+			}
+
+			@Override
+			public List<ExternalTupleSet> next() {
+
+				if (hasNextCalled) {
+					hasNextCalled = false;
+					return next;
+				} else if (isEmpty) {
+					throw new NoSuchElementException();
+				} else {
+					if (this.hasNext()) {
+						hasNextCalled = false;
+						return next;
+					} else {
+						throw new NoSuchElementException();
+					}
+				}
+			}
+
+			@Override
+			public void remove() {
+
+				throw new UnsupportedOperationException(
+						"Cannot delete from iterator!");
+
+			}
+
+		};
+
+	}
+
+	private Iterator<List<Integer>> getValidCombos(
+			List<ExternalTupleSet> indexList) {
+
+		final List<ExternalTupleSet> list = indexList;
+		final int indexSize = list.size();
+		final Iterator<List<Integer>> iter = getCombos(indexSize);
+
+		return new Iterator<List<Integer>>() {
+
+			private List<Integer> next = null;
+			private boolean hasNextCalled = false;
+			private boolean isEmpty = false;
+
+			@Override
+			public boolean hasNext() {
+				if (!hasNextCalled && !isEmpty) {
+
+					while (iter.hasNext()) {
+						List<Integer> tempNext = iter.next();
+						if (isValid(tempNext, list)) {
+							next = tempNext;
+							hasNextCalled = true;
+							return true;
+						}
+
+					}
+
+					isEmpty = true;
+					return false;
+
+				} else if (isEmpty) {
+					return false;
+				} else {
+					return true;
+				}
+			}
+
+			@Override
+			public List<Integer> next() {
+
+				if (hasNextCalled) {
+					hasNextCalled = false;
+					return next;
+				} else if (isEmpty) {
+					throw new NoSuchElementException();
+				} else {
+					if (this.hasNext()) {
+						hasNextCalled = false;
+						return next;
+					} else {
+						throw new NoSuchElementException();
+					}
+
+				}
+
+			}
+
+			@Override
+			public void remove() {
+
+				throw new UnsupportedOperationException(
+						"Cannot delete from iterator!");
+
+			}
+
+		};
+	}
+
+	private Iterator<List<Integer>> getCombos(int indexListSize) {
+
+		final int indexSize = indexListSize;
+		final int maxSubListSize = spFilterSet.size() / 2;
+
+		return new Iterator<List<Integer>>() {
+
+			private List<Integer> next = null;
+			private boolean hasNextCalled = false;
+			private boolean isEmpty = false;
+			private int subListSize = Math.min(maxSubListSize, indexSize) + 1;
+			Iterator<List<Integer>> subList = null;
+
+			@Override
+			public boolean hasNext() {
+
+				if (!hasNextCalled && !isEmpty) {
+					if (subList != null && subList.hasNext()) {
+						next = subList.next();
+						hasNextCalled = true;
+						return true;
+					} else {
+						subListSize--;
+						if (subListSize == 0) {
+							isEmpty = true;
+							return false;
+						}
+						subList = getCombos(subListSize, indexSize);
+						if (subList == null) {
+							throw new IllegalStateException(
+									"Combos cannot be null!");
+						}
+						next = subList.next();
+						hasNextCalled = true;
+						return true;
+
+					}
+				} else if (isEmpty) {
+					return false;
+				} else {
+					return true;
+				}
+			}
+
+			@Override
+			public List<Integer> next() {
+
+				if (hasNextCalled) {
+					hasNextCalled = false;
+					return next;
+				} else if (isEmpty) {
+					throw new NoSuchElementException();
+				} else {
+					if (this.hasNext()) {
+						hasNextCalled = false;
+						return next;
+					} else {
+						throw new NoSuchElementException();
+					}
+
+				}
+
+			}
+
+			@Override
+			public void remove() {
+				throw new UnsupportedOperationException(
+						"Cannot delete from iterator!");
+			}
+
+		};
+
+	}
+
+	private Iterator<List<Integer>> getCombos(int subListSize, int indexListSize) {
+
+		if (subListSize > indexListSize) {
+			throw new IllegalArgumentException(
+					"Sublist size must be less than or equal to list size!");
+		}
+
+		final int subSize = subListSize;
+		final int indexSize = indexListSize;
+
+		return new Iterator<List<Integer>>() {
+
+			private List<Integer> next = null;
+			private List<Integer> tempList = Lists.newArrayList();
+			private boolean calledHasNext = false;
+			private boolean isEmpty = false;
+
+			@Override
+			public boolean hasNext() {
+
+				if (!calledHasNext && !isEmpty) {
+					if (next == null) {
+						for (int i = 0; i < subSize; i++) {
+							tempList.add(i);
+						}
+						next = tempList;
+						calledHasNext = true;
+						return true;
+					} else {
+						next = getNext(next, indexSize - 1);
+						if (next == null) {
+							isEmpty = true;
+							return false;
+						} else {
+							calledHasNext = true;
+							return true;
+						}
+
+					}
+				} else if (isEmpty) {
+					return false;
+				} else {
+					return true;
+				}
+
+			}
+
+			@Override
+			public List<Integer> next() {
+
+				if (calledHasNext) {
+					calledHasNext = false;
+					return next;
+				} else if (isEmpty) {
+					throw new NoSuchElementException();
+				} else {
+					if (this.hasNext()) {
+						calledHasNext = false;
+						return next;
+					} else {
+						throw new NoSuchElementException();
+					}
+				}
+			}
+
+			@Override
+			public void remove() {
+				throw new UnsupportedOperationException();
+
+			}
+
+		};
+	}
+
+	private List<Integer> getNext(List<Integer> prev, int maxInt) {
+
+		List<Integer> returnList = Lists.newArrayList();
+		int size = prev.size();
+		int incrementPos = -1;
+		int incrementVal = 0;
+
+		for (int i = 0; i < size; i++) {
+			if (prev.get(size - (i + 1)) != maxInt - i) {
+				incrementPos = size - (i + 1);
+				break;
+			}
+		}
+
+		if (incrementPos == -1) {
+			return null;
+		} else {
+
+			incrementVal = prev.get(incrementPos);
+			for (int i = 0; i < incrementPos; i++) {
+				returnList.add(prev.get(i));
+			}
+
+			for (int j = incrementPos; j < size; j++) {
+				returnList.add(++incrementVal);
+			}
+
+			return returnList;
+		}
+	}
+
+	private boolean isValid(List<Integer> combo,
+			List<ExternalTupleSet> indexList) {
+
+		String s1 = Joiner.on("\u0000").join(combo).trim();
+
+		if (invalidCombos.contains(s1)) {
+			return false;
+		} else {
+			int valid = indicesDisjoint(combo, indexList);
+
+			if (valid >= 0) {
+				String s2 = "";
+				for (int i = 0; i < valid + 1; i++) {
+					if (s2.length() == 0) {
+						s2 = s2 + combo.get(i);
+					} else {
+						s2 = s2 + "\u0000" + combo.get(i);
+					}
+				}
+				invalidCombos.add(s2);
+
+				for (int i = valid + 1; i < combo.size(); i++) {
+					s2 = s2 + "\u0000" + combo.get(i);
+					invalidCombos.add(s2);
+				}
+
+				return false;
+			} else {
+				return true;
+			}
+		}
+
+	}
+
+	private int indicesDisjoint(List<Integer> combo,
+			List<ExternalTupleSet> indexList) {
+
+		Set<QueryModelNode> indexNodes = Sets.newHashSet();
+		Set<QueryModelNode> tempNodes;
+		TupleExpr temp;
+
+		int j = 0;
+		for (Integer i : combo) {
+			temp = indexList.get(i).getTupleExpr();
+			SpFilterCollector spf = new SpFilterCollector();
+			temp.visit(spf);
+			tempNodes = spf.getSpFilterSet();
+			if (Sets.intersection(indexNodes, tempNodes).size() == 0) {
+				indexNodes = Sets.union(indexNodes, tempNodes);
+				if (indexNodes.size() > spFilterSet.size()) {
+					return j;
+				}
+			} else {
+				return j;
+			}
+			j++;
+		}
+
+		return -1;
+	}
+
+	private static class SpFilterCollector extends
+			QueryModelVisitorBase<RuntimeException> {
+
+		private Set<QueryModelNode> spFilterSet = Sets.newHashSet();
+
+		public int getNodeNumber() {
+			return spFilterSet.size();
+		}
+
+		public Set<QueryModelNode> getSpFilterSet() {
+			return spFilterSet;
+		}
+
+		@Override
+		public void meet(StatementPattern node) {
+
+			spFilterSet.add(node);
+			return;
+
+		}
+
+		@Override
+		public void meet(Filter node) {
+
+			spFilterSet.add(node.getCondition());
+			node.getArg().visit(this);
+		}
+
+	}
 }