You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rya.apache.org by pu...@apache.org on 2016/04/07 14:40:37 UTC

[3/4] incubator-rya git commit: RYA-50 Updated Rya PCJs to support SPARQL queries with OPTIONALs.

RYA-50 Updated Rya PCJs to support SPARQL queries with OPTIONALs.


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

Branch: refs/heads/develop
Commit: a24515b0d1c7c3e459d68a867160850aa7d451c1
Parents: c378f64
Author: Kevin Chilton <ke...@localhost.localdomain>
Authored: Tue Mar 22 14:49:54 2016 -0400
Committer: Kevin Chilton <ke...@localhost.localdomain>
Committed: Mon Apr 4 18:40:09 2016 -0400

----------------------------------------------------------------------
 .../accumulo/precompQuery/AccumuloPcjQuery.java |  40 +-
 .../indexing/external/PrecompJoinOptimizer.java |  20 +
 .../external/tupleSet/AccumuloIndexSet.java     |   4 +-
 .../tupleSet/AccumuloPcjSerializer.java         | 199 ++++++----
 .../external/tupleSet/BindingSetConverter.java  | 108 +++++
 .../tupleSet/BindingSetStringConverter.java     | 111 +++---
 .../indexing/external/tupleSet/PcjTables.java   |  26 +-
 .../indexing/external/PCJOptionalTestIT.java    | 324 +++++++++++++++
 .../external/PcjIntegrationTestingUtil.java     |   9 +-
 .../external/PrecompJoinOptimizerTest.java      |   2 +
 .../tupleSet/AccumuloPcjSerialzerTest.java      | 154 +++++---
 .../tupleSet/BindingSetStringConverterTest.java | 151 +++++--
 .../tupleSet/PcjTablesIntegrationTests.java     |  13 +-
 .../rya/indexing/pcj/fluo/api/CreatePcj.java    |  15 +-
 .../indexing/pcj/fluo/app/BindingSetRow.java    |  23 +-
 .../pcj/fluo/app/FilterResultUpdater.java       |  10 +-
 .../pcj/fluo/app/FluoStringConverter.java       | 152 -------
 .../pcj/fluo/app/JoinResultUpdater.java         | 395 +++++++++++++------
 .../pcj/fluo/app/QueryResultUpdater.java        |  10 +-
 .../fluo/app/observers/BindingSetUpdater.java   |   7 +-
 .../pcj/fluo/app/observers/FilterObserver.java  |  10 +-
 .../pcj/fluo/app/observers/JoinObserver.java    |  10 +-
 .../fluo/app/observers/QueryResultObserver.java |   9 +-
 .../app/observers/StatementPatternObserver.java |  10 +-
 .../pcj/fluo/app/query/FluoQueryColumns.java    |   2 +
 .../fluo/app/query/FluoQueryMetadataDAO.java    |   7 +
 .../pcj/fluo/app/query/JoinMetadata.java        |  35 ++
 .../fluo/app/query/SparqlFluoQueryBuilder.java  |  64 +--
 .../pcj/fluo/app/FluoStringConverterTest.java   |  74 ----
 .../pcj/fluo/app/LeftOuterJoinTest.java         | 174 ++++++++
 .../indexing/pcj/fluo/app/NaturalJoinTest.java  | 166 ++++++++
 .../pcj/fluo/demo/FluoAndHistoricPcjsDemo.java  |   7 +-
 .../apache/rya/indexing/pcj/fluo/ITBase.java    |   9 +-
 .../fluo/app/query/FluoQueryMetadataDAOIT.java  |   2 +
 .../indexing/pcj/fluo/integration/QueryIT.java  |  44 +++
 .../pcj/fluo/integration/RyaExportIT.java       |   7 +-
 36 files changed, 1730 insertions(+), 673 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/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
index 08c0f78..dd1e9c9 100644
--- a/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java
+++ b/extras/indexing/src/main/java/mvm/rya/accumulo/precompQuery/AccumuloPcjQuery.java
@@ -37,7 +37,9 @@ 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.BindingSetConverter.BindingSetConversionException;
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
+import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
@@ -68,9 +70,11 @@ import com.google.common.collect.Sets;
  *
  */
 public class AccumuloPcjQuery implements PcjQuery {
+	private static final AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
+	
 	private final Connector accCon;
 	private final String tableName;
-
+	
 	public AccumuloPcjQuery(Connector accCon, String tableName) {
 		this.accCon = accCon;
 		this.tableName = tableName;
@@ -119,10 +123,9 @@ public class AccumuloPcjQuery implements PcjQuery {
 					}
 				}
 				try {
-					rangePrefix = AccumuloPcjSerializer.serialize(rangeBs,
-							commonVars.toArray(new String[commonVars.size()]));
-				} catch (final RyaTypeResolverException e) {
-					e.printStackTrace();
+					rangePrefix = converter.convert(rangeBs, new VariableOrder(commonVars));
+				} catch (final BindingSetConversionException e) {
+					throw new QueryEvaluationException(e);
 				}
 				final Range r = Range.prefix(new Text(rangePrefix));
 				map.put(r, bSet);
@@ -149,10 +152,9 @@ public class AccumuloPcjQuery implements PcjQuery {
 					}
 				}
 				try {
-					rangePrefix = AccumuloPcjSerializer.serialize(rangeBs,
-							commonVars.toArray(new String[commonVars.size()]));
-				} catch (final RyaTypeResolverException e) {
-					e.printStackTrace();
+					rangePrefix = converter.convert(rangeBs, new VariableOrder(commonVars));
+				} catch (final BindingSetConversionException e) {
+					throw new QueryEvaluationException(e);
 				}
 				final Range r = Range.prefix(new Text(rangePrefix));
 				ranges.add(r);
@@ -224,7 +226,7 @@ public class AccumuloPcjQuery implements PcjQuery {
 						BindingSet bs;
 						try {
 							bs = getBindingSetWithoutConstants(k, tableVarMap);
-						} catch (final RyaTypeResolverException e) {
+						} catch (final BindingSetConversionException e) {
 							throw new QueryEvaluationException(e);
 						}
 						currentSolutionBs = new QueryBindingSet();
@@ -239,7 +241,7 @@ public class AccumuloPcjQuery implements PcjQuery {
 							try {
 								rangePrefix = getPrefixByte(bs, constValMap,
 										prefixVars);
-							} catch (final RyaTypeResolverException e) {
+							} catch (final BindingSetConversionException e) {
 								throw new QueryEvaluationException(e);
 							}
 							final Range r = Range.prefix(new Text(rangePrefix));
@@ -312,7 +314,7 @@ public class AccumuloPcjQuery implements PcjQuery {
 	 */
 	private static byte[] getPrefixByte(BindingSet bs,
 			Map<String, org.openrdf.model.Value> valMap, List<String> prefixVars)
-			throws RyaTypeResolverException {
+			throws BindingSetConversionException {
 		final QueryBindingSet bSet = new QueryBindingSet();
 		for (final String var : prefixVars) {
 			if (var.startsWith(ExternalTupleSet.CONST_PREFIX)) {
@@ -322,8 +324,8 @@ public class AccumuloPcjQuery implements PcjQuery {
 				bSet.addBinding(var, bs.getBinding(var).getValue());
 			}
 		}
-		return AccumuloPcjSerializer.serialize(bSet,
-				prefixVars.toArray(new String[prefixVars.size()]));
+		
+		return converter.convert(bSet, new VariableOrder(prefixVars));
 	}
 
 	/**
@@ -331,15 +333,17 @@ public class AccumuloPcjQuery implements PcjQuery {
 	 * @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
+	 * @throws BindingSetConversionException 
 	 */
 	private static BindingSet getBindingSetWithoutConstants(Key key,
-			Map<String, String> tableVarMap) throws RyaTypeResolverException {
+			Map<String, String> tableVarMap) throws BindingSetConversionException {
 		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));
+		
+		BindingSet bindingSet = converter.convert(row, new VariableOrder(varOrder));
+		final QueryBindingSet temp = new QueryBindingSet(bindingSet);
+		
 		final QueryBindingSet bs = new QueryBindingSet();
 		for (final String var : temp.getBindingNames()) {
 			if (!tableVarMap.get(var).startsWith(ExternalTupleSet.CONST_PREFIX)) {

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/PrecompJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/PrecompJoinOptimizer.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/PrecompJoinOptimizer.java
index eb0f042..f8c6c77 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/external/PrecompJoinOptimizer.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/PrecompJoinOptimizer.java
@@ -20,6 +20,7 @@ package mvm.rya.indexing.external;
  */
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -241,6 +242,25 @@ public class PrecompJoinOptimizer implements QueryOptimizer, Configurable {
 			this.segmentFilters = segmentFilters;
 		}
 
+
+		// this handles the case when the optional/LeftJoin is the first node
+		// below the Projection node.  Checks to see if any of the ExternalTupleSets
+		// match the LeftJoin exactly.
+		//TODO ExteranlTupleSet won't match this LeftJoin if query contains Filters and order of
+		//filters does not match order of filters in ExternalTupleSet after filters are pushed down
+		@Override
+		public void meet(LeftJoin node) {
+
+			updateFilters(segmentFilters, true);
+
+			List<TupleExpr> joinArgs = matchExternalTupleSets(Arrays.asList((TupleExpr) node), tupList);
+			if(joinArgs.size() == 1 && joinArgs.get(0) instanceof ExternalTupleSet) {
+				node.replaceWith(joinArgs.get(0));
+			}
+			return;
+		}
+
+
 		@Override
 		public void meet(Join node) {
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloIndexSet.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloIndexSet.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloIndexSet.java
index 9c1bc7b..6011a52 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloIndexSet.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloIndexSet.java
@@ -104,7 +104,7 @@ public class AccumuloIndexSet extends ExternalTupleSet implements ExternalBatchi
 
     @Override
     public String getSignature() {
-        return "AccumuloIndexSet(" + tablename + ") : " + Joiner.on(", ").join(this.getTupleExpr().getAssuredBindingNames());
+        return "AccumuloIndexSet(" + tablename + ") : " + Joiner.on(", ").join(this.getTupleExpr().getBindingNames());
     }
 
     /**
@@ -130,7 +130,7 @@ public class AccumuloIndexSet extends ExternalTupleSet implements ExternalBatchi
             throw new MalformedQueryException("SPARQL query '" + sparql + "' does not contain a Projection.");
         }
         setProjectionExpr(projection.get());
-        
+
         Set<VariableOrder> orders = null;
         try {
 			orders = pcj.getPcjMetadata(accCon, tablename).getVarOrders();

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerializer.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerializer.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerializer.java
index 452ea61..bb87d7f 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerializer.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerializer.java
@@ -19,104 +19,169 @@ package mvm.rya.indexing.external.tupleSet;
  * under the License.
  */
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkArgument;
 import static mvm.rya.api.RdfCloudTripleStoreConstants.DELIM_BYTE;
 import static mvm.rya.api.RdfCloudTripleStoreConstants.DELIM_BYTES;
 import static mvm.rya.api.RdfCloudTripleStoreConstants.TYPE_DELIM_BYTE;
 
-import java.util.ArrayList;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
+
+import javax.annotation.ParametersAreNonnullByDefault;
 
 import mvm.rya.api.domain.RyaType;
 import mvm.rya.api.resolver.RdfToRyaConversions;
 import mvm.rya.api.resolver.RyaContext;
 import mvm.rya.api.resolver.RyaToRdfConversions;
 import mvm.rya.api.resolver.RyaTypeResolverException;
+import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
 
 import org.openrdf.model.Value;
+import org.openrdf.query.Binding;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.algebra.evaluation.QueryBindingSet;
 
-import com.google.common.base.Preconditions;
 import com.google.common.primitives.Bytes;
 
 /**
- * AccumuloPcjSerializer provides two methods, serialize and deserialize, which
- * are used for writing BindingSets to PCJ tables and reading serialized byte
- * representations of BindingSets from PCJ tables.
- *
+ * Converts {@link BindingSet}s to byte[]s and back again. The bytes do not
+ * include the binding names and are ordered with a {@link VariableOrder}.
  */
-public class AccumuloPcjSerializer {
-
-	/**
-	 *
-	 * @param bs {@link BindingSet} to be serialized
-	 * @param varOrder order in which binding values should be written to byte array
-	 * @return byte array containing serialized values written in order indicated by varOrder
-	 * @throws RyaTypeResolverException
-	 */
-	public static byte[] serialize(BindingSet bs, String[] varOrder) throws RyaTypeResolverException {
-		byte[] byteArray = null;
-		int i = 0;
-		Preconditions.checkNotNull(bs);
-		Preconditions.checkNotNull(varOrder);
-		Preconditions.checkArgument(bs.size() == varOrder.length);
-		for(final String varName: varOrder) {
-			final RyaType rt = RdfToRyaConversions.convertValue(bs.getBinding(varName).getValue());
-			final byte[][] serializedVal = RyaContext.getInstance().serializeType(rt);
-			if(i == 0) {
-				byteArray = Bytes.concat(serializedVal[0], serializedVal[1], DELIM_BYTES);
-			} else {
-				byteArray = Bytes.concat(byteArray, serializedVal[0], serializedVal[1], DELIM_BYTES);
+@ParametersAreNonnullByDefault
+public class AccumuloPcjSerializer implements BindingSetConverter<byte[]> {
+    
+    @Override
+    public byte[] convert(BindingSet bindingSet, VariableOrder varOrder) throws BindingSetConversionException {
+    	checkNotNull(bindingSet);
+        checkNotNull(varOrder);
+	    checkBindingsSubsetOfVarOrder(bindingSet, varOrder);
+	    
+	    // A list that holds all of the byte segments that will be concatenated at the end.
+	    // This minimizes byte[] construction.
+	    final List<byte[]> byteSegments = new LinkedList<>();
+		
+	    try {
+			for(final String varName: varOrder) {
+			    // Only write information for a variable name if the binding set contains it.
+			    if(bindingSet.hasBinding(varName)) {
+			        final RyaType rt = RdfToRyaConversions.convertValue(bindingSet.getBinding(varName).getValue());
+		            final byte[][] serializedVal = RyaContext.getInstance().serializeType(rt);
+		            byteSegments.add(serializedVal[0]);
+		            byteSegments.add(serializedVal[1]);
+			    }
+			    
+			    // But always write the value delimiter. If a value is missing, you'll see two delimiters next to each-other.
+			    byteSegments.add(DELIM_BYTES);
 			}
-			i++;
-		}
+			
+			return concat(byteSegments);
+	    } catch (RyaTypeResolverException e) {
+	    	throw new BindingSetConversionException("Could not convert the BindingSet into a byte[].", e);
+	    }
+    }
 
-		return byteArray;
-	}
+    @Override
+    public BindingSet convert(byte[] bindingSetBytes, VariableOrder varOrder) throws BindingSetConversionException {
+        checkNotNull(bindingSetBytes);
+        checkNotNull(varOrder);
+        
+        try {
+    		// Slice the row into bindings.
+    		List<byte[]> values = splitlByDelimByte(bindingSetBytes);
+    		String[] varOrderStrings = varOrder.toArray();
+    		checkArgument(values.size() == varOrderStrings.length);
+    		
+    		// Convert the Binding bytes into a BindingSet.
+    		final QueryBindingSet bindingSet = new QueryBindingSet();
+    		
+    		for(int i = 0; i < varOrderStrings.length; i++) {
+    		    byte[] valueBytes = values.get(i);
+    		    if(valueBytes.length > 0) {
+    		        String name = varOrderStrings[i];
+    		        Value value = deserializeValue(valueBytes);
+    		        bindingSet.addBinding(name, value);
+    		    }
+    		}
+    		
+    		return bindingSet;
+        } catch (RyaTypeResolverException e) {
+            throw new BindingSetConversionException("Could not convert the byte[] into a BindingSet.", e); 
+        }
+    }
+	
+    /**
+     * Checks to see if the names of all the {@link Binding}s in the {@link BindingSet}
+     * are a subset of the variables names in {@link VariableOrder}.
+     * 
+     * @param bindingSet - The binding set whose Bindings will be inspected. (not null)
+     * @param varOrder - The names of the bindings that may appear in the BindingSet. (not null)
+     * @throws IllegalArgumentException Indicates the names of the bindings are
+     *   not a subset of the variable order. 
+     */
+    private static void checkBindingsSubsetOfVarOrder(BindingSet bindingSet, VariableOrder varOrder) throws IllegalArgumentException {
+        checkNotNull(bindingSet);
+        checkNotNull(varOrder);
 
-	/**
-	 *
-	 * @param row byte rowId (read from Accumulo {@link Key})
-	 * @param varOrder indicates the order in which values are written in row
-	 * @return {@link BindingSet} formed from serialized values in row and variables in varOrder
-	 * @throws RyaTypeResolverException
-	 */
-	public static BindingSet deSerialize(byte[] row, String[] varOrder) throws RyaTypeResolverException {
-		Preconditions.checkNotNull(row);
-		Preconditions.checkNotNull(varOrder);
-		final int lastIndex = Bytes.lastIndexOf(row, DELIM_BYTE);
-		Preconditions.checkArgument(lastIndex >= 0);
-		final List<byte[]> byteList = getByteValues(Arrays.copyOf(row, lastIndex), new ArrayList<byte[]>());
-		final QueryBindingSet bs = new QueryBindingSet();
-		Preconditions.checkArgument(byteList.size() == varOrder.length);
-		for(int i = 0; i < byteList.size(); i++) {
-			bs.addBinding(varOrder[i], getValue(byteList.get(i)));
-		}
-		return bs;
-	}
+        Set<String> bindingNames = bindingSet.getBindingNames();
+        List<String> varNames = varOrder.getVariableOrders();
+        checkArgument(varNames.containsAll(bindingNames), "The BindingSet contains a Binding whose name is not part of the VariableOrder.");
+    }
+    
+    private static final byte[] concat(Iterable<byte[]> byteSegments) {
+        checkNotNull(byteSegments);
 
-	private static List<byte[]> getByteValues(byte[] row, List<byte[]> byteList) {
-		 final int firstIndex = Bytes.indexOf(row, DELIM_BYTE);
-		 if(firstIndex < 0) {
-			 byteList.add(row);
-			 return byteList;
-		 } else {
-			 byteList.add(Arrays.copyOf(row, firstIndex));
-			 getByteValues(Arrays.copyOfRange(row, firstIndex+1, row.length), byteList);
-		 }
+        // Allocate a byte array that is able to hold the segments.
+        int length = 0;
+        for(byte[] byteSegment : byteSegments) {
+            length += byteSegment.length;
+        }
+        byte[] result = new byte[length];
+        
+        // Copy the segments to the byte array and return it.
+        ByteBuffer buff = ByteBuffer.wrap(result);
+        for(byte[] byteSegment : byteSegments) {
+            buff.put(byteSegment);
+        }
+        return result;
+    }
+	
+	private static List<byte[]> splitlByDelimByte(byte[] bindingSetBytes) {
+        checkNotNull(bindingSetBytes);
 
-		 return byteList;
+        List<byte[]> values = new LinkedList<>();
+        
+        ByteBuffer buff = ByteBuffer.wrap(bindingSetBytes);
+        int start = 0;
+        while(buff.hasRemaining()) {
+            if(buff.get() == DELIM_BYTE) {
+                // Mark the position of the value delimiter.
+                int end = buff.position();
+                
+                // Move to the start of the value and copy the bytes into an array.
+                byte[] valueBytes = new byte[(end - start) -1];
+                buff.position(start);
+                buff.get(valueBytes);
+                buff.position(end);
+                values.add(valueBytes);
+                
+                // Move the start of the next value to the end of this one.
+                start = end;
+            }
+        }
+	    
+        return values;
 	}
 
-	private static Value getValue(byte[] byteVal) throws RyaTypeResolverException {
-
+	private static Value deserializeValue(byte[] byteVal) throws RyaTypeResolverException {
 		 final int typeIndex = Bytes.indexOf(byteVal, TYPE_DELIM_BYTE);
-		 Preconditions.checkArgument(typeIndex >= 0);
+		 checkArgument(typeIndex >= 0);
 		 final byte[] data = Arrays.copyOf(byteVal, typeIndex);
 		 final byte[] type = Arrays.copyOfRange(byteVal, typeIndex, byteVal.length);
 		 final RyaType rt = RyaContext.getInstance().deserialize(Bytes.concat(data,type));
 		 return RyaToRdfConversions.convertValue(rt);
-
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetConverter.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetConverter.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetConverter.java
new file mode 100644
index 0000000..9c10dfa
--- /dev/null
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetConverter.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package mvm.rya.indexing.external.tupleSet;
+
+import javax.annotation.ParametersAreNonnullByDefault;
+
+import org.openrdf.query.Binding;
+import org.openrdf.query.BindingSet;
+
+import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
+
+/**
+ * Converts {@link BindingSet}s into other representations. This library is
+ * intended to convert between BindingSet and whatever format it is being
+ * stored as. These formats are often optimized for query evaluation.
+ *
+ * @param <T> Defines the type of model {@link BindingSet}s will be converted into/from.
+ */
+@ParametersAreNonnullByDefault
+public interface BindingSetConverter<T> {
+
+   /**
+    * Converts a {@link BindingSet} into the target model. The target model
+    * may not include every {@link Binding} that was in the original BindingSet,
+    * it may not include the binding names, and it may order the binding values.
+    * All of this information is specified using a {@link VariableOrder}.
+    * </p>
+    * Because the resulting model may not include the binding names from the
+    * original object, you must hold onto that information if you want to
+    * convert the resulting model back into a BindingSet later. Because the
+    * resulting model may only contain a subset of the original BindingSet's
+    * bindings, some information may be lost, so you may not be able to convert
+    * the target model back into the original BindingSet.
+    *
+    * @param bindingSet - The BindingSet that will be converted. (not null)
+    * @param varOrder - Which bindings and in what order they will appear in the
+    *   resulting model. (not null)
+    * @return The BindingSet formatted as the target model.
+    * @throws BindingSetConversionException The BindingSet was unable to be
+    *   converted into the target model. This will happen if the BindingSet has
+    *   a binding whose name is not in the VariableOrder or if one of the values
+    *   could not be converted into the target model.
+    */
+   public T convert(BindingSet bindingSet, VariableOrder varOrder) throws BindingSetConversionException;
+
+   /**
+    * Converts the target model representation of a {@link BindingSet} as is
+    * created by {@link #convert(BindingSet, VariableOrder)} back into a
+    * BindingSet.
+    * </p>
+    * You must provide the Binding names and the order they were written to
+    * by using a {@link VariableOrder}.
+    * </p>
+    * If there is no value for one of the variable order names, then that binding
+    * will be missing from the resulting BindingSet.
+    *
+    * @param bindingSet - The BindingSet formatted as the target model that will
+    *   be converted. (not null)
+    * @param varOrder - The VariableOrder that was used to create the target model. (not null)
+    * @return The {@link BindingSet} representation of the target model.
+    * @throws BindingSetConversionException The target model was unable to be
+    *   converted back into a BindingSet.
+    */
+   public BindingSet convert(T bindingSet, VariableOrder varOrder) throws BindingSetConversionException;
+
+   /**
+    * One of the conversion methods of {@link BindingSetConverter} was unable to
+    * to convert the {@link BindingSet} to/from the converted model.
+    */
+   public static class BindingSetConversionException extends Exception {
+       private static final long serialVersionUID = 1L;
+
+       /**
+        * Constructs an instance of {@link BindingSetConversionException}.
+        *
+        * @param message - Describes why this exception was thrown.
+        */
+       public BindingSetConversionException(String message) {
+           super(message);
+       }
+
+       /**
+        * BindingSetConversionException
+        *
+        * @param message - Describes why this exception was thrown.
+        * @param cause - The exception that caused this one to be thrown.
+        */
+       public BindingSetConversionException(String message, Throwable cause) {
+           super(message, cause);
+       }
+   }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverter.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverter.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverter.java
index d28186d..6d69d5b 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverter.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverter.java
@@ -16,16 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package mvm.rya.indexing.external.tupleSet;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.util.Iterator;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
+import javax.annotation.ParametersAreNonnullByDefault;
+
 import org.openrdf.model.URI;
 import org.openrdf.model.Value;
 import org.openrdf.model.ValueFactory;
@@ -36,89 +37,68 @@ import org.openrdf.query.Binding;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.algebra.evaluation.QueryBindingSet;
 
+import com.google.common.base.Joiner;
+
 import mvm.rya.api.domain.RyaType;
 import mvm.rya.api.resolver.RdfToRyaConversions;
 import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
 
 /**
- * Converts {@link BindingSet}s to Strings and back again.
+ * Converts {@link BindingSet}s to Strings and back again. The Strings do not
+ * include the binding names and are ordered with a {@link VariableOrder}.
  */
-public class BindingSetStringConverter {
+@ParametersAreNonnullByDefault
+public class BindingSetStringConverter implements BindingSetConverter<String> {
 
-    private static final String BINDING_DELIM = ":::";
-    private static final String TYPE_DELIM = "<<~>>";
+    public static final String BINDING_DELIM = ":::";
+    public static final String TYPE_DELIM = "<<~>>";
+    public static final String NULL_VALUE_STRING = Character.toString( '\0' );
 
     private static final ValueFactory valueFactory = new ValueFactoryImpl();
 
-    /**
-     * Converts a {@link BindingSet} to a String. You must provide the order the
-     * {@link Binding}s will be written to.
-     * </p>
-     * The resulting string does not include the binding names from the original
-     * object, so that must be kept with the resulting String if you want to
-     * convert it back to a BindingSet later.
-     * </p>
-     *
-     * @param bindingSet - The BindingSet that will be converted. (not null)
-     * @param varOrder - The order the bindings will appear in the resulting String. (not null)
-     * @return A {@code String} version of {@code bindingSet} whose binding are
-     *   ordered based on {@code varOrder}.
-     */
-    public static String toString(BindingSet bindingSet, VariableOrder varOrder) {
-        checkSameVariables(bindingSet, varOrder);
-
-        final StringBuilder bindingSetString = new StringBuilder();
-
-        Iterator<String> it = varOrder.iterator();
-        while(it.hasNext()) {
-            // Add a value to the binding set.
-            String varName = it.next();
-            final Value value = bindingSet.getBinding(varName).getValue();
-            final RyaType ryaValue = RdfToRyaConversions.convertValue(value);
-            bindingSetString.append( ryaValue.getData() ).append(TYPE_DELIM).append( ryaValue.getDataType() );
-
-            // If there are more values to add, include a delimiter between them.
-            if(it.hasNext()) {
-                bindingSetString.append(BINDING_DELIM);
+    @Override
+    public String convert(BindingSet bindingSet, VariableOrder varOrder) {
+        checkBindingsSubsetOfVarOrder(bindingSet, varOrder);
+
+        // Convert each Binding to a String.
+        List<String> bindingStrings = new ArrayList<>();
+        for(String varName : varOrder) {
+            if(bindingSet.hasBinding(varName)) {
+                // Add a value to the binding set.
+                final Value value = bindingSet.getBinding(varName).getValue();
+                final RyaType ryaValue = RdfToRyaConversions.convertValue(value);
+                String bindingString = ryaValue.getData() + TYPE_DELIM + ryaValue.getDataType();
+                bindingStrings.add(bindingString);
+            } else {
+                // Add a null value to the binding set.
+                bindingStrings.add(NULL_VALUE_STRING);                
             }
         }
-
-        return bindingSetString.toString();
+        
+        // Join the bindings using the binding delim.
+        return Joiner.on(BINDING_DELIM).join(bindingStrings);
     }
 
     /**
      * Checks to see if the names of all the {@link Binding}s in the {@link BindingSet}
-     * match the variable names in the {@link VariableOrder}.
-     *
+     * are a subset of the variables names in {@link VariableOrder}.
+     * 
      * @param bindingSet - The binding set whose Bindings will be inspected. (not null)
-     * @param varOrder - The names of the bindings that must appear in the BindingSet. (not null)
-     * @throws IllegalArgumentException Indicates the number of bindings did not match
-     *   the number of variables or that the binding names did not match the names
-     *   of the variables.
+     * @param varOrder - The names of the bindings that may appear in the BindingSet. (not null)
+     * @throws IllegalArgumentException Indicates the names of the bindings are
+     *   not a subset of the variable order. 
      */
-    private static void checkSameVariables(BindingSet bindingSet, VariableOrder varOrder) throws IllegalArgumentException {
+    private static void checkBindingsSubsetOfVarOrder(BindingSet bindingSet, VariableOrder varOrder) throws IllegalArgumentException {
         checkNotNull(bindingSet);
         checkNotNull(varOrder);
 
         Set<String> bindingNames = bindingSet.getBindingNames();
-        List<String> varOrderList = varOrder.getVariableOrders();
-        checkArgument(bindingNames.size() == varOrderList.size(), "The number of Bindings must match the length of the VariableOrder.");
-        checkArgument(bindingNames.containsAll(varOrderList), "The names of the Bindings must match the variable names in VariableOrder.");
+        List<String> varNames = varOrder.getVariableOrders();
+        checkArgument(varNames.containsAll(bindingNames), "The BindingSet contains a Binding whose name is not part of the VariableOrder.");
     }
 
-    /**
-     * Converts the String representation of a {@link BindingSet} as is created
-     * by {@link #toString(BindingSet, VariableOrder)} back into a BindingSet.
-     * <p>
-     * You must provide the Binding names in the order they were written to.
-     * </p>
-     *
-     * @param bindingSetString - The binding set values as a String. (not null)
-     * @param varOrder - The order the bindings appear in the String version of
-     *   the BindingSet. (not null)
-     * @return A {@link BindingSet} representation of the String.
-     */
-    public static BindingSet fromString(final String bindingSetString, final VariableOrder varOrder) {
+    @Override
+    public BindingSet convert(String bindingSetString, VariableOrder varOrder) {
         checkNotNull(bindingSetString);
         checkNotNull(varOrder);
 
@@ -128,9 +108,12 @@ public class BindingSetStringConverter {
 
         final QueryBindingSet bindingSet = new QueryBindingSet();
         for(int i = 0; i < bindingStrings.length; i++) {
-            final String name = varOrrderArr[i];
-            final Value value = toValue(bindingStrings[i]);
-            bindingSet.addBinding(name, value);
+            String bindingString = bindingStrings[i];
+            if(!NULL_VALUE_STRING.equals(bindingString)) {
+                final String name = varOrrderArr[i];
+                final Value value = toValue(bindingStrings[i]);
+                bindingSet.addBinding(name, value);
+            }
         }
         return bindingSet;
     }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/PcjTables.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/PcjTables.java b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/PcjTables.java
index f87df51..fc940f6 100644
--- a/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/PcjTables.java
+++ b/extras/indexing/src/main/java/mvm/rya/indexing/external/tupleSet/PcjTables.java
@@ -8,9 +8,9 @@ package mvm.rya.indexing.external.tupleSet;
  * 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
@@ -35,8 +35,6 @@ import java.util.Set;
 import javax.annotation.ParametersAreNonnullByDefault;
 import javax.annotation.concurrent.Immutable;
 
-import mvm.rya.api.resolver.RyaTypeResolverException;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -76,6 +74,9 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
+import mvm.rya.api.resolver.RyaTypeResolverException;
+import mvm.rya.indexing.external.tupleSet.BindingSetConverter.BindingSetConversionException;
+
 /**
  * Functions that create and maintain the PCJ tables that are used by Rya.
  */
@@ -137,6 +138,16 @@ public class PcjTables {
         }
 
         /**
+         * Constructs an instance of {@link VariableOrdeR{.
+         *
+         * @param varOrder - An ordered collection of Binding Set variables. (not null)
+         */
+        public VariableOrder(Collection<String> varOrder) {
+            checkNotNull(varOrder);
+            this.variableOrder = ImmutableList.copyOf(varOrder);
+        }
+
+        /**
          * Constructs an instance of {@link VariableOrder}.
          *
          * @param varOrderString - The String representation of a VariableOrder. (not null)
@@ -621,17 +632,18 @@ public class PcjTables {
         checkNotNull(result);
 
         Set<Mutation> mutations = new HashSet<>();
-
+        AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
+        
         for(final VariableOrder varOrder : varOrders) {
             try {
                 // Serialize the result to the variable order.
-                byte[] serializedResult = AccumuloPcjSerializer.serialize(result, varOrder.toArray());
+                byte[] serializedResult = converter.convert(result, varOrder);
 
                 // 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) {
+            } catch(BindingSetConversionException e) {
                 throw new PcjException("Could not serialize a result.", e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/test/java/mvm/rya/indexing/external/PCJOptionalTestIT.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/PCJOptionalTestIT.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/PCJOptionalTestIT.java
new file mode 100644
index 0000000..e3ef741
--- /dev/null
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/PCJOptionalTestIT.java
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package mvm.rya.indexing.external;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import mvm.rya.api.persist.RyaDAOException;
+import mvm.rya.indexing.external.PrecompJoinOptimizerIntegrationTest.CountingResultHandler;
+import mvm.rya.indexing.external.PrecompJoinOptimizerTest.NodeCollector;
+import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
+import mvm.rya.indexing.external.tupleSet.PcjTables.PcjVarOrderFactory;
+import mvm.rya.indexing.external.tupleSet.SimpleExternalTupleSet;
+import mvm.rya.rdftriplestore.inference.InferenceEngineException;
+
+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.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.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.openrdf.model.URI;
+import org.openrdf.model.impl.LiteralImpl;
+import org.openrdf.model.impl.URIImpl;
+import org.openrdf.model.vocabulary.RDF;
+import org.openrdf.model.vocabulary.RDFS;
+import org.openrdf.query.MalformedQueryException;
+import org.openrdf.query.QueryEvaluationException;
+import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.TupleQuery;
+import org.openrdf.query.TupleQueryResultHandlerException;
+import org.openrdf.query.algebra.Projection;
+import org.openrdf.query.algebra.QueryModelNode;
+import org.openrdf.query.algebra.TupleExpr;
+import org.openrdf.query.parser.ParsedQuery;
+import org.openrdf.query.parser.sparql.SPARQLParser;
+import org.openrdf.repository.RepositoryException;
+import org.openrdf.repository.sail.SailRepository;
+import org.openrdf.repository.sail.SailRepositoryConnection;
+import org.openrdf.sail.SailException;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.base.Optional;
+
+public class PCJOptionalTestIT {
+
+
+	private SailRepositoryConnection conn, pcjConn;
+	private SailRepository repo, pcjRepo;
+	private Connector accCon;
+	String tablePrefix = "table_";
+	URI sub, sub2, obj, obj2, subclass, subclass2, talksTo, sub3, subclass3;
+
+	@Before
+	public void init() throws RepositoryException,
+			TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			TableNotFoundException, InferenceEngineException {
+
+		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");
+		sub3 = new URIImpl("uri:entity3");
+		subclass3 = new URIImpl("uri:class3");
+
+
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub2, RDFS.LABEL, new LiteralImpl("label2"));
+		conn.add(sub2, talksTo, obj2);
+		conn.add(sub3, RDF.TYPE, subclass3);
+		conn.add(sub3, RDFS.LABEL, new LiteralImpl("label3"));
+
+
+		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 testEvaluateSingeIndexExactMatch()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException {
+
+		String indexSparqlString = ""//
+				+ "SELECT ?e ?c ?l ?o" //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "  OPTIONAL{?e <uri:talksTo> ?o } . "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "c", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e a ?c . "//
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  OPTIONAL {?e <uri:talksTo> ?o } . "//
+				+ "}";//
+
+		CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+		TupleQuery tupQuery = pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString);
+		tupQuery.evaluate(crh);
+
+		Assert.assertEquals(3, crh.getCount());
+
+	}
+
+
+
+	@Test
+	public void testEvaluateSingeIndex()
+			throws TupleQueryResultHandlerException, QueryEvaluationException,
+			MalformedQueryException, RepositoryException, AccumuloException,
+			AccumuloSecurityException, TableExistsException, RyaDAOException,
+			SailException, TableNotFoundException, PcjException, InferenceEngineException {
+
+		String indexSparqlString = ""//
+				+ "SELECT ?e ?l ?o" //
+				+ "{" //
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l "//
+				+ "  OPTIONAL{?e <uri:talksTo> ?o } . "//
+				+ "}";//
+
+		PcjIntegrationTestingUtil.createAndPopulatePcj(conn, accCon, tablePrefix
+				+ "INDEX_1", indexSparqlString, new String[] { "e", "l", "o" },
+				Optional.<PcjVarOrderFactory> absent());
+		String queryString = ""//
+				+ "SELECT ?e ?c ?l ?o " //
+				+ "{" //
+				+ "  ?e <http://www.w3.org/2000/01/rdf-schema#label> ?l . "//
+				+ "  OPTIONAL {?e <uri:talksTo> ?o } . "//
+				+ "  ?e a ?c . "//
+				+ "}";//
+
+		CountingResultHandler crh = new CountingResultHandler();
+		PcjIntegrationTestingUtil.deleteCoreRyaTables(accCon, tablePrefix);
+		PcjIntegrationTestingUtil.closeAndShutdown(conn, repo);
+
+		repo = PcjIntegrationTestingUtil.getPcjRepo(tablePrefix, "instance");
+		conn = repo.getConnection();
+		conn.add(sub, RDF.TYPE, subclass);
+		conn.add(sub2, RDF.TYPE, subclass2);
+		conn.add(sub3, RDF.TYPE, subclass3);
+
+
+		TupleQuery tupQuery = pcjConn.prepareTupleQuery(QueryLanguage.SPARQL, queryString);
+		tupQuery.evaluate(crh);
+
+		Assert.assertEquals(3, crh.getCount());
+
+	}
+
+
+
+
+
+
+	@Test
+	public void testSimpleOptionalTest1() throws Exception {
+
+		String query = ""//
+				+ "SELECT ?u ?s ?t " //
+				+ "{" //
+				+ "  ?s a ?t ."//
+				+ "  OPTIONAL{?t <http://www.w3.org/2000/01/rdf-schema#label> ?u } ."//
+				+ "  ?u <uri:talksTo> ?s . "//
+				+ "}";//
+
+		final SPARQLParser parser = new SPARQLParser();
+
+		final ParsedQuery pq1 = parser.parseQuery(query, null);
+
+		final SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet(
+				(Projection) pq1.getTupleExpr().clone());
+
+		final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+
+		list.add(extTup1);
+
+		final List<QueryModelNode> optTupNodes = Lists.newArrayList();
+		optTupNodes.add(extTup1);
+
+		final PrecompJoinOptimizer pcj = new PrecompJoinOptimizer(list, true);
+		final TupleExpr te = pq1.getTupleExpr();
+		pcj.optimize(te, null, null);
+
+		final NodeCollector nc = new NodeCollector();
+		te.visit(nc);
+
+		final List<QueryModelNode> qNodes = nc.getNodes();
+
+		Assert.assertEquals(qNodes.size(), optTupNodes.size());
+		for (final QueryModelNode node : qNodes) {
+			Assert.assertTrue(optTupNodes.contains(node));
+		}
+
+	}
+
+
+	@Test
+	public void testSimpleOptionalTest2() throws Exception {
+
+		String query = ""//
+				+ "SELECT ?u ?s ?t " //
+				+ "{" //
+				+ "  ?s a ?t ."//
+				+ "  OPTIONAL{?t <http://www.w3.org/2000/01/rdf-schema#label> ?u } ."//
+				+ "  ?u <uri:talksTo> ?s . "//
+				+ "  ?s a ?u ."//
+				+ "}";//
+
+
+		String pcj = ""//
+				+ "SELECT ?d ?b ?c " //
+				+ "{" //
+				+ "  ?b a ?c ."//
+				+ "  OPTIONAL{?c <http://www.w3.org/2000/01/rdf-schema#label> ?d } ."//
+				+ "  ?d <uri:talksTo> ?b . "//
+				+ "}";//
+
+
+		String relabel_pcj = ""//
+				+ "SELECT ?u ?s ?t " //
+				+ "{" //
+				+ "  ?s a ?t ."//
+				+ "  OPTIONAL{?t <http://www.w3.org/2000/01/rdf-schema#label> ?u } ."//
+				+ "  ?u <uri:talksTo> ?s . "//
+				+ "}";//
+
+
+		final SPARQLParser parser = new SPARQLParser();
+
+		final ParsedQuery pq1 = parser.parseQuery(query, null);
+		final ParsedQuery pq2 = parser.parseQuery(pcj, null);
+		final ParsedQuery pq3 = parser.parseQuery(relabel_pcj, null);
+
+		final SimpleExternalTupleSet extTup1 = new SimpleExternalTupleSet(
+				(Projection) pq2.getTupleExpr());
+		final SimpleExternalTupleSet extTup2 = new SimpleExternalTupleSet(
+				(Projection) pq3.getTupleExpr());
+
+		final List<ExternalTupleSet> list = new ArrayList<ExternalTupleSet>();
+
+		list.add(extTup1);
+
+		final List<QueryModelNode> optTupNodes = Lists.newArrayList();
+		optTupNodes.add(extTup2);
+
+		final PrecompJoinOptimizer opt = new PrecompJoinOptimizer(list, true);
+		final TupleExpr te = pq1.getTupleExpr();
+		opt.optimize(te, null, null);
+
+		final NodeCollector nc = new NodeCollector();
+		te.visit(nc);
+
+		final List<QueryModelNode> qNodes = nc.getNodes();
+
+		Assert.assertEquals(qNodes.size(), optTupNodes.size() + 1);
+		for (QueryModelNode node : optTupNodes) {
+			Assert.assertTrue(qNodes.contains(node));
+		}
+
+	}
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/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
index 42e8b09..1be88e2 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/PcjIntegrationTestingUtil.java
@@ -31,6 +31,7 @@ import mvm.rya.api.resolver.RyaTypeResolverException;
 import mvm.rya.sail.config.RyaSailFactory;
 import mvm.rya.indexing.accumulo.ConfigUtils;
 import mvm.rya.indexing.external.tupleSet.AccumuloPcjSerializer;
+import mvm.rya.indexing.external.tupleSet.BindingSetConverter.BindingSetConversionException;
 import mvm.rya.indexing.external.tupleSet.ExternalTupleSet;
 import mvm.rya.indexing.external.tupleSet.PcjTables;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
@@ -70,6 +71,8 @@ import com.google.common.collect.Sets;
 
 public class PcjIntegrationTestingUtil {
 
+	private static final AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
+	
 	public static Set<QueryModelNode> getTupleSets(TupleExpr te) {
 		final ExternalTupleVisitor etv = new ExternalTupleVisitor();
 		te.visit(etv);
@@ -386,20 +389,18 @@ public class PcjIntegrationTestingUtil {
 		for (final VariableOrder varOrder : varOrders) {
 			try {
 				// Serialize the result to the variable order.
-				byte[] serializedResult = AccumuloPcjSerializer.serialize(
-						result, varOrder.toArray());
+				byte[] serializedResult = converter.convert(result, varOrder);
 
 				// 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) {
+			} catch (BindingSetConversionException e) {
 				throw new PcjException("Could not serialize a result.", e);
 			}
 		}
 
 		return mutations;
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/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 6e8c721..9428ae4 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
@@ -174,6 +174,8 @@ public class PrecompJoinOptimizerTest {
     }
 
 
+
+
     @Test
     public void testSingleIndex2() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerialzerTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerialzerTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerialzerTest.java
index 2fcacb0..4efbb30 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerialzerTest.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/AccumuloPcjSerialzerTest.java
@@ -20,99 +20,141 @@ package mvm.rya.indexing.external.tupleSet;
  */
 
 import mvm.rya.api.resolver.RyaTypeResolverException;
+import mvm.rya.indexing.external.tupleSet.BindingSetConverter.BindingSetConversionException;
+import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
+
+import static org.junit.Assert.assertEquals;
 
 import org.junit.Test;
 import org.openrdf.model.impl.LiteralImpl;
 import org.openrdf.model.impl.URIImpl;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.algebra.evaluation.QueryBindingSet;
+import org.openrdf.query.impl.MapBindingSet;
 
-import com.vividsolutions.jts.util.Assert;
-
+/**
+ * Tests the methods of {@link AccumuloPcjSerialzer}.
+ */
 public class AccumuloPcjSerialzerTest {
 
+    /**
+     * The BindingSet has fewer Bindings than there are variables in the variable
+     * order, but they are all in the variable order. This is the case where
+     * the missing bindings were optional.
+     */
+    @Test
+    public void serialize_bindingsSubsetOfVarOrder() throws BindingSetConversionException {
+        // Setup the Binding Set.
+        final MapBindingSet originalBindingSet = new MapBindingSet();
+        originalBindingSet.addBinding("x", new URIImpl("http://a"));
+        originalBindingSet.addBinding("y", new URIImpl("http://b"));
+        
+        // Setup the variable order.
+        final VariableOrder varOrder = new VariableOrder("x", "a", "y", "b");
+        
+        // Create the byte[] representation of the BindingSet.
+        BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+        byte[] serialized = converter.convert(originalBindingSet, varOrder);
+        
+        // Deserialize the byte[] back into the binding set.
+        BindingSet deserialized = converter.convert(serialized, varOrder);
+        
+        // Ensure the deserialized value matches the original.
+        assertEquals(originalBindingSet, deserialized);
+    }
+
+    /**
+     * The BindingSet has a Binding whose name is not in the variable order.
+     * This is illegal.
+     */
+    @Test(expected = IllegalArgumentException.class)
+    public void serialize_bindingNotInVariableOrder() throws RyaTypeResolverException, BindingSetConversionException {
+        // Setup the Binding Set.
+        final MapBindingSet originalBindingSet = new MapBindingSet();
+        originalBindingSet.addBinding("x", new URIImpl("http://a"));
+        originalBindingSet.addBinding("y", new URIImpl("http://b"));
+        originalBindingSet.addBinding("z", new URIImpl("http://d"));
+
+        // Setup the variable order.
+        final VariableOrder varOrder = new VariableOrder("x", "y");
+        
+        // Create the byte[] representation of the BindingSet. This will throw an exception.
+        BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+        converter.convert(originalBindingSet, varOrder);
+    }
+    
 	@Test
-	public void basicShortUriBsTest() {
+	public void basicShortUriBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new URIImpl("http://uri1"));
 		bs.addBinding("Y",new URIImpl("http://uri2"));
-		final String[] varOrder = new String[]{"X","Y"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("X","Y");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
 
 	@Test
-	public void basicLongUriBsTest() {
+	public void basicLongUriBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new URIImpl("http://uri1"));
 		bs.addBinding("Y",new URIImpl("http://uri2"));
 		bs.addBinding("Z",new URIImpl("http://uri3"));
 		bs.addBinding("A",new URIImpl("http://uri4"));
 		bs.addBinding("B",new URIImpl("http://uri5"));
-		final String[] varOrder = new String[]{"X","Y","Z","A","B"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("X","Y","Z","A","B");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
 
 	@Test
-	public void basicShortStringLiteralBsTest() {
+	public void basicShortStringLiteralBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new LiteralImpl("literal1"));
 		bs.addBinding("Y",new LiteralImpl("literal2"));
-		final String[] varOrder = new String[]{"X","Y"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("X","Y");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
 
 	@Test
-	public void basicShortMixLiteralBsTest() {
+	public void basicShortMixLiteralBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new LiteralImpl("literal1"));
 		bs.addBinding("Y",new LiteralImpl("5", new URIImpl("http://www.w3.org/2001/XMLSchema#integer")));
-		final String[] varOrder = new String[]{"X","Y"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("X","Y");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
 
 	@Test
-	public void basicLongMixLiteralBsTest() {
+	public void basicLongMixLiteralBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new LiteralImpl("literal1"));
 		bs.addBinding("Y",new LiteralImpl("5", new URIImpl("http://www.w3.org/2001/XMLSchema#integer")));
 		bs.addBinding("Z",new LiteralImpl("5.0", new URIImpl("http://www.w3.org/2001/XMLSchema#double")));
 		bs.addBinding("W",new LiteralImpl("1000", new URIImpl("http://www.w3.org/2001/XMLSchema#long")));
-		final String[] varOrder = new String[]{"W","X","Y","Z"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("W","X","Y","Z");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
 
 	@Test
-	public void basicMixUriLiteralBsTest() {
+	public void basicMixUriLiteralBsTest() throws BindingSetConversionException {
 		final QueryBindingSet bs = new QueryBindingSet();
 		bs.addBinding("X",new LiteralImpl("literal1"));
 		bs.addBinding("Y",new LiteralImpl("5", new URIImpl("http://www.w3.org/2001/XMLSchema#integer")));
@@ -121,13 +163,11 @@ public class AccumuloPcjSerialzerTest {
 		bs.addBinding("A",new URIImpl("http://uri1"));
 		bs.addBinding("B",new URIImpl("http://uri2"));
 		bs.addBinding("C",new URIImpl("http://uri3"));
-		final String[] varOrder = new String[]{"A","W","X","Y","Z","B","C"};
-		try {
-			final byte[] byteVal = AccumuloPcjSerializer.serialize(bs, varOrder);
-			final BindingSet newBs = AccumuloPcjSerializer.deSerialize(byteVal, varOrder);
-			Assert.equals(bs, newBs);
-		} catch (final RyaTypeResolverException e) {
-			e.printStackTrace();
-		}
+		final VariableOrder varOrder = new VariableOrder("A","W","X","Y","Z","B","C");
+		
+		BindingSetConverter<byte[]> converter = new AccumuloPcjSerializer();
+		final byte[] byteVal = converter.convert(bs, varOrder);
+		final BindingSet newBs = converter.convert(byteVal, varOrder);
+		assertEquals(bs, newBs);
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverterTest.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverterTest.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverterTest.java
index 0a7f399..725e557 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverterTest.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/BindingSetStringConverterTest.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package mvm.rya.indexing.external.tupleSet;
 
 import static org.junit.Assert.assertEquals;
@@ -32,6 +31,7 @@ import org.openrdf.model.impl.URIImpl;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.impl.MapBindingSet;
 
+import mvm.rya.indexing.external.tupleSet.BindingSetConverter.BindingSetConversionException;
 import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
 
 /**
@@ -40,7 +40,7 @@ import mvm.rya.indexing.external.tupleSet.PcjTables.VariableOrder;
 public class BindingSetStringConverterTest {
 
     @Test
-    public void toString_URIs() {
+    public void toString_URIs() throws BindingSetConversionException {
         // Setup the binding set that will be converted.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new URIImpl("http://a"));
@@ -49,9 +49,10 @@ public class BindingSetStringConverterTest {
 
         // Convert it to a String.
         final VariableOrder varOrder = new VariableOrder("y", "z", "x");
-        final String bindingSetString = BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final String bindingSetString = converter.convert(originalBindingSet, varOrder);
 
-        // Ensure it converted to the expected result.
+        // Ensure it converted to the expected result.l
         final String expected =
                 "http://b<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
                 "http://c<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
@@ -61,14 +62,15 @@ public class BindingSetStringConverterTest {
     }
 
     @Test
-    public void toString_Decimal() {
+    public void toString_Decimal() throws BindingSetConversionException {
         // Setup the binding set that will be converted.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new DecimalLiteralImpl(new BigDecimal(2.5)));
 
         // Convert it to a String.
         final VariableOrder varOrder = new VariableOrder("x");
-        final String bindingSetString = BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final String bindingSetString = converter.convert(originalBindingSet, varOrder);
 
         // Ensure it converted to the expected result.
         final String expected = "2.5<<~>>http://www.w3.org/2001/XMLSchema#decimal";
@@ -76,14 +78,15 @@ public class BindingSetStringConverterTest {
     }
 
     @Test
-    public void toString_Boolean() {
+    public void toString_Boolean() throws BindingSetConversionException {
         // Setup the binding set that will be converted.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new BooleanLiteralImpl(true));
 
         // Convert it to a String.
         final VariableOrder varOrder = new VariableOrder("x");
-        final String bindingSetString = BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final String bindingSetString = converter.convert(originalBindingSet, varOrder);
 
         // Ensure it converted to the expected result.
         final String expected = "true<<~>>http://www.w3.org/2001/XMLSchema#boolean";
@@ -91,64 +94,96 @@ public class BindingSetStringConverterTest {
     }
 
     @Test
-    public void toString_Integer() {
+    public void toString_Integer() throws BindingSetConversionException {
         // Setup the binding set that will be converted.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new IntegerLiteralImpl(BigInteger.valueOf(5)));
 
         // Convert it to a String.
         final VariableOrder varOrder = new VariableOrder("x");
-        final String bindingSetString = BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final String bindingSetString = converter.convert(originalBindingSet, varOrder);
 
         // Ensure it converted to the expected result.
         final String expected = "5<<~>>http://www.w3.org/2001/XMLSchema#integer";
         assertEquals(expected, bindingSetString);
     }
 
-    @Test(expected = IllegalArgumentException.class)
-    public void toString_varOrderTooShort() {
-        // Setup the binding set that will be converted.
+    /**
+     * All of the Bindings in the BindingSet exactly match the variable order.
+     * This is the simplest case and is legal.
+     */
+    @Test
+    public void toString_bindingsMatchVarOrder() throws BindingSetConversionException {
+        // Setup the Binding Set.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new URIImpl("http://a"));
         originalBindingSet.addBinding("y", new URIImpl("http://b"));
 
-        // This variable order that is too short.
-        final VariableOrder varOrder = new VariableOrder("y");
+        // Setup the variable order.
+        final VariableOrder varOrder = new VariableOrder("x", "y");
 
-        // The conversion should throw an exception.
-        BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        // Create the String representation of the BindingSet.
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        String bindingSetString = converter.convert(originalBindingSet, varOrder);
+
+        // Ensure the expected value was created.
+        String expected =
+                "http://a<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
+                "http://b<<~>>http://www.w3.org/2001/XMLSchema#anyURI";
+        assertEquals(expected, bindingSetString);
     }
 
-    @Test(expected = IllegalArgumentException.class)
-    public void toString_varOrderTooLong() {
-        // Setup the binding set that will be converted.
+    /**
+     * The BindingSet has fewer Bindings than there are variables in the variable
+     * order, but they are all in the variable order. This is the case where
+     * the missing bindings were optional.
+     */
+    @Test
+    public void toString_bindingsSubsetOfVarOrder() throws BindingSetConversionException {
+        // Setup the Binding Set.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new URIImpl("http://a"));
         originalBindingSet.addBinding("y", new URIImpl("http://b"));
 
-        // This variable order is too long.
-        final VariableOrder varOrder = new VariableOrder("x", "y", "z");
+        // Setup the variable order.
+        final VariableOrder varOrder = new VariableOrder("x", "a", "y", "b");
 
-        // The conversion should throw an exception.
-        BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        // Create the String representation of the BindingSet.
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        String bindingSetString = converter.convert(originalBindingSet, varOrder);
+
+        // Ensure the expected value was created.
+        String expected =
+                "http://a<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
+                BindingSetStringConverter.NULL_VALUE_STRING + ":::" +
+                "http://b<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
+                BindingSetStringConverter.NULL_VALUE_STRING;
+        assertEquals(expected, bindingSetString);
     }
 
+    /**
+     * The BindingSet has a Binding whose name is not in the variable order.
+     * This is illegal.
+     */
     @Test(expected = IllegalArgumentException.class)
-    public void toString_varOrderWrongBindingNames() {
-        // Setup the binding set that will be converted.
+    public void toString_bindingNotInVariableOrder() throws BindingSetConversionException {
+        // Setup the Binding Set.
         final MapBindingSet originalBindingSet = new MapBindingSet();
         originalBindingSet.addBinding("x", new URIImpl("http://a"));
         originalBindingSet.addBinding("y", new URIImpl("http://b"));
+        originalBindingSet.addBinding("z", new URIImpl("http://d"));
 
-        // This variable order has the wrong binding names.
-        final VariableOrder varOrder = new VariableOrder("x", "a");
+        // Setup the variable order.
+        final VariableOrder varOrder = new VariableOrder("x", "y");
 
-        // The conversion should throw an exception.
-        BindingSetStringConverter.toString(originalBindingSet, varOrder);
+        // Create the String representation of the BindingSet. This will throw an exception.
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        converter.convert(originalBindingSet, varOrder);
     }
 
     @Test
-    public void fromString() {
+    public void fromString() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString =
                 "http://b<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
@@ -157,7 +192,8 @@ public class BindingSetStringConverterTest {
 
         // Convert it to a BindingSet
         final VariableOrder varOrder = new VariableOrder("y", "z", "x");
-        final BindingSet bindingSet = BindingSetStringConverter.fromString(bindingSetString, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final BindingSet bindingSet = converter.convert(bindingSetString, varOrder);
 
         // Ensure it converted to the expected result.
         final MapBindingSet expected = new MapBindingSet();
@@ -168,13 +204,40 @@ public class BindingSetStringConverterTest {
         assertEquals(expected, bindingSet);
     }
 
+    /**
+     * Ensures that when a binding set is converted from a String back to a 
+     * BindingSet, null values do not get converted into Bindings.
+     */
+    @Test
+    public void fromString_nullValues() throws BindingSetConversionException {
+        // Setup the String that will be converted.
+        final String bindingSetString =
+                "http://value 1<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
+                BindingSetStringConverter.NULL_VALUE_STRING + ":::" +
+                "http://value 2<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
+                BindingSetStringConverter.NULL_VALUE_STRING;
+        
+        // Convert it to a BindingSet
+        VariableOrder varOrder = new VariableOrder("x", "a", "y", "b");
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        BindingSet bindingSet = converter.convert(bindingSetString, varOrder);
+        
+        // Ensure it converted to the expected reuslt.
+        final MapBindingSet expected = new MapBindingSet();
+        expected.addBinding("x", new URIImpl("http://value 1"));
+        expected.addBinding("y", new URIImpl("http://value 2"));
+        
+        assertEquals(expected, bindingSet);
+    }
+    
     @Test
-    public void fromString_Decimal() {
+    public void fromString_Decimal() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString = "2.5<<~>>http://www.w3.org/2001/XMLSchema#decimal";
 
         // Convert it to a BindingSet
-        final BindingSet bindingSet = BindingSetStringConverter.fromString(bindingSetString, new VariableOrder("x"));
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final BindingSet bindingSet = converter.convert(bindingSetString, new VariableOrder("x"));
 
         // Ensure it converted to the expected result.
         final MapBindingSet expected = new MapBindingSet();
@@ -184,12 +247,13 @@ public class BindingSetStringConverterTest {
     }
 
     @Test
-    public void fromString_Boolean() {
+    public void fromString_Boolean() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString = "true<<~>>http://www.w3.org/2001/XMLSchema#boolean";
 
         // Convert it to a BindingSet
-        final BindingSet bindingSet = BindingSetStringConverter.fromString(bindingSetString, new VariableOrder("x"));
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final BindingSet bindingSet = converter.convert(bindingSetString, new VariableOrder("x"));
 
         // Ensure it converted to the expected result.
         final MapBindingSet expected = new MapBindingSet();
@@ -199,12 +263,13 @@ public class BindingSetStringConverterTest {
     }
 
     @Test
-    public void fromString_Integer() {
+    public void fromString_Integer() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString = "5<<~>>http://www.w3.org/2001/XMLSchema#integer";
 
         // Convert it to a BindingSet
-        final BindingSet bindingSet = BindingSetStringConverter.fromString(bindingSetString, new VariableOrder("x"));
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        final BindingSet bindingSet = converter.convert(bindingSetString, new VariableOrder("x"));
 
         // Ensure it converted to the expected result.
         final MapBindingSet expected = new MapBindingSet();
@@ -214,7 +279,7 @@ public class BindingSetStringConverterTest {
     }
 
     @Test(expected = IllegalArgumentException.class)
-    public void fromString_varOrderTooShort() {
+    public void fromString_varOrderTooShort() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString =
                 "http://a<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
@@ -224,11 +289,12 @@ public class BindingSetStringConverterTest {
         VariableOrder varOrder = new VariableOrder("x");
 
         // The conversion should throw an exception.
-        BindingSetStringConverter.fromString(bindingSetString, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        converter.convert(bindingSetString, varOrder);
     }
 
     @Test(expected = IllegalArgumentException.class)
-    public void fromString_varOrderTooLong() {
+    public void fromString_varOrderTooLong() throws BindingSetConversionException {
         // Setup the String that will be converted.
         final String bindingSetString =
                 "http://a<<~>>http://www.w3.org/2001/XMLSchema#anyURI:::" +
@@ -238,6 +304,7 @@ public class BindingSetStringConverterTest {
         VariableOrder varOrder = new VariableOrder("x", "y", "z");
 
         // The conversion should throw an exception.
-        BindingSetStringConverter.fromString(bindingSetString, varOrder);
+        BindingSetConverter<String> converter = new BindingSetStringConverter();
+        converter.convert(bindingSetString, varOrder);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/PcjTablesIntegrationTests.java
----------------------------------------------------------------------
diff --git a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/PcjTablesIntegrationTests.java b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/PcjTablesIntegrationTests.java
index e3adc16..bb21d33 100644
--- a/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/PcjTablesIntegrationTests.java
+++ b/extras/indexing/src/test/java/mvm/rya/indexing/external/tupleSet/PcjTablesIntegrationTests.java
@@ -33,6 +33,7 @@ import mvm.rya.accumulo.AccumuloRyaDAO;
 import mvm.rya.api.RdfCloudTripleStoreConfiguration;
 import mvm.rya.api.resolver.RyaTypeResolverException;
 import mvm.rya.indexing.accumulo.ConfigUtils;
+import mvm.rya.indexing.external.tupleSet.BindingSetConverter.BindingSetConversionException;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjMetadata;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjTableNameFactory;
@@ -83,6 +84,8 @@ import com.google.common.io.Files;
 public class PcjTablesIntegrationTests {
     private static final Logger log = Logger.getLogger(PcjTablesIntegrationTests.class);
 
+    private static final AccumuloPcjSerializer converter = new AccumuloPcjSerializer();
+    
     protected static final String RYA_TABLE_PREFIX = "demo_";
 
     // Rya data store and connections.
@@ -137,7 +140,7 @@ public class PcjTablesIntegrationTests {
      * The method being tested is {@link PcjTables#addResults(Connector, String, java.util.Collection)}
      */
     @Test
-    public void addResults() throws PcjException, TableNotFoundException, RyaTypeResolverException {
+    public void addResults() throws PcjException, TableNotFoundException, BindingSetConversionException {
         final String sparql =
                 "SELECT ?name ?age " +
                 "{" +
@@ -189,7 +192,7 @@ public class PcjTablesIntegrationTests {
      * The method being tested is: {@link PcjTables#populatePcj(Connector, String, RepositoryConnection, String)}
      */
     @Test
-    public void populatePcj() throws RepositoryException, PcjException, TableNotFoundException, RyaTypeResolverException {
+    public void populatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException {
         // Load some Triples into Rya.
         Set<Statement> triples = new HashSet<>();
         triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
@@ -257,7 +260,7 @@ public class PcjTablesIntegrationTests {
      * The method being tested is: {@link PcjTables#createAndPopulatePcj(RepositoryConnection, Connector, String, String, String[], Optional)}
      */
     @Test
-    public void createAndPopulatePcj() throws RepositoryException, PcjException, TableNotFoundException, RyaTypeResolverException {
+    public void createAndPopulatePcj() throws RepositoryException, PcjException, TableNotFoundException, BindingSetConversionException {
         // Load some Triples into Rya.
         Set<Statement> triples = new HashSet<>();
         triples.add( new StatementImpl(new URIImpl("http://Alice"), new URIImpl("http://hasAge"), new NumericLiteralImpl(14, XMLSchema.INTEGER)) );
@@ -322,7 +325,7 @@ public class PcjTablesIntegrationTests {
      * multimap stores a set of deserialized binding sets that were in the PCJ
      * table for every variable order that is found in the PCJ metadata.
      */
-    private static Multimap<String, BindingSet> loadPcjResults(Connector accumuloConn, String pcjTableName) throws PcjException, TableNotFoundException, RyaTypeResolverException {
+    private static Multimap<String, BindingSet> loadPcjResults(Connector accumuloConn, String pcjTableName) throws PcjException, TableNotFoundException, BindingSetConversionException {
         Multimap<String, BindingSet> fetchedResults = HashMultimap.create();
 
         // Get the variable orders the data was written to.
@@ -336,7 +339,7 @@ public class PcjTablesIntegrationTests {
 
             for(Entry<Key, Value> entry : scanner) {
                 byte[] serializedResult = entry.getKey().getRow().getBytes();
-                BindingSet result = AccumuloPcjSerializer.deSerialize(serializedResult, varOrder.toArray());
+                BindingSet result = converter.convert(serializedResult, varOrder);
                 fetchedResults.put(varOrder.toString(), result);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-rya/blob/a24515b0/extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java
----------------------------------------------------------------------
diff --git a/extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java b/extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java
index b99d293..12e32b6 100644
--- a/extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java
+++ b/extras/rya.pcj.fluo/pcj.fluo.api/src/main/java/org/apache/rya/indexing/pcj/fluo/api/CreatePcj.java
@@ -36,10 +36,12 @@ import org.apache.rya.indexing.pcj.fluo.app.query.FluoQueryMetadataDAO;
 import org.apache.rya.indexing.pcj.fluo.app.query.SparqlFluoQueryBuilder;
 import org.apache.rya.indexing.pcj.fluo.app.query.SparqlFluoQueryBuilder.NodeIds;
 import org.apache.rya.indexing.pcj.fluo.app.query.StatementPatternMetadata;
+import org.openrdf.query.Binding;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.QueryEvaluationException;
 import org.openrdf.query.algebra.StatementPattern;
+import org.openrdf.query.impl.MapBindingSet;
 import org.openrdf.query.parser.ParsedQuery;
 import org.openrdf.query.parser.sparql.SPARQLParser;
 import org.openrdf.sail.SailConnection;
@@ -48,6 +50,7 @@ import org.openrdf.sail.SailException;
 import info.aduna.iteration.CloseableIteration;
 import io.fluo.api.client.FluoClient;
 import io.fluo.api.types.TypedTransaction;
+import mvm.rya.indexing.external.tupleSet.BindingSetStringConverter;
 import mvm.rya.indexing.external.tupleSet.PcjTables;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjException;
 import mvm.rya.indexing.external.tupleSet.PcjTables.PcjTableNameFactory;
@@ -229,13 +232,21 @@ public class CreatePcj {
         checkNotNull(spMetadata);
         checkNotNull(batch);
 
+        BindingSetStringConverter converter = new BindingSetStringConverter();
+
         try(TypedTransaction tx = STRING_TYPED_LAYER.wrap(fluo.newTransaction())) {
             // Get the node's variable order.
             final String spNodeId = spMetadata.getNodeId();
-            final String[] varOrder = spMetadata.getVariableOrder().toArray();
+            final VariableOrder varOrder = spMetadata.getVariableOrder();
 
             for(final BindingSet bindingSet : batch) {
-                final String bindingSetStr = FluoStringConverter.toBindingSetString(bindingSet, varOrder);
+                MapBindingSet spBindingSet = new MapBindingSet();
+                for(String var : varOrder) {
+                    Binding binding = bindingSet.getBinding(var);
+                    spBindingSet.addBinding(binding);
+                }
+
+                String bindingSetStr = converter.convert(spBindingSet, varOrder);
 
                 // Write the binding set entry to Fluo for the statement pattern.
                 tx.mutate().row(spNodeId + NODEID_BS_DELIM + bindingSetStr)