You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2013/07/31 02:59:05 UTC

[1/2] Clean up types, remove extraneous types, update BOOLEAN/BIT to be consistently BIT Implement ValueHolders for all types through code generation, remove manually coded value holders. Rename ValueVector.randomizeData() to ValueVector.generateTestData

Updated Branches:
  refs/heads/master 9f69ed05b -> 742f4c119


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 4e2b820..29660cb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -34,7 +34,6 @@ import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.NonRepeatedMutator;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -134,11 +133,7 @@ public class ProjectRecordBatch implements RecordBatch{
       projector.projectRecords(recordCount, 0);
       for(ValueVector v : this.outputVectors){
         ValueVector.Mutator m = v.getMutator();
-        if(m instanceof NonRepeatedMutator){
-          ((NonRepeatedMutator) m).setValueCount(recordCount);
-        }else{
-          throw new UnsupportedOperationException();
-        }
+        m.setValueCount(recordCount);
       }
       return upstream; // change if upstream changed, otherwise normal.
     default:
@@ -171,7 +166,7 @@ public class ProjectRecordBatch implements RecordBatch{
       }
       
       // add value vector to transfer if direct reference and this is allowed, otherwise, add to evaluation stack.
-      if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVector() == SelectionVectorMode.NONE){
+      if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE){
         ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
         ValueVector vvIn = incoming.getValueVectorById(vectorRead.getFieldId(), TypeHelper.getValueVectorClass(vectorRead.getMajorType().getMinorType(), vectorRead.getMajorType().getMode()));
         Preconditions.checkNotNull(incoming);
@@ -190,7 +185,7 @@ public class ProjectRecordBatch implements RecordBatch{
       
     }
     
-    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(incoming.getSchema().getSelectionVector());
+    SchemaBuilder bldr = BatchSchema.newBuilder().setSelectionVectorMode(incoming.getSchema().getSelectionVectorMode());
     for(ValueVector v : outputVectors){
       bldr.addField(v.getField());
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
index 646e6d1..36c6109 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectorTemplate.java
@@ -58,7 +58,7 @@ public abstract class ProjectorTemplate implements Projector {
   @Override
   public final void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, List<TransferPair> transfers)  throws SchemaChangeException{
 
-    this.svMode = incoming.getSchema().getSelectionVector(); 
+    this.svMode = incoming.getSchema().getSelectionVectorMode(); 
     switch(svMode){
     case FOUR_BYTE:
       this.vector4 = incoming.getSelectionVector4();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
index 208ebc6..7b90717 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
@@ -19,7 +19,6 @@ import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.FixedWidthVector;
-import org.apache.drill.exec.vector.NonRepeatedMutator;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
 
@@ -119,11 +118,7 @@ public class RemovingRecordBatch implements RecordBatch{
       copier.copyRecords();
       for(ValueVector v : this.outputVectors){
         ValueVector.Mutator m = v.getMutator();
-        if(m instanceof NonRepeatedMutator){
-          ((NonRepeatedMutator) m).setValueCount(recordCount);
-        }else{
-          throw new UnsupportedOperationException();
-        }
+        m.setValueCount(recordCount);
       }
       return upstream; // change if upstream changed, otherwise normal.
     default:
@@ -169,7 +164,7 @@ public class RemovingRecordBatch implements RecordBatch{
   }
   
   private Copier getGeneratedCopier() throws SchemaChangeException{
-    Preconditions.checkArgument(incoming.getSchema().getSelectionVector() == SelectionVectorMode.TWO_BYTE);
+    Preconditions.checkArgument(incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE);
     
     List<VectorAllocator> allocators = Lists.newArrayList();
     for(ValueVector i : incoming){
@@ -205,7 +200,7 @@ public class RemovingRecordBatch implements RecordBatch{
     }
     this.outSchema = bldr.build();
     
-    switch(incoming.getSchema().getSelectionVector()){
+    switch(incoming.getSchema().getSelectionVectorMode()){
     case NONE:
       return getStraightCopier();
     case TWO_BYTE:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index bb07e56..1ef6f44 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -23,13 +23,13 @@ import java.util.List;
 
 public class BatchSchema implements Iterable<MaterializedField> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
-  final SelectionVectorMode selectionVector;
+  final SelectionVectorMode selectionVectorMode;
   ;
   private final List<MaterializedField> fields;
 
   BatchSchema(SelectionVectorMode selectionVector, List<MaterializedField> fields) {
     this.fields = fields;
-    this.selectionVector = selectionVector;
+    this.selectionVectorMode = selectionVector;
   }
 
   public static SchemaBuilder newBuilder() {
@@ -41,13 +41,13 @@ public class BatchSchema implements Iterable<MaterializedField> {
     return fields.iterator();
   }
 
-  public SelectionVectorMode getSelectionVector() {
-    return selectionVector;
+  public SelectionVectorMode getSelectionVectorMode() {
+    return selectionVectorMode;
   }
 
   @Override
   public String toString() {
-    return "BatchSchema [fields=" + fields + ", selectionVector=" + selectionVector + "]";
+    return "BatchSchema [fields=" + fields + ", selectionVector=" + selectionVectorMode + "]";
   }
 
   public static enum SelectionVectorMode {
@@ -60,5 +60,34 @@ public class BatchSchema implements Iterable<MaterializedField> {
     }
   }
 
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((fields == null) ? 0 : fields.hashCode());
+    result = prime * result + ((selectionVectorMode == null) ? 0 : selectionVectorMode.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    BatchSchema other = (BatchSchema) obj;
+    if (fields == null) {
+      if (other.fields != null)
+        return false;
+    } else if (!fields.equals(other.fields))
+      return false;
+    if (selectionVectorMode != other.selectionVectorMode)
+      return false;
+    return true;
+  }
+  
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 732dab3..abde61a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -140,6 +140,33 @@ public class MaterializedField{
     return !iter.hasNext();
   }
   
+  
+  
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((def == null) ? 0 : def.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    MaterializedField other = (MaterializedField) obj;
+    if (def == null) {
+      if (other.def != null)
+        return false;
+    } else if (!def.equals(other.def))
+      return false;
+    return true;
+  }
+
   @Override
   public String toString() {
     return "MaterializedField [" + def.toString() + "]";

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
index 4ab908f..c2f2a69 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
@@ -3,6 +3,7 @@ package org.apache.drill.exec.record;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -12,7 +13,7 @@ public class NullExpression implements LogicalExpression{
 
   public static final NullExpression INSTANCE = new NullExpression();
   
-  final MajorType t = MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(MinorType.NULL).build();
+  final MajorType t = Types.optional(MinorType.NULL);
   
   @Override
   public MajorType getMajorType() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index f49e77f..e7f381f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -77,7 +77,7 @@ public class WritableBatch {
   }
   
   public static WritableBatch get(RecordBatch batch) {
-    if(batch.getSchema() != null && batch.getSchema().getSelectionVector() != SelectionVectorMode.NONE) throw new UnsupportedOperationException("Only batches without selections vectors are writable.");
+    if(batch.getSchema() != null && batch.getSchema().getSelectionVectorMode() != SelectionVectorMode.NONE) throw new UnsupportedOperationException("Only batches without selections vectors are writable.");
     return getBatchNoSV(batch.getRecordCount(), batch);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
index 88f0c79..33938cc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
@@ -56,6 +56,13 @@ public class SelectionVector2 implements Closeable{
     buffer = allocator.buffer(size * 2);
   }
   
+  public SelectionVector2 clone(){
+    SelectionVector2 newSV = new SelectionVector2(allocator);
+    newSV.recordCount = recordCount;
+    newSV.buffer = buffer;
+    clear();
+    return newSV;
+  }
   
   public void clear() {
     if (buffer != DeadBuf.DEAD_BUFFER) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
index d0b27fc..0e2c052 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/schema/json/jackson/JacksonHelper.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.schema.json.jackson;
 
 import java.io.IOException;
 
+import org.apache.drill.common.types.Types;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -29,13 +30,13 @@ import com.fasterxml.jackson.core.JsonToken;
 
 public class JacksonHelper {
 
-    public static final MajorType STRING_TYPE = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(DataMode.OPTIONAL).build();
-    public static final MajorType BOOLEAN_TYPE = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.OPTIONAL).build();
-    public static final MajorType ARRAY_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(DataMode.REPEATED).build();
-    public static final MajorType MAP_TYPE = MajorType.newBuilder().setMinorType(MinorType.MAP).setMode(DataMode.REPEATED).build();
-    public static final MajorType INT_TYPE = MajorType.newBuilder().setMinorType(MinorType.INT).setMode(DataMode.OPTIONAL).build();
-    public static final MajorType FLOAT_TYPE = MajorType.newBuilder().setMinorType(MinorType.FLOAT4).setMode(DataMode.OPTIONAL).build();
-    public static final MajorType NULL_TYPE = MajorType.newBuilder().setMinorType(MinorType.LATE).setMode(DataMode.OPTIONAL).build();
+    public static final MajorType STRING_TYPE = Types.optional(MinorType.VARCHAR);
+    public static final MajorType BOOLEAN_TYPE = Types.optional(MinorType.BIT);
+    public static final MajorType ARRAY_TYPE = Types.repeated(MinorType.LATE);
+    public static final MajorType MAP_TYPE = Types.repeated(MinorType.MAP);
+    public static final MajorType INT_TYPE = Types.optional(MinorType.INT);
+    public static final MajorType FLOAT_TYPE = Types.optional(MinorType.FLOAT4);
+    public static final MajorType NULL_TYPE = Types.optional(MinorType.LATE);
 
     public static MajorType getFieldType(JsonToken token) {
         switch(token) {
@@ -64,11 +65,11 @@ public class JacksonHelper {
         switch (fieldType) {
             case INT:
                 return parser.getIntValue();
-            case VARCHAR4:
+            case VARCHAR:
                 return parser.getValueAsString();
             case FLOAT4:
                 return parser.getFloatValue();
-            case BOOLEAN:
+            case BIT:
                 return parser.getBooleanValue();
             case LATE:
                 return null;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index 3a57410..f1b5538 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -35,11 +35,10 @@ import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.NullableBitVector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
 import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableVarChar4Vector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
-
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
@@ -367,20 +366,20 @@ public class JSONRecordReader implements RecordReader {
                     }
                     return holder.hasEnoughSpace(32);
                 }
-                case VARCHAR4: {
+                case VARCHAR: {
                     if (val == null) {
                         return (index + 1) * 4 <= holder.getLength();
                     } else {
                         byte[] bytes = ((String) val).getBytes(UTF_8);
                         int length = bytes.length;
                         holder.incAndCheckLength(length);
-                        NullableVarChar4Vector varLen4 = (NullableVarChar4Vector) holder.getValueVector();
-                        NullableVarChar4Vector.Mutator m = varLen4.getMutator();
+                        NullableVarCharVector varLen4 = (NullableVarCharVector) holder.getValueVector();
+                        NullableVarCharVector.Mutator m = varLen4.getMutator();
                         m.set(index, bytes);
                         return holder.hasEnoughSpace(length);
                     }
                 }
-                case BOOLEAN: {
+                case BIT: {
                     holder.incAndCheckLength(1);
                     NullableBitVector bit = (NullableBitVector) holder.getValueVector();
                     if (val != null) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index 512c13e..661ceba 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -29,7 +29,7 @@ abstract class BaseValueVector implements ValueVector{
     public void reset(){}
   }
   
-  abstract class BaseMutator implements NonRepeatedMutator{
+  abstract class BaseMutator implements Mutator{
     public void reset(){}
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 0f5b7d0..e81cc9b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -9,22 +9,22 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
+
 /**
- * Bit implements a vector of bit-width values.  Elements in the vector are accessed
- * by position from the logical start of the vector.
- *   The width of each element is 1 bit.
- *   The equivalent Java primitive is an int containing the value '0' or '1'.
- *
+ * Bit implements a vector of bit-width values. Elements in the vector are accessed by position from the logical start
+ * of the vector. The width of each element is 1 bit. The equivalent Java primitive is an int containing the value '0'
+ * or '1'.
+ * 
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
-public final class BitVector extends BaseDataValueVector implements FixedWidthVector{
+public final class BitVector extends BaseDataValueVector implements FixedWidthVector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
 
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
 
   private int valueCapacity;
-  
+
   public BitVector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
   }
@@ -32,11 +32,12 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
   private int getSizeFromCount(int valueCount) {
     return (int) Math.ceil(valueCount / 8);
   }
-  
+
   /**
-   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-   *
-   * @param valueCount  The number of values which can be contained within this vector.
+   * Allocate a new memory space for this vector. Must be called prior to using the ValueVector.
+   * 
+   * @param valueCount
+   *          The number of values which can be contained within this vector.
    */
   public void allocateNew(int valueCount) {
     clear();
@@ -47,9 +48,9 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       data.setByte(i, 0);
     }
   }
-  
+
   @Override
-  public int load(int valueCount, ByteBuf buf){
+  public int load(int valueCount, ByteBuf buf) {
     clear();
     this.valueCount = valueCount;
     int len = getSizeFromCount(valueCount);
@@ -57,18 +58,18 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
     data.retain();
     return len;
   }
-  
-  public void copyValue(int inIndex, int outIndex, BitVector target){
+
+  public void copyValue(int inIndex, int outIndex, BitVector target) {
     target.mutator.set(outIndex, this.accessor.get(inIndex));
   }
-  
+
   @Override
   public void load(FieldMetadata metadata, ByteBuf buffer) {
     assert this.field.getDef().equals(metadata.getDef());
     int loaded = load(metadata.getValueCount(), buffer);
     assert metadata.getBufferLength() == loaded;
   }
-  
+
   @Override
   public int getValueCapacity() {
     return valueCapacity;
@@ -78,109 +79,123 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
     return new Mutator();
   }
 
-  public Accessor getAccessor(){
+  public Accessor getAccessor() {
     return new Accessor();
   }
-  
-  public TransferPair getTransferPair(){
+
+  public TransferPair getTransferPair() {
     return new TransferImpl();
   }
-  
-  public void transferTo(BitVector target){
+
+  public void transferTo(BitVector target) {
     target.data = data;
     target.data.retain();
     target.valueCount = valueCount;
     clear();
   }
-  
-  private class TransferImpl implements TransferPair{
+
+  private class TransferImpl implements TransferPair {
     BitVector to;
-    
-    public TransferImpl(){
+
+    public TransferImpl() {
       this.to = new BitVector(getField(), allocator);
     }
-    
-    public BitVector getTo(){
+
+    public BitVector getTo() {
       return to;
     }
-    
-    public void transfer(){
+
+    public void transfer() {
       transferTo(to);
     }
   }
-  
-  public class Accessor extends BaseAccessor{
+
+  public class Accessor extends BaseAccessor {
 
     /**
-     * Get the byte holding the desired bit, then mask all other bits.  Iff the result is 0, the
-     * bit was not set.
-     *
-     * @param  index   position of the bit in the vector
+     * Get the byte holding the desired bit, then mask all other bits. Iff the result is 0, the bit was not set.
+     * 
+     * @param index
+     *          position of the bit in the vector
      * @return 1 if set, otherwise 0
      */
-    public int get(int index) {
+    public final int get(int index) {
       // logger.debug("BIT GET: index: {}, byte: {}, mask: {}, masked byte: {}",
-      //             index,
-      //             data.getByte((int)Math.floor(index/8)),
-      //             (int)Math.pow(2, (index % 8)),
-      //             data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
-      return ((data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8))) == 0) ? 0 : 1;
+      // index,
+      // data.getByte((int)Math.floor(index/8)),
+      // (int)Math.pow(2, (index % 8)),
+      // data.getByte((int)Math.floor(index/8)) & (int)Math.pow(2, (index % 8)));
+      return ((data.getByte((int) Math.floor(index / 8)) & (int) Math.pow(2, (index % 8))) == 0) ? 0 : 1;
     }
-    
+
     @Override
-    public Object getObject(int index) {
+    public final Object getObject(int index) {
       return new Boolean(get(index) != 0);
     }
-    
-    public int getValueCount() {
+
+    public final int getValueCount() {
       return valueCount;
     }
-    
+
+    public final void get(int index, BitHolder holder) {
+      holder.value = get(index);
+    }
+
+    final void get(int index, NullableBitHolder holder) {
+      holder.value = get(index);
+    }
   }
-  
+
   /**
-   * MutableBit implements a vector of bit-width values.  Elements in the vector are accessed
-   * by position from the logical start of the vector.  Values should be pushed onto the vector
-   * sequentially, but may be randomly accessed.
-   *
+   * MutableBit implements a vector of bit-width values. Elements in the vector are accessed by position from the
+   * logical start of the vector. Values should be pushed onto the vector sequentially, but may be randomly accessed.
+   * 
    * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
    */
-  public class Mutator extends BaseMutator{
+  public class Mutator extends BaseMutator {
+
+    private Mutator() {
+    }
 
-    private Mutator(){}
-    
     /**
      * Set the bit at the given index to the specified value.
-     *
-     * @param index   position of the bit to set
-     * @param value   value to set (either 1 or 0)
+     * 
+     * @param index
+     *          position of the bit to set
+     * @param value
+     *          value to set (either 1 or 0)
      */
-    public void set(int index, int value) {
-      byte currentByte = data.getByte((int)Math.floor(index/8));
+    public final void set(int index, int value) {
+      byte currentByte = data.getByte((int) Math.floor(index / 8));
       if (value != 0) {
         // true
         currentByte |= (byte) Math.pow(2, (index % 8));
-      }
-      else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte) Math.pow(2, (index % 8))) {
+      } else if ((currentByte & (byte) Math.pow(2, (index % 8))) == (byte) Math.pow(2, (index % 8))) {
         // false, and bit was previously set
         currentByte -= (byte) Math.pow(2, (index % 8));
       }
-      data.setByte((int) Math.floor(index/8), currentByte);
+      data.setByte((int) Math.floor(index / 8), currentByte);
+    }
+
+    public final void set(int index, BitHolder holder) {
+      set(index, holder.value);
+    }
+
+    final void set(int index, NullableBitHolder holder) {
+      set(index, holder.value);
     }
 
-    public void setValueCount(int valueCount) {
+    public final void setValueCount(int valueCount) {
       BitVector.this.valueCount = valueCount;
       data.writerIndex(getSizeFromCount(valueCount));
     }
 
     @Override
-    public void randomizeData() {
-      if (data != DeadBuf.DEAD_BUFFER) {
-        Random r = new Random();
-        for (int i = 0; i < data.capacity() - 1; i++) {
-          byte[] bytes = new byte[1];
-          r.nextBytes(bytes);
-          data.setByte(i, bytes[0]);
+    public final void generateTestData() {
+      boolean even = true;
+      for (int i = 0; i < valueCount; i++, even = !even) {
+        if (even) {
+          set(i, 1);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
index 008d17d..17e072b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -21,5 +21,5 @@ public interface FixedWidthVector extends ValueVector{
   public int load(int valueCount, ByteBuf buf);
   
   
-  public abstract NonRepeatedMutator getMutator();
+  public abstract Mutator getMutator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
deleted file mode 100644
index eba5e2e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/NonRepeatedMutator.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package org.apache.drill.exec.vector;
-
-import org.apache.drill.exec.vector.ValueVector.Mutator;
-
-public interface NonRepeatedMutator extends Mutator{
-  public void setValueCount(int valueCount);
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index 9328a80..717c087 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -26,33 +26,52 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 
 /**
- * ValueVectorTypes defines a set of template-generated classes which implement type-specific
- * value vectors.  The template approach was chosen due to the lack of multiple inheritence.  It
- * is also important that all related logic be as efficient as possible.
+ * ValueVectorTypes defines a set of template-generated classes which implement type-specific value vectors. The
+ * template approach was chosen due to the lack of multiple inheritence. It is also important that all related logic be
+ * as efficient as possible.
  */
 public interface ValueVector extends Closeable {
 
-//  /**
-//   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-//   * calculate the size based on width and record count.
-//   */
-//  public abstract int getAllocatedSize();
-
-
   /**
-   * Alternative to clear().  Allows use as closeable in try-with-resources.
+   * Alternative to clear(). Allows use as closeable in try-with-resources.
    */
   public void close();
-  
+
   /**
    * Release the underlying ByteBuf and reset the ValueVector to empty.
    */
   public void clear();
-  
-  
+
+  /**
+   * Get information about how this field is materialized.
+   * 
+   * @return
+   */
+  public MaterializedField getField();
+
+  /**
+   * Get a transfer pair to allow transferring this vectors data between this vector and a destination vector of the same
+   * type. Will also generate a second instance of this vector class that is connected through the TransferPair.
+   * 
+   * @return 
+   */
   public TransferPair getTransferPair();
 
-  
+  /**
+   * Given the current buffer allocation, return the maximum number of values that this buffer can contain.
+   * 
+   * @return Maximum values buffer can contain. In the case of a Repeated field, this is the number of atoms, not
+   *         repeated groups.
+   */
+  public int getValueCapacity();
+
+  /**
+   * Get Accessor to read value vector data.
+   * 
+   * @return
+   */
+  public abstract Accessor getAccessor();
+
   /**
    * Return the underlying buffers associated with this vector. Note that this doesn't impact the
    * reference counts for this buffer so it only should be used for in-context access. Also note
@@ -69,19 +88,6 @@ public interface ValueVector extends Closeable {
    * @param buffer The buffer that contains the ValueVector.
    */
   public void load(FieldMetadata metadata, ByteBuf buffer);
-
-
-  /**
-   * Given the current buffer allocation, return the maximum number of values that this buffer can contain.
-   * @return Maximum values buffer can contain.  In the case of a Repeated field, this is the number of atoms, not repeated groups.
-   */
-  public int getValueCapacity();
-  
-  /**
-   * Get information about how this field is materialized.
-   * @return
-   */
-  public MaterializedField getField();
   
   /**
    * Get the metadata for this field.  Used in serialization
@@ -90,44 +96,43 @@ public interface ValueVector extends Closeable {
   public FieldMetadata getMetadata();
   
   /**
-   * Get Accessor to read value vector data.
-   * @return 
-   */
-  public abstract Accessor getAccessor();
-  
-  /**
    * Get a Mutator to update this vectors data.
+   * 
    * @return
    */
   public abstract Mutator getMutator();
 
-  
-  public interface Accessor{
+  public interface Accessor {
 
-//    /**
-//     * Get the number of records allocated for this value vector.
-//     * @return number of allocated records
-//     */
-//    public int getRecordCount();
+    // /**
+    // * Get the number of records allocated for this value vector.
+    // * @return number of allocated records
+    // */
+    // public int getRecordCount();
 
     /**
-     * Get the Java Object representation of the element at the specified position.  Useful for testing.
-     *
-     * @param index   Index of the value to get
+     * Get the Java Object representation of the element at the specified position. Useful for testing.
+     * 
+     * @param index
+     *          Index of the value to get
      */
     public abstract Object getObject(int index);
-    
+
     public int getValueCount();
+
     public void reset();
   }
-  
-  
-    
-  
-  
-  public interface Mutator{
+
+  public interface Mutator {
+    /**
+     * Set the top number values (optional/required) or number of value groupings (repeated) in this vector.
+     * 
+     * @param valueCount
+     */
+    public void setValueCount(int valueCount);
+
     public void reset();
-    public void randomizeData();
+
+    public void generateTestData();
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
index 8229870..86c7f0b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
@@ -27,5 +27,5 @@ public interface VariableWidthVector extends ValueVector{
    */
   public int load(int dataBytes, int valueCount, ByteBuf buf);
   
-  public abstract NonRepeatedMutator getMutator();
+  public abstract Mutator getMutator();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
index a360cea..2d6575b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
@@ -26,6 +26,6 @@ message NamePart {
 message FieldDef {
   repeated NamePart name = 1; // multipart description of entire field name
   optional common.MajorType major_type = 2; // the type associated with this field.
-  repeated FieldDef field = 3; // only in the cases of type == MAP or REPEATMAP
+  //repeated FieldDef field = 3; // only in the cases of type == MAP or REPEATMAP
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
index 5643c0f..7e2506c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
@@ -43,4 +43,5 @@ message FieldMetadata {
   optional int32 var_byte_length = 3;
   optional int32 group_count = 4; // number of groups.  (number of repeated records)
   optional int32 buffer_length = 5;
+  repeated FieldMetadata child = 6;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
index c610374..78f3731 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/expr/ExpressionTest.java
@@ -18,6 +18,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -39,8 +40,7 @@ public class ExpressionTest {
 
   @Test
   public void testSpecial(final @Injectable RecordBatch batch) throws Exception {
-    final TypedFieldId tfid = new TypedFieldId(MajorType.newBuilder().setMode(DataMode.OPTIONAL)
-        .setMinorType(MinorType.INT).build(), 0);
+    final TypedFieldId tfid = new TypedFieldId(Types.optional(MinorType.INT),0);
 
     new NonStrictExpectations() {
       {
@@ -56,8 +56,7 @@ public class ExpressionTest {
 
   @Test
   public void testSchemaExpression(final @Injectable RecordBatch batch) throws Exception {
-    final TypedFieldId tfid = new TypedFieldId(MajorType.newBuilder().setMode(DataMode.OPTIONAL)
-        .setMinorType(MinorType.BIGINT).build(), 0);
+    final TypedFieldId tfid = new TypedFieldId(Types.optional(MinorType.BIGINT), 0);
 
     new Expectations() {
       {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
index 68b8881..dae5f78 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
@@ -38,7 +38,7 @@ public class ExpressionTreeMaterializerTest {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionTreeMaterializerTest.class);
 
-  final MajorType boolConstant = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BOOLEAN).build();
+  final MajorType boolConstant = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BIT).build();
   final MajorType bigIntType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.BIGINT).build();
   final MajorType intType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.INT).build();
 
@@ -81,7 +81,7 @@ public class ExpressionTreeMaterializerTest {
   public void testMaterializingLateboundTree(final @Injectable RecordBatch batch) throws SchemaChangeException {
     new NonStrictExpectations() {
       {
-        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BOOLEAN), -4);
+        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIT), -4);
         batch.getValueVectorId(new FieldReference("test1", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
       }
     };
@@ -190,7 +190,7 @@ public class ExpressionTreeMaterializerTest {
           public String[] getArgumentNamesByPosition() {
             return new String[0];
           }
-        }, OutputTypeDeterminer.FIXED_BOOLEAN), Lists.newArrayList((LogicalExpression) new FieldReference("test",
+        }, OutputTypeDeterminer.FIXED_BIT), Lists.newArrayList((LogicalExpression) new FieldReference("test",
         ExpressionPosition.UNKNOWN)), ExpressionPosition.UNKNOWN);
     LogicalExpression newExpr = ExpressionTreeMaterializer.materialize(functionCallExpr, batch, ec);
     assertTrue(newExpr instanceof FunctionCall);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 0fed756..08f118c 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -13,7 +13,7 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
 import org.apache.drill.exec.vector.NullableUInt4Vector;
-import org.apache.drill.exec.vector.NullableVarChar2Vector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.junit.Test;
@@ -62,7 +62,7 @@ public class TestValueVector {
     MajorType.Builder typeBuilder = MajorType.newBuilder();
     FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(MinorType.VARCHAR2)
+        .setMinorType(MinorType.VARCHAR)
         .setMode(DataMode.OPTIONAL)
         .setWidth(2);
     defBuilder
@@ -70,8 +70,8 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    NullableVarChar2Vector v = new NullableVarChar2Vector(field, allocator);
-    NullableVarChar2Vector.Mutator m = v.getMutator();
+    NullableVarCharVector v = new NullableVarCharVector(field, allocator);
+    NullableVarCharVector.Mutator m = v.getMutator();
     v.allocateNew(1024*10, 1024);
 
     // Create and set 3 sample strings
@@ -254,7 +254,7 @@ public class TestValueVector {
     MajorType.Builder typeBuilder = MajorType.newBuilder();
     FieldDef.Builder defBuilder = FieldDef.newBuilder();
     typeBuilder
-        .setMinorType(MinorType.BOOLEAN)
+        .setMinorType(MinorType.BIT)
         .setMode(DataMode.REQUIRED)
         .setWidth(4);
     defBuilder

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index 4a0358e..661b029 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -96,11 +96,11 @@ public class JSONRecordReaderTest {
     assertEquals(2, jr.next());
     assertEquals(3, addFields.size());
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
-    assertField(addFields.get(1), 0, MinorType.BOOLEAN, true, "b");
-    assertField(addFields.get(2), 0, MinorType.VARCHAR4, "hi!".getBytes(UTF_8), "c");
+    assertField(addFields.get(1), 0, MinorType.BIT, true, "b");
+    assertField(addFields.get(2), 0, MinorType.VARCHAR, "hi!".getBytes(UTF_8), "c");
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
-    assertField(addFields.get(1), 1, MinorType.BOOLEAN, false, "b");
-    assertField(addFields.get(2), 1, MinorType.VARCHAR4, "drill!".getBytes(UTF_8), "c");
+    assertField(addFields.get(1), 1, MinorType.BIT, false, "b");
+    assertField(addFields.get(2), 1, MinorType.VARCHAR, "drill!".getBytes(UTF_8), "c");
 
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());
@@ -126,20 +126,20 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
     assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
-    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+    assertField(addFields.get(3), 0, MinorType.BIT, true, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR, "test1".getBytes(UTF_8), "str1");
 
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 1, MinorType.INT, 3, "b");
-    assertField(addFields.get(3), 1, MinorType.BOOLEAN, false, "bool");
-    assertField(addFields.get(4), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+    assertField(addFields.get(3), 1, MinorType.BIT, false, "bool");
+    assertField(addFields.get(4), 1, MinorType.VARCHAR, "test2".getBytes(UTF_8), "str1");
     assertField(addFields.get(5), 1, MinorType.INT, 4, "d");
 
     assertField(addFields.get(0), 2, MinorType.INT, 12345, "test");
     assertField(addFields.get(2), 2, MinorType.FLOAT4, (float) 5.16, "c");
-    assertField(addFields.get(3), 2, MinorType.BOOLEAN, true, "bool");
+    assertField(addFields.get(3), 2, MinorType.BIT, true, "bool");
     assertField(addFields.get(5), 2, MinorType.INT, 6, "d");
-    assertField(addFields.get(6), 2, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+    assertField(addFields.get(6), 2, MinorType.VARCHAR, "test3".getBytes(UTF_8), "str2");
     assertTrue(mutator.getRemovedFields().isEmpty());
     assertEquals(0, jr.next());
   }
@@ -166,15 +166,15 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 1, "b");
     assertField(addFields.get(2), 0, MinorType.FLOAT4, (float) 2.15, "c");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
-    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test1".getBytes(UTF_8), "str1");
+    assertField(addFields.get(3), 0, MinorType.BIT, true, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR, "test1".getBytes(UTF_8), "str1");
     assertTrue(removedFields.isEmpty());
     assertEquals(1, jr.next());
     assertEquals(6, addFields.size());
     assertField(addFields.get(0), 0, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 0, MinorType.INT, 3, "b");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, false, "bool");
-    assertField(addFields.get(4), 0, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "str1");
+    assertField(addFields.get(3), 0, MinorType.BIT, false, "bool");
+    assertField(addFields.get(4), 0, MinorType.VARCHAR, "test2".getBytes(UTF_8), "str1");
     assertField(addFields.get(5), 0, MinorType.INT, 4, "d");
     assertEquals(1, removedFields.size());
     //assertEquals(3, (int) removedFields.get(0));
@@ -182,10 +182,10 @@ public class JSONRecordReaderTest {
     assertEquals(1, jr.next());
     assertEquals(8, addFields.size()); // The reappearing of field 'c' is also included
     assertField(addFields.get(0), 0, MinorType.INT, 12345, "test");
-    assertField(addFields.get(3), 0, MinorType.BOOLEAN, true, "bool");
+    assertField(addFields.get(3), 0, MinorType.BIT, true, "bool");
     assertField(addFields.get(5), 0, MinorType.INT, 6, "d");
     assertField(addFields.get(6), 0, MinorType.FLOAT4, (float) 5.16, "c");
-    assertField(addFields.get(7), 0, MinorType.VARCHAR4, "test3".getBytes(UTF_8), "str2");
+    assertField(addFields.get(7), 0, MinorType.VARCHAR, "test3".getBytes(UTF_8), "str2");
     assertEquals(2, removedFields.size());
 //    assertTrue(removedFields.contains(5));
 //    assertTrue(removedFields.contains(2));
@@ -210,14 +210,14 @@ public class JSONRecordReaderTest {
     assertEquals(5, addFields.size());
     assertField(addFields.get(0), 0, MinorType.INT, 123, "test");
     assertField(addFields.get(1), 0, MinorType.MAP, null, "a");
-    assertField(addFields.get(2), 0, MinorType.VARCHAR4, "test".getBytes(UTF_8), "b");
+    assertField(addFields.get(2), 0, MinorType.VARCHAR, "test".getBytes(UTF_8), "b");
     assertField(addFields.get(3), 0, MinorType.MAP, null, "a");
-    assertField(addFields.get(4), 0, MinorType.BOOLEAN, true, "d");
+    assertField(addFields.get(4), 0, MinorType.BIT, true, "d");
     assertField(addFields.get(0), 1, MinorType.INT, 1234, "test");
     assertField(addFields.get(1), 1, MinorType.MAP, null, "a");
-    assertField(addFields.get(2), 1, MinorType.VARCHAR4, "test2".getBytes(UTF_8), "b");
+    assertField(addFields.get(2), 1, MinorType.VARCHAR, "test2".getBytes(UTF_8), "b");
     assertField(addFields.get(3), 1, MinorType.MAP, null, "a");
-    assertField(addFields.get(4), 1, MinorType.BOOLEAN, true, "d");
+    assertField(addFields.get(4), 1, MinorType.BIT, true, "d");
 
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
index 5b6c19b..90ba2c1 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/scan_screen_logical.json
@@ -20,11 +20,8 @@
 
                     {
                         name : "superhero_name",
-                        type : "VARCHAR2",
-                        mode : "REQUIRED",
-                        width : 100,
-                        precision : 1,
-                        scale : 2
+                        type : "VARCHAR",
+                        mode : "REQUIRED"
 
                     }
              ]

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
index e2160b5..8b9c2f6 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/rops/CollapsingAggregateROP.java
@@ -159,7 +159,7 @@ public class CollapsingAggregateROP extends SingleInputROPBase<CollapsingAggrega
       // if we're in target mode and this row matches the target criteria, we're going to copy carry over values and mark foundTarget = true.
       if(targetMode){
         DataValue v = targetEvaluator.eval();
-        if(v.getDataType().getMinorType() == MinorType.BOOLEAN && v.getAsBooleanValue().getBoolean()){
+        if(v.getDataType().getMinorType() == MinorType.BIT && v.getAsBooleanValue().getBoolean()){
           foundTarget = true;
           for(int i =0 ; i < carryovers.length; i++){
             carryoverValues[i] = carryovers[i].eval();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
index 7a72b2c..201c8fc 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ScalarValues.java
@@ -42,7 +42,7 @@ public final class ScalarValues {
   
   public static class StringScalar extends BaseDataValue implements StringValue, ComparableValue, BasicEvaluator {
     private CharSequence seq;
-    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARCHAR4).setMode(DataMode.OPTIONAL).build();
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARCHAR).setMode(DataMode.OPTIONAL).build();
     
     public StringScalar(CharSequence seq){
       this.seq = seq;
@@ -123,7 +123,7 @@ public final class ScalarValues {
   
   public static class BooleanScalar extends BaseDataValue implements BooleanValue, BasicEvaluator{
     private boolean b;
-    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.OPTIONAL).build();
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.BIT).setMode(DataMode.OPTIONAL).build();
     
     public BooleanScalar(boolean b){
       this.b = b;
@@ -441,7 +441,7 @@ public final class ScalarValues {
   }
   
   public static class BytesScalar extends BaseDataValue implements BytesValue{
-    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARBINARY4).setMode(DataMode.OPTIONAL).build();
+    private MajorType type = MajorType.newBuilder().setMinorType(MinorType.VARBINARY).setMode(DataMode.OPTIONAL).build();
     private BytesWritable.Comparator comparator = new BytesWritable.Comparator();
     private final static HashFunction HASH = Hashing.murmur3_32();
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
index 3a6be51..71365c3 100644
--- a/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
+++ b/sandbox/prototype/exec/ref/src/main/java/org/apache/drill/exec/ref/values/ValueReader.java
@@ -26,7 +26,7 @@ public class ValueReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueReader.class);
   
   public static boolean getBoolean(DataValue v){
-    if(v.getDataType().getMinorType() == MinorType.BOOLEAN && v.getDataType().getMode() != DataMode.REPEATED){
+    if(v.getDataType().getMinorType() == MinorType.BIT && v.getDataType().getMode() != DataMode.REPEATED){
       return v.getAsBooleanValue().getBoolean();
     }else{
       throw new DrillRuntimeException(String.format("Unable to get boolean.  Type was a %s", v.getClass().getCanonicalName()));


[2/2] git commit: Clean up types, remove extraneous types, update BOOLEAN/BIT to be consistently BIT Implement ValueHolders for all types through code generation, remove manually coded value holders. Rename ValueVector.randomizeData() to ValueVector.gene

Posted by ja...@apache.org.
Clean up types, remove extraneous types, update BOOLEAN/BIT to be consistently BIT
Implement ValueHolders for all types through code generation, remove manually coded value holders.
Rename ValueVector.randomizeData() to ValueVector.generateTestData() and provide consistent return results for repeatability and data validation.


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

Branch: refs/heads/master
Commit: 742f4c119bccfe832672bc39998d8df6481d18e4
Parents: 9f69ed0
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jul 30 17:58:10 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue Jul 30 17:58:10 2013 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/expression/Arg.java |   5 +-
 .../common/expression/ExpressionValidator.java  |   2 +-
 .../common/expression/OutputTypeDeterminer.java |   3 +-
 .../common/expression/ValueExpressions.java     |  11 +-
 .../common/expression/fn/BooleanFunctions.java  |  14 +-
 .../common/expression/fn/StringFunctions.java   |   4 +-
 .../common/expression/fn/TypeFunctions.java     |   2 +-
 .../common/expression/fn/UnaryFunctions.java    |   4 +-
 .../org/apache/drill/common/types/Types.java    |  27 ++--
 .../common/src/main/protobuf/Types.proto        |  24 +--
 .../ValueVectors/data/ValueVectorTypes.tdd      |  58 +++----
 .../templates/FixedValueVectors.java            |  70 +++++++--
 .../templates/NullableValueVectors.java         |  54 +++++--
 .../templates/RepeatedValueVectors.java         | 101 ++++++------
 .../ValueVectors/templates/TypeHelper.java      |  82 +++++-----
 .../ValueVectors/templates/ValueHolders.java    |  64 ++++++++
 .../templates/VariableLengthVectors.java        |  42 ++++-
 .../drill/exec/compile/CodeModelTools.java      |  31 ----
 .../apache/drill/exec/expr/CodeGenerator.java   |  40 +----
 .../drill/exec/expr/fn/impl/Alternator.java     |   4 +-
 .../drill/exec/expr/fn/impl/MathFunctions.java  |  17 +-
 .../drill/exec/expr/holders/BooleanHolder.java  |  10 --
 .../drill/exec/expr/holders/Float8Holder.java   |  14 --
 .../drill/exec/expr/holders/IntHolder.java      |  10 --
 .../drill/exec/expr/holders/LongHolder.java     |  10 --
 .../expr/holders/NullableBooleanHolder.java     |  11 --
 .../exec/expr/holders/NullableFloat8Holder.java |  13 --
 .../exec/expr/holders/NullableIntHolder.java    |  11 --
 .../exec/expr/holders/NullableLongHolder.java   |  11 --
 .../exec/expr/holders/VarBinaryHolder.java      |  12 --
 .../drill/exec/memory/BufferAllocator.java      |  32 ++++
 .../exec/memory/DirectBufferAllocator.java      |  10 ++
 .../drill/exec/physical/base/Exchange.java      |  11 ++
 .../physical/config/HashToRandomExchange.java   |   5 +
 .../exec/physical/config/MockRecordReader.java  |  17 +-
 .../exec/physical/config/UnionExchange.java     |   5 +
 .../physical/impl/filter/FilterRecordBatch.java |  21 +--
 .../physical/impl/filter/FilterTemplate.java    |   2 +-
 .../impl/project/ProjectRecordBatch.java        |  11 +-
 .../impl/project/ProjectorTemplate.java         |   2 +-
 .../impl/svremover/RemovingRecordBatch.java     |  11 +-
 .../apache/drill/exec/record/BatchSchema.java   |  39 ++++-
 .../drill/exec/record/MaterializedField.java    |  27 ++++
 .../drill/exec/record/NullExpression.java       |   3 +-
 .../apache/drill/exec/record/WritableBatch.java |   2 +-
 .../exec/record/selection/SelectionVector2.java |   7 +
 .../exec/schema/json/jackson/JacksonHelper.java |  19 +--
 .../drill/exec/store/JSONRecordReader.java      |  11 +-
 .../drill/exec/vector/BaseValueVector.java      |   2 +-
 .../org/apache/drill/exec/vector/BitVector.java | 157 ++++++++++---------
 .../drill/exec/vector/FixedWidthVector.java     |   2 +-
 .../drill/exec/vector/NonRepeatedMutator.java   |   7 -
 .../apache/drill/exec/vector/ValueVector.java   | 111 ++++++-------
 .../drill/exec/vector/VariableWidthVector.java  |   2 +-
 .../java-exec/src/main/protobuf/SchemaDef.proto |   2 +-
 .../src/main/protobuf/UserBitShared.proto       |   1 +
 .../apache/drill/exec/expr/ExpressionTest.java  |   7 +-
 .../record/ExpressionTreeMaterializerTest.java  |   6 +-
 .../exec/record/vector/TestValueVector.java     |  10 +-
 .../drill/exec/store/JSONRecordReaderTest.java  |  40 ++---
 .../src/test/resources/scan_screen_logical.json |   7 +-
 .../exec/ref/rops/CollapsingAggregateROP.java   |   2 +-
 .../drill/exec/ref/values/ScalarValues.java     |   6 +-
 .../drill/exec/ref/values/ValueReader.java      |   2 +-
 64 files changed, 720 insertions(+), 640 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
index 64ec351..c8d847a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/Arg.java
@@ -20,6 +20,7 @@ package org.apache.drill.common.expression;
 import java.util.List;
 
 import org.apache.drill.common.expression.visitors.ConstantChecker;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -84,8 +85,8 @@ public class Arg {
   private static MajorType[] getMajorTypes(boolean allowNulls, MinorType... types){
     List<MajorType> mts = Lists.newArrayList();
     for(MinorType t : types){
-      if(allowNulls) mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.OPTIONAL).build());
-      mts.add(MajorType.newBuilder().setMinorType(t).setMode(DataMode.REQUIRED).build());
+      if(allowNulls) mts.add(Types.optional(t));
+      mts.add(Types.required(t));
     }
     return mts.toArray(new MajorType[mts.size()]);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
index 98a9075..b3a034b 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ExpressionValidator.java
@@ -26,7 +26,7 @@ public class ExpressionValidator implements ExprVisitor<Void, ErrorCollector, Ru
     int i = 0;
     for (IfCondition c : ifExpr.conditions) {
       MajorType mt = c.condition.getMajorType();
-      if (mt.getMode() != DataMode.REQUIRED || mt.getMinorType() != MinorType.BOOLEAN){
+      if (mt.getMode() != DataMode.REQUIRED || mt.getMinorType() != MinorType.BIT){
         errors.addGeneralError(c.condition.getPosition(),String.format(
                         "Failure composing If Expression.  All conditions must return a required value and be of type boolean.  Condition %d was DatMode %s and Type %s.",
                         i, mt.getMode(), mt.getMinorType()));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
index 0bec184..828f2b6 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/OutputTypeDeterminer.java
@@ -22,10 +22,11 @@ import java.util.List;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 
 public interface OutputTypeDeterminer {
 
-  public static OutputTypeDeterminer FIXED_BOOLEAN = new FixedType(MajorType.newBuilder().setMinorType(MinorType.BOOLEAN).setMode(DataMode.REQUIRED).build());
+  public static OutputTypeDeterminer FIXED_BIT = new FixedType(Types.required(MinorType.BIT));
   
   public MajorType getOutputType(List<LogicalExpression> expressions);
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 1acffcb..d613ebe 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -71,7 +71,7 @@ public class ValueExpressions {
 
     @Override
     public MajorType getMajorType() {
-      return Types.REQUIRED_BOOLEAN;
+      return Types.REQUIRED_BIT;
     }
 
     @Override
@@ -88,8 +88,7 @@ public class ValueExpressions {
   public static class DoubleExpression extends LogicalExpressionBase {
     private double d;
 
-    private static final MajorType DOUBLE_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.FLOAT8)
-        .setMode(DataMode.REQUIRED).build();
+    private static final MajorType DOUBLE_CONSTANT = Types.required(MinorType.FLOAT8);
 
     public DoubleExpression(double d, ExpressionPosition pos) {
       super(pos);
@@ -114,8 +113,7 @@ public class ValueExpressions {
 
   public static class LongExpression extends LogicalExpressionBase {
 
-    private static final MajorType LONG_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.BIGINT)
-        .setMode(DataMode.REQUIRED).build();
+    private static final MajorType LONG_CONSTANT = Types.required(MinorType.BIGINT);
 
     private long l;
 
@@ -141,8 +139,7 @@ public class ValueExpressions {
 
   public static class QuotedString extends ValueExpression<String> {
 
-    private static final MajorType QUOTED_STRING_CONSTANT = MajorType.newBuilder().setMinorType(MinorType.VARCHAR2)
-        .setMode(DataMode.REQUIRED).build();
+    private static final MajorType QUOTED_STRING_CONSTANT = Types.required(MinorType.VARCHAR);
 
     public QuotedString(String value, ExpressionPosition pos) {
       super(value, pos);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
index 65d4c76..13bc051 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/BooleanFunctions.java
@@ -32,13 +32,13 @@ public class BooleanFunctions implements CallProvider {
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[] {
-        FunctionDefinition.operator("or", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "or", "||"),
-        FunctionDefinition.operator("and", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BOOLEAN), OutputTypeDeterminer.FIXED_BOOLEAN, "and", "&&"),
-        FunctionDefinition.operator("greater than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, ">"),
-        FunctionDefinition.operator("less than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, "<"),
-        FunctionDefinition.operator("equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, "==", "<>"),
-        FunctionDefinition.operator("greater than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, ">="),
-        FunctionDefinition.operator("less than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BOOLEAN, "<="), };
+        FunctionDefinition.operator("or", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BIT), OutputTypeDeterminer.FIXED_BIT, "or", "||"),
+        FunctionDefinition.operator("and", new AllowedTypeList(2, Integer.MAX_VALUE, Types.REQUIRED_BIT), OutputTypeDeterminer.FIXED_BIT, "and", "&&"),
+        FunctionDefinition.operator("greater than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, ">"),
+        FunctionDefinition.operator("less than", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "<"),
+        FunctionDefinition.operator("equal", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "==", "<>"),
+        FunctionDefinition.operator("greater than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, ">="),
+        FunctionDefinition.operator("less than or equal to", new ComparableArguments(2), OutputTypeDeterminer.FIXED_BIT, "<="), };
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
index 565843f..70f6eae 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/StringFunctions.java
@@ -31,8 +31,8 @@ public class StringFunctions implements CallProvider{
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
         FunctionDefinition.simple("regex_like", new BasicArgumentValidator( //
-            new Arg(true, false, "pattern", MinorType.VARCHAR2, MinorType.VARCHAR4), //
-            new Arg(false, true, "value", MinorType.FIXEDCHAR, MinorType.VARCHAR2, MinorType.VARCHAR4) ), FixedType.FIXED_BOOLEAN),
+            new Arg(true, false, "pattern", MinorType.VARCHAR, MinorType.VARCHAR), //
+            new Arg(false, true, "value", MinorType.FIXEDCHAR, MinorType.VARCHAR, MinorType.VARCHAR) ), FixedType.FIXED_BIT),
     };
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/TypeFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/TypeFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/TypeFunctions.java
index 726a63a..77df207 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/TypeFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/TypeFunctions.java
@@ -27,7 +27,7 @@ public class TypeFunctions implements CallProvider{
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
-        FunctionDefinition.simple("isNumber", new AnyTypeAllowed(1), OutputTypeDeterminer.FIXED_BOOLEAN),
+        FunctionDefinition.simple("isNumber", new AnyTypeAllowed(1), OutputTypeDeterminer.FIXED_BIT),
     };
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
index 5570d10..8b21d71 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/fn/UnaryFunctions.java
@@ -29,8 +29,8 @@ public class UnaryFunctions implements CallProvider{
   @Override
   public FunctionDefinition[] getFunctionDefintions() {
     return new FunctionDefinition[]{
-        FunctionDefinition.operator("isNull", new ArgumentValidators.AnyTypeAllowed(1), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BOOLEAN)),
-        FunctionDefinition.operator("not", new BasicArgumentValidator(Types.REQUIRED_BOOLEAN), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BOOLEAN), "!"),
+        FunctionDefinition.operator("isNull", new ArgumentValidators.AnyTypeAllowed(1), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BIT)),
+        FunctionDefinition.operator("not", new BasicArgumentValidator(Types.REQUIRED_BIT), new OutputTypeDeterminer.FixedType(Types.REQUIRED_BIT), "!"),
         FunctionDefinition.operator("negative", new ArgumentValidators.NumericTypeAllowed(1, true), new OutputTypeDeterminer.SameAsFirstInput(), "u-"),
     };
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
index 757bed0..0490d8a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -9,7 +9,7 @@ public class Types {
   
   public static final MajorType NULL = required(MinorType.NULL);
   public static final MajorType LATE_BIND_TYPE = optional(MinorType.LATE);
-  public static final MajorType REQUIRED_BOOLEAN = required(MinorType.BOOLEAN);
+  public static final MajorType REQUIRED_BIT = required(MinorType.BIT);
   
   public static enum Comparability{
     UNKNOWN, NONE, EQUAL, ORDERED;
@@ -41,14 +41,9 @@ public class Types {
   
   public static boolean isFixedWidthType(MajorType type){
     switch(type.getMinorType()){
-    case MSGPACK2:
-    case MSGPACK4:
-    case PROTO2:
-    case PROTO4:
-    case VARBINARY2:
-    case VARBINARY4:
-    case VARCHAR2:
-    case VARCHAR4:
+    case VARBINARY:
+    case VAR16CHAR:
+    case VARCHAR:
       return false;
     default:
       return true;
@@ -60,8 +55,9 @@ public class Types {
     if(type.getMode() == DataMode.REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDCHAR:
-    case VARCHAR2:
-    case VARCHAR4:
+    case FIXED16CHAR:
+    case VARCHAR:
+    case VAR16CHAR:
       return true;
     default: 
       return false;
@@ -72,8 +68,7 @@ public class Types {
     if(type.getMode() == DataMode.REPEATED) return false;
     switch(type.getMinorType()){
     case FIXEDBINARY:
-    case VARBINARY2:
-    case VARBINARY4:
+    case VARBINARY:
       return true;
     default: 
       return false;
@@ -91,11 +86,7 @@ public class Types {
     case REPEATMAP:
       return Comparability.NONE;
     case INTERVAL:
-    case BOOLEAN:
-    case MSGPACK2:
-    case MSGPACK4:
-    case PROTO2:
-    case PROTO4:
+    case BIT:
       return Comparability.EQUAL;
     default:
       return Comparability.ORDERED;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/common/src/main/protobuf/Types.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Types.proto b/sandbox/prototype/common/src/main/protobuf/Types.proto
index 05a70c5..067072a 100644
--- a/sandbox/prototype/common/src/main/protobuf/Types.proto
+++ b/sandbox/prototype/common/src/main/protobuf/Types.proto
@@ -15,8 +15,8 @@ enum MinorType {
     BIGINT = 6;   //  eight byte signed integer
     DECIMAL4 = 7;   //  a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
     DECIMAL8 = 8;   //  a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
-    DECIMAL12 = 9;   //  a decimal supporting precision between 
-    DECIMAL16 = 10;   //  a decimal supporting precision between 19 and 37 (6 bits for decimal location, 1 sign)
+    DECIMAL12 = 9;   //  a decimal supporting precision between 19 and 28 (5 bits for decimal location, 1 sign)
+    DECIMAL16 = 10;   //  a decimal supporting precision between 29 and 37 (6 bits for decimal location, 1 sign)
     MONEY = 11;   //  signed decimal with two digit precision
     DATE = 12;   //  days since 4713bc 
     TIME = 13;   //  time in micros before or after 2000/1/1
@@ -26,21 +26,21 @@ enum MinorType {
     INTERVAL = 17;   //  TBD
     FLOAT4 = 18;   //  4 byte ieee 754 
     FLOAT8 = 19;   //  8 byte ieee 754
-    BOOLEAN = 20;   //  single bit value
+    BIT = 20;   //  single bit value (boolean)
     FIXEDCHAR = 21;   //  utf8 fixed length string, padded with spaces
-    VARCHAR2 = 23;   //  utf8 variable length string (up to 2^16 in length)
-    VARCHAR4 = 24;   //  utf8 variable length string (up to 2^32 in length)
-    FIXEDBINARY = 25;   //  fixed length binary, padded with 0 bytes
-    VARBINARY2 = 27;   //  variable length binary (up to 2^16 in length)
-    VARBINARY4 = 28;   //  variable length binary (up to 2^32 in length)
+    FIXED16CHAR = 22;
+    FIXEDBINARY = 23;   //  fixed length binary, padded with 0 bytes
+    VARCHAR = 24;   //  utf8 variable length string
+    VAR16CHAR = 25; // utf16 variable length string
+    VARBINARY = 26;   //  variable length binary
     UINT1 = 29;   //  unsigned 1 byte integer
     UINT2 = 30;   //  unsigned 2 byte integer
     UINT4 = 31;   //  unsigned 4 byte integer
     UINT8 = 32;   //  unsigned 8 byte integer
-    PROTO2 = 33;   //  protobuf encoded complex type. (up to 2^16 in length)
-    PROTO4 = 34;   //  protobuf encoded complex type. (up to 2^32 in length)
-    MSGPACK2 = 35;   //  msgpack encoded complex type. (up to 2^16 in length)
-    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
+//    PROTO2 = 33;   //  protobuf encoded complex type. (up to 2^16 in length)
+//    PROTO4 = 34;   //  protobuf encoded complex type. (up to 2^32 in length)
+//    MSGPACK2 = 35;   //  msgpack encoded complex type. (up to 2^16 in length)
+//    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
     NULL = 37; // a value of unknown type (e.g. a missing reference).
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
index 69cc277..d201671 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
@@ -1,62 +1,77 @@
 {
+  modes: [
+    {name: "Optional", prefix: "Nullable"},
+    {name: "Required", prefix: ""},
+    {name: "Repeated", prefix: "Repeated"}
+    ],
   types: [
     {
       major: "Fixed",
       width: 1,
       javaType: "byte",
+      boxedType: "Byte",
       minor: [
-        { class: "TinyInt" },
-        { class: "UInt1" }
+        { class: "TinyInt", valueHolder: "IntHolder"},
+        { class: "UInt1", valueHolder: "UInt1Holder"}
       ]
     },
     {
       major: "Fixed",
       width: 2,
       javaType: "char",
+      boxedType: "Character",
       minor: [
-        { class: "UInt2" }
+        { class: "UInt2", valueHolder: "UInt2Holder"}
       ]
     },    {
       major: "Fixed",
       width: 2,
       javaType: "short",
+      boxedType: "Short",
       minor: [
-        { class: "SmallInt" },
+        { class: "SmallInt", valueHolder: "Int2Holder"},
       ]
     },
     {
       major: "Fixed",
       width: 4,
       javaType: "int",
+      boxedType: "Integer",
       minor: [
-        { class: "Int" },
-        { class: "UInt4" },
-        { class: "Decimal4", maxPrecisionDigits: 8, scale: 1, javaType: "float"},
+        { class: "Int", valueHolder: "IntHolder" },
+        { class: "UInt4", valueHolder: "UInt4Holder" },
         { class: "Float4", javaType: "float" },
+        <#-- 
+        { class: "Decimal4", maxPrecisionDigits: 8, scale: 4, javaType: "float"},
         { class: "Date" }
+        -->
       ]
     },
     {
       major: "Fixed",
       width: 8,
       javaType: "long",
+      boxedType: "Long",
       minor: [
         { class: "BigInt" },
         { class: "UInt8" },
-        { class: "Decimal8", maxPrecisionDigits: 18, scale: 1, javaType: "double"},
-        { class: "Float8", javaType: "double" },
-        { class: "Money", maxPrecisionDigits: 2, scale: 1, javaType: "double" }
+        { class: "Float8", javaType: "double" , boxedType: "Double"},
+        <#-- 
+        { class: "Decimal8", maxPrecisionDigits: 18, scale: 4, javaType: "double", boxedType: "Double"},
+        { class: "Money", maxPrecisionDigits: 2, scale: 1, }, 
         { class: "Time" },
         { class: "TimeStamp" },
         { class: "DateTime" }
+        -->
       ]
     },
+    <#--
     {
       major: "Fixed",
       width: 12,
       javaType: "ByteBuf",
       minor: [
-        { class: "Decimal12", maxPrecisionDigits: 28, scale: 1},
+        { class: "Decimal12", maxPrecisionDigits: 28, scale: 5},
         { class: "TimeTZ" },
         { class: "Interval" }
       ]
@@ -66,29 +81,18 @@
       width: 16,
       javaType: "ByteBuf",
       minor: [
-        { class: "Decimal16", maxPrecisionDigits: 37, scale: 1}
-      ]
-    },
-    {
-      major: "VarLen",
-      width: 2,
-      javaType: "short",
-      minor: [
-        { class: "VarBinary2" },
-        { class: "VarChar2" },
-        { class: "Proto2" },
-        { class: "MsgPack2" }
+        { class: "Decimal16", maxPrecisionDigits: 37, scale: 6}
       ]
     },
+    -->
     {
       major: "VarLen",
       width: 4,
       javaType: "int",
       minor: [
-        { class: "VarBinary4" },
-        { class: "VarChar4" }
-        { class: "Proto4" },
-        { class: "MsgPack4" }
+        { class: "VarBinary" },
+        { class: "VarChar" }
+        { class: "Var16Char" }
       ]
     },
     {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
index 61515b0..6ffa968 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -14,10 +14,11 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.BaseValueVector;
-import org.apache.drill.exec.vector.MsgPack2Vector;
 
+import java.util.Arrays;
 import java.util.Random;
 
+
 /**
  * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
  * by position, starting from the logical start of the vector.  Values should be pushed onto the
@@ -141,6 +142,16 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       return dst;
     }
 
+    public void get(int index, ${minor.class}Holder holder){
+      holder.buffer = data;
+      holder.start = index * ${type.width};
+    }
+    
+    void get(int index, Nullable${minor.class}Holder holder){
+      holder.buffer = data;
+      holder.start = index * ${type.width};
+    }
+
     @Override
     public Object getObject(int index) {
       ByteBuf dst = allocator.buffer(${type.width});
@@ -157,6 +168,14 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public Object getObject(int index) {
       return get(index);
     }
+    
+    public void get(int index, ${minor.class}Holder holder){
+      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+    }
+
+    void get(int index, Nullable${minor.class}Holder holder){
+      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+    }
 
 
    </#if> <#-- type.width -->
@@ -186,14 +205,22 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
      data.setBytes(index * ${type.width}, value);
    }
    
+   public void set(int index, ${minor.class}Holder holder){
+     data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
+   }
+   
+   void set(int index, Nullable${minor.class}Holder holder){
+     data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
+   }
+   
    @Override
-   public void randomizeData() {
-     if (data != DeadBuf.DEAD_BUFFER) {
-       Random r = new Random();
-       for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
-         byte[] bytes = new byte[${type.width}];
-         r.nextBytes(bytes);
-         data.setByte(i, bytes[0]);
+   public void generateTestData() {
+     setValueCount(getValueCapacity());
+     boolean even = true;
+     for(int i =0; i < valueCount; i++, even = !even){
+       byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
+       for(int w = 0; w < ${type.width}; w++){
+         data.setByte(i + w, b);
        }
      }
    }
@@ -203,20 +230,31 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
    }
    
+   public void set(int index, ${minor.class}Holder holder){
+     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
+   }
+
+   void set(int index, Nullable${minor.class}Holder holder){
+     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
+   }
+
    @Override
-   public void randomizeData() {
-     if (data != DeadBuf.DEAD_BUFFER) {
-       Random r = new Random();
-       for(int i =0; i < data.capacity()-${type.width}; i += ${type.width}){
-         data.set${(minor.javaType!type.javaType)?cap_first}(i,
-             r.next<#if (type.width >= 4)>${(minor.javaType!type.javaType)?cap_first}
-                   <#else>Int
-                   </#if>());
+   public void generateTestData() {
+     setValueCount(getValueCapacity());
+     boolean even = true;
+     for(int i =0; i < valueCount; i++, even = !even){
+       if(even){
+         set(i, ${minor.boxedType!type.boxedType}.MIN_VALUE);
+       }else{
+         set(i, ${minor.boxedType!type.boxedType}.MAX_VALUE);
        }
      }
    }
+
   </#if> <#-- type.width -->
   
+
+  
    public void setValueCount(int valueCount) {
      ${minor.class}Vector.this.valueCount = valueCount;
      data.writerIndex(${type.width} * valueCount);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
index dfec62e..f1086cb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -1,7 +1,11 @@
 <@pp.dropOutputFile />
 <#list types as type>
 <#list type.minor as minor>
-<@pp.changeOutputFile name="Nullable${minor.class}Vector.java" />
+
+<#assign className = "Nullable${minor.class}Vector" />
+<#assign valuesName = "${minor.class}Vector" />
+<@pp.changeOutputFile name="${className}.java" />
+
 package org.apache.drill.exec.vector;
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -9,8 +13,7 @@ import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
-import java.util.Random;
-import java.util.Vector;
+import java.util.List;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
@@ -23,6 +26,8 @@ import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.UInt2Vector;
 import org.apache.drill.exec.vector.UInt4Vector;
 
+import com.google.common.collect.Lists;
+
 /**
  * Nullable${minor.class} implements a vector of values which could be null.  Elements in the vector
  * are first checked against a fixed length vector of boolean values.  Then the element is retrieved
@@ -31,15 +36,15 @@ import org.apache.drill.exec.vector.UInt4Vector;
  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
  */
 @SuppressWarnings("unused")
-public final class Nullable${minor.class}Vector extends BaseValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector {
+public final class ${className} extends BaseValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector{
 
   private int valueCount;
-  private final BitVector bits;
-  private final ${minor.class}Vector values;
+  final BitVector bits;
+  final ${valuesName} values;
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
 
-  public Nullable${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+  public ${className}(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
     this.bits = new BitVector(null, allocator);
     this.values = new ${minor.class}Vector(null, allocator);
@@ -53,7 +58,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
   public ByteBuf[] getBuffers() {
     return new ByteBuf[]{bits.data, values.data};
   }
-
+  
   @Override
   public void clear() {
     valueCount = 0;
@@ -75,7 +80,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
              .setBufferLength(getBufferSize())
              .build();
   }
-  
+
   @Override
   public void allocateNew(int totalBytes, int valueCount) {
     values.allocateNew(totalBytes, valueCount);
@@ -219,6 +224,11 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
       return bits.getAccessor().get(index);
     }
     
+    public void get(int index, Nullable${minor.class}Holder holder){
+      holder.isSet = bits.getAccessor().get(index);
+      values.getAccessor().get(index, holder);
+    }
+    
     @Override
     public Object getObject(int index) {
       return isNull(index) ? null : values.getAccessor().getObject(index);
@@ -231,7 +241,7 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
     public void reset(){}
   }
   
-  public final class Mutator implements NonRepeatedMutator{
+  public final class Mutator implements ValueVector.Mutator{
     
     private int setCount;
     
@@ -250,6 +260,25 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
       values.getMutator().set(index, value);
     }
 
+
+    public void setSkipNull(int index, ${minor.class}Holder holder){
+      values.getMutator().set(index, holder);
+    }
+
+    public void setSkipNull(int index, Nullable${minor.class}Holder holder){
+      values.getMutator().set(index, holder);
+    }
+    
+    public void set(int index, Nullable${minor.class}Holder holder){
+      bits.getMutator().set(index, holder.isSet);
+      values.getMutator().set(index, holder);
+    }
+
+    public void set(int index, ${minor.class}Holder holder){
+      bits.getMutator().set(index, 1);
+      values.getMutator().set(index, holder);
+    }
+    
     public void setValueCount(int valueCount) {
       assert valueCount >= 0;
       Nullable${minor.class}Vector.this.valueCount = valueCount;
@@ -261,8 +290,9 @@ public final class Nullable${minor.class}Vector extends BaseValueVector implemen
       return valueCount == setCount;
     }
     
-    public void randomizeData(){
-      throw new UnsupportedOperationException();
+    public void generateTestData(){
+      bits.getMutator().generateTestData();
+      values.getMutator().generateTestData();
     }
     
     public void reset(){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
index 30bc086..99b24de 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -1,5 +1,4 @@
-import org.apache.drill.exec.vector.UInt2Vector;
-import org.apache.drill.exec.vector.UInt4Vector;
+
 
 <@pp.dropOutputFile />
 <#list types as type>
@@ -7,6 +6,9 @@ import org.apache.drill.exec.vector.UInt4Vector;
 <@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
 package org.apache.drill.exec.vector;
 
+import org.apache.drill.exec.vector.UInt2Vector;
+import org.apache.drill.exec.vector.UInt4Vector;
+
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 import io.netty.buffer.ByteBuf;
@@ -21,6 +23,8 @@ import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
+import java.util.List;
+import com.google.common.collect.Lists;
 
 @SuppressWarnings("unused")
 /**
@@ -39,7 +43,6 @@ import org.apache.drill.exec.record.TransferPair;
   private int parentValueCount;
   private int childValueCount;
   
-  private final UInt2Vector counts;    // number of repeated elements in each record
   private final UInt4Vector offsets;   // offsets to start of each record
   private final ${minor.class}Vector values;
   private final Mutator mutator = new Mutator();
@@ -48,7 +51,6 @@ import org.apache.drill.exec.record.TransferPair;
   
   public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    this.counts = new UInt2Vector(null, allocator);
     this.offsets = new UInt4Vector(null, allocator);
     this.values = new ${minor.class}Vector(null, allocator);
   }
@@ -58,7 +60,7 @@ import org.apache.drill.exec.record.TransferPair;
   }
   
   int getBufferSize(){
-    return counts.getBufferSize() + offsets.getBufferSize() + values.getBufferSize();
+    return offsets.getBufferSize() + values.getBufferSize();
   }
   
   public TransferPair getTransferPair(){
@@ -66,7 +68,6 @@ import org.apache.drill.exec.record.TransferPair;
   }
   
   public void transferTo(Repeated${minor.class}Vector target){
-    counts.transferTo(target.counts);
     offsets.transferTo(target.offsets);
     values.transferTo(target.values);
     target.parentValueCount = parentValueCount;
@@ -108,8 +109,7 @@ import org.apache.drill.exec.record.TransferPair;
   }
   
   public void allocateNew(int totalBytes, int parentValueCount, int childValueCount) {
-    counts.allocateNew(parentValueCount);
-    offsets.allocateNew(parentValueCount);
+    offsets.allocateNew(parentValueCount+1);
     values.allocateNew(totalBytes, childValueCount);
     mutator.reset();
     accessor.reset();
@@ -121,8 +121,7 @@ import org.apache.drill.exec.record.TransferPair;
     this.parentValueCount = parentValueCount;
     this.childValueCount = childValueCount;
     int loaded = 0;
-    loaded += counts.load(parentValueCount, buf);
-    loaded += offsets.load(parentValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    loaded += offsets.load(parentValueCount+1, buf.slice(loaded, buf.capacity() - loaded));
     loaded += values.load(dataBytes, childValueCount, buf.slice(loaded, buf.capacity() - loaded));
     return loaded;
   }
@@ -152,9 +151,8 @@ import org.apache.drill.exec.record.TransferPair;
   
   public void allocateNew(int parentValueCount, int childValueCount) {
     clear();
+    offsets.allocateNew(parentValueCount+1);
     values.allocateNew(childValueCount);
-    counts.allocateNew(parentValueCount);
-    offsets.allocateNew(parentValueCount);
     mutator.reset();
     accessor.reset();
   }
@@ -164,8 +162,7 @@ import org.apache.drill.exec.record.TransferPair;
     this.parentValueCount = parentValueCount;
     this.childValueCount = childValueCount;
     int loaded = 0;
-    loaded += counts.load(parentValueCount, buf);
-    loaded += offsets.load(parentValueCount, buf.slice(loaded, buf.capacity() - loaded));
+    loaded += offsets.load(parentValueCount+1, buf.slice(loaded, buf.capacity() - loaded));
     loaded += values.load(childValueCount, buf.slice(loaded, buf.capacity() - loaded));
     return loaded;
   }
@@ -177,39 +174,12 @@ import org.apache.drill.exec.record.TransferPair;
     assert metadata.getBufferLength() == loaded;
   }
   </#if>
-  
-//  /**
-//   * Get the size requirement (in bytes) for the given number of values.  Only accurate
-//   * for fixed width value vectors.
-//   */
-//  public int getTotalSizeFromCount(int valueCount) {
-//    return values.getSizeFromCount(valueCount) +
-//           counts.getSizeFromCount(valueCount) +
-//           offsets.getSizeFromCount(valueCount);
-//  }
-//  
-//  public int getSizeFromCount(int valueCount){
-//    return getTotalSizeFromCount(valueCount);
-//  }
-
-//  /**
-//   * Get the explicitly specified size of the allocated buffer, if available.  Otherwise
-//   * calculate the size based on width and record count.
-//   */
-//  public int getAllocatedSize() {
-//    return values.getAllocatedSize() +
-//           counts.getAllocatedSize() +
-//           offsets.getAllocatedSize();
-//  }
-
-
 
   public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{counts.data, offsets.data, values.data};
+    return new ByteBuf[]{offsets.data, values.data};
   }
 
   public void clear(){
-    counts.clear();
     offsets.clear();
     values.clear();
     parentValueCount = 0;
@@ -229,11 +199,17 @@ import org.apache.drill.exec.record.TransferPair;
      * Get the elements at the given index.
      */
     public int getCount(int index) {
-      return counts.getAccessor().get(index);
+      return offsets.getAccessor().get(index+1) - offsets.getAccessor().get(index);
     }
     
     public Object getObject(int index) {
-      throw new UnsupportedOperationException();
+      List<Object> vals = Lists.newArrayList();
+      int start = offsets.getAccessor().get(index);
+      int end = offsets.getAccessor().get(index+1);
+      for(int i = start; i < end; i++){
+        vals.add(values.getAccessor().getObject(i));
+      }
+      return vals;
     }
 
     /**
@@ -247,8 +223,6 @@ import org.apache.drill.exec.record.TransferPair;
     public <#if type.major == "VarLen">byte[]
            <#else>${minor.javaType!type.javaType}
            </#if> get(int index, int positionIndex) {
-
-      assert positionIndex < counts.getAccessor().get(index);
       return values.getAccessor().get(offsets.getAccessor().get(index) + positionIndex);
     }
 
@@ -286,21 +260,40 @@ import org.apache.drill.exec.record.TransferPair;
                                <#elseif type.major == "VarLen"> byte[]
                                <#else> int
                                </#if> value) {
-      counts.getMutator().set(index, counts.getAccessor().get(index) + 1);
-      offsets.getMutator().set(index, offsets.getAccessor().get(index - 1) + counts.getAccessor().get(index-1));
-      values.getMutator().set(offsets.getAccessor().get(index), value);
+      int nextOffset = offsets.getAccessor().get(index+1);
+      values.getMutator().set(nextOffset, value);
+      offsets.getMutator().set(index+1, nextOffset+1);
     }
 
+    public void add(int index, ${minor.class}Holder holder){
+      int nextOffset = offsets.getAccessor().get(index+1);
+      values.getMutator().set(nextOffset, holder);
+      offsets.getMutator().set(index+1, nextOffset+1);
+    }
     
-    public void setGroupAndValueCount(int groupCount, int valueCount) {
+    public void add(int index, Repeated${minor.class}Holder holder){
+      
+      ${minor.class}Vector.Accessor accessor = holder.vector.getAccessor();
+      ${minor.class}Holder innerHolder = new ${minor.class}Holder();
+      
+      for(int i = holder.start; i < holder.end; i++){
+        accessor.get(i, innerHolder);
+        add(index, innerHolder);
+      }
+    }
+    
+    /**
+     * Set the number of value groups in this repeated field.
+     * @param groupCount Count of Value Groups.
+     */
+    public void setValueCount(int groupCount) {
       parentValueCount = groupCount;
-      childValueCount = valueCount;
-      counts.getMutator().setValueCount(groupCount);
+      childValueCount = offsets.getAccessor().get(groupCount+1);
       offsets.getMutator().setValueCount(groupCount);
-      values.getMutator().setValueCount(valueCount);
+      values.getMutator().setValueCount(childValueCount);
     }
     
-    public void randomizeData(){
+    public void generateTestData(){
       throw new UnsupportedOperationException();
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
index b73cd21..ecb5551 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
@@ -23,27 +23,26 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.MaterializedField;
 
+import com.sun.codemodel.JType;
+import com.sun.codemodel.JCodeModel;
+
 public class TypeHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
 
-  private static final int WIDTH_ESTIMATE_1 = 10;
-  private static final int WIDTH_ESTIMATE_2 = 50000;
-  private static final int WIDTH_ESTIMATE_4 = 1024*1024;
+  private static final int WIDTH_ESTIMATE = 50;
 
   public static int getSize(MajorType major) {
     switch (major.getMinorType()) {
 <#list types as type>
   <#list type.minor as minor>
-    <#if minor.class != "Bit">
-      case ${minor.class?upper_case}:
-        return ${type.width}<#if minor.class?substring(0, 3) == "Var" ||
-                                 minor.class?substring(0, 3) == "PRO" ||
-                                 minor.class?substring(0, 3) == "MSG"> + WIDTH_ESTIMATE_${type.width}</#if>;
-    </#if>
+    case ${minor.class?upper_case}:
+      return ${type.width}<#if minor.class?substring(0, 3) == "Var" ||
+                               minor.class?substring(0, 3) == "PRO" ||
+                               minor.class?substring(0, 3) == "MSG"> + WIDTH_ESTIMATE</#if>;
   </#list>
 </#list>
-      case BOOLEAN: return 1;
       case FIXEDCHAR: return major.getWidth();
+      case FIXED16CHAR: return major.getWidth();
       case FIXEDBINARY: return major.getWidth();
     }
     throw new UnsupportedOperationException();
@@ -53,17 +52,6 @@ public class TypeHelper {
     switch (type) {
 <#list types as type>
   <#list type.minor as minor>
-    <#if minor.class == "Bit">
-      case BOOLEAN:
-        switch (mode) {
-          case REQUIRED:
-            return ${minor.class}Vector.class;
-          case OPTIONAL:
-            return Nullable${minor.class}Vector.class;
-          case REPEATED:
-            return Repeated${minor.class}Vector.class;
-        }
-    <#else>
       case ${minor.class?upper_case}:
         switch (mode) {
           case REQUIRED:
@@ -73,7 +61,6 @@ public class TypeHelper {
           case REPEATED:
             return Repeated${minor.class}Vector.class;
         }
-    </#if>
   </#list>
 </#list>
     default:
@@ -82,35 +69,46 @@ public class TypeHelper {
     throw new UnsupportedOperationException();
   }
 
-
-  public static ValueVector getNewVector(MaterializedField field, BufferAllocator allocator){
-    MajorType type = field.getType();
-
-    switch (type.getMinorType()) {
+  public static JType getHolderType(JCodeModel model, MinorType type, DataMode mode){
+    switch (type) {
 <#list types as type>
   <#list type.minor as minor>
-    <#if minor.class != "Bit">
       case ${minor.class?upper_case}:
-        switch (type.getMode()) {
+        switch (mode) {
           case REQUIRED:
-            return new ${minor.class}Vector(field, allocator);
+            return model._ref(${minor.class}Holder.class);
           case OPTIONAL:
-            return new Nullable${minor.class}Vector(field, allocator);
+            return model._ref(Nullable${minor.class}Holder.class);
           case REPEATED:
-            return new Repeated${minor.class}Vector(field, allocator);
+            return model._ref(Repeated${minor.class}Holder.class);
         }
-    </#if>
   </#list>
 </#list>
-      case BOOLEAN:
-        switch (type.getMode()) {
-          case REQUIRED:
-            return new BitVector(field, allocator);
-          case OPTIONAL:
-            return new NullableBitVector(field, allocator);
-          case REPEATED:
-            return new RepeatedBitVector(field, allocator);
-        }
+      default:
+        break;
+      }
+      throw new UnsupportedOperationException();
+  }
+
+  public static ValueVector getNewVector(MaterializedField field, BufferAllocator allocator){
+    MajorType type = field.getType();
+
+    switch (type.getMinorType()) {
+<#list types as type>
+  <#list type.minor as minor>
+    case ${minor.class?upper_case}:
+      switch (type.getMode()) {
+        case REQUIRED:
+          return new ${minor.class}Vector(field, allocator);
+        case OPTIONAL:
+          return new Nullable${minor.class}Vector(field, allocator);
+        case REPEATED:
+          return new Repeated${minor.class}Vector(field, allocator);
+      }
+  </#list>
+</#list>
+    default:
+      break;
     }
     // All ValueVector types have been handled.
     throw new UnsupportedOperationException(type.getMinorType() + " type is not supported. Mode: " + type.getMode());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueHolders.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueHolders.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueHolders.java
new file mode 100644
index 0000000..dad7712
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueHolders.java
@@ -0,0 +1,64 @@
+<@pp.dropOutputFile />
+<#list modes as mode>
+<#list types as type>
+<#list type.minor as minor>
+
+<#assign className="${mode.prefix}${minor.class}Holder" />
+<@pp.changeOutputFile name="${className}.java" />
+package org.apache.drill.exec.vector;
+
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.holders.ValueHolder;
+import io.netty.buffer.ByteBuf;
+
+public final class ${className} implements ValueHolder{
+  
+  public static final MajorType TYPE = Types.${mode.name?lower_case}(MinorType.${minor.class?upper_case});
+
+    <#if mode.name != "Repeated">
+      
+    public static final int WIDTH = ${type.width};
+      <#if mode.name == "Optional">
+      /** Whether the given holder holds a valid value.  1 means non-null.  0 means null. **/
+      public int isSet;
+      </#if>
+      
+      <#if type.major != "VarLen">
+      
+      <#if (type.width > 8)>
+      public int start;
+      public ByteBuf buffer;
+      <#else>
+        public ${minor.javaType!type.javaType} value;
+      
+      </#if>
+      <#else>
+      /** The first offset (inclusive) into the buffer. **/
+      public int start;
+      
+      /** The last offset (exclusive) into the buffer. **/
+      public int end;
+      
+      /** The buffer holding actual values. **/
+      public ByteBuf buffer;
+      </#if>
+    <#else> 
+    
+      /** The first index (inclusive) into the Vector. **/
+      public int start;
+      
+      /** The last index (exclusive) into the Vector. **/
+      public int end;
+      
+      /** The Vector holding the actual values. **/
+      public ${minor.class}Vector vector;
+    </#if>
+  
+    
+}
+
+</#list>
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
index 7daadee..0871b82 100644
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -21,6 +21,9 @@ import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.ByteHolder;
+import org.mortbay.jetty.servlet.Holder;
+
+import com.google.common.base.Charsets;
 
 import antlr.collections.impl.Vector;
 
@@ -172,6 +175,18 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       return dst;
     }
     
+    public void get(int index, ${minor.class}Holder holder){
+      holder.start = offsetVector.getAccessor().get(index);
+      holder.end = offsetVector.getAccessor().get(index + 1);
+      holder.buffer = data;
+    }
+    
+    void get(int index, Nullable${minor.class}Holder holder){
+      holder.start = offsetVector.getAccessor().get(index);
+      holder.end = offsetVector.getAccessor().get(index + 1);
+      holder.buffer = data;
+    }
+
     public void get(int index, ByteHolder holder){
       assert index >= 0;
       holder.start = offsetVector.getAccessor().get(index);
@@ -222,14 +237,37 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       data.setBytes(currentOffset, bb);
     }
 
+    void set(int index, Nullable${minor.class}Holder holder){
+      int length = holder.end - holder.start;
+      int currentOffset = offsetVector.getAccessor().get(index);
+      offsetVector.getMutator().set(index + 1, currentOffset + length);
+      data.setBytes(currentOffset, holder.buffer, holder.start, length);
+    }
+    
+    public void set(int index, ${minor.class}Holder holder){
+      int length = holder.end - holder.start;
+      int currentOffset = offsetVector.getAccessor().get(index);
+      offsetVector.getMutator().set(index + 1, currentOffset + length);
+      data.setBytes(currentOffset, holder.buffer, holder.start, length);
+    }
+    
     public void setValueCount(int valueCount) {
       ${minor.class}Vector.this.valueCount = valueCount;
-      data.writerIndex(valueCount * ${type.width});
+      data.writerIndex(offsetVector.getAccessor().get(valueCount));
       offsetVector.getMutator().setValueCount(valueCount+1);
     }
 
     @Override
-    public void randomizeData(){}
+    public void generateTestData(){
+      boolean even = true;
+      for(int i =0; i < valueCount; i++, even = !even){
+        if(even){
+          set(i, new String("aaaaa").getBytes(Charsets.UTF_8));
+        }else{
+          set(i, new String("bbbbbbbbbb").getBytes(Charsets.UTF_8));
+        }
+      }
+    }
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
deleted file mode 100644
index 5e7f1bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CodeModelTools.java
+++ /dev/null
@@ -1,31 +0,0 @@
-package org.apache.drill.exec.compile;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.expr.CodeGenerator;
-
-import com.sun.codemodel.JType;
-
-public class CodeModelTools {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeModelTools.class);
-  
-  public static JType getType(MinorType mt, DataMode mode, CodeGenerator g){
-    switch (mt) {
-    case BOOLEAN:
-      return g.getModel().BOOLEAN;
-    case INT:
-      return g.getModel().INT;
-    case BIGINT:
-      return g.getModel().LONG;
-    default:
-      throw new UnsupportedOperationException();
-    }
-  }
-  
-  
-  public static JType getType(MajorType mt, CodeGenerator g) {
-    return getType(mt.getMinorType(), mt.getMode(), g);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
index b8dc2ce..13c7781 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -9,16 +9,9 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.compile.TemplateClassDefinition;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.holders.BooleanHolder;
-import org.apache.drill.exec.expr.holders.Float8Holder;
-import org.apache.drill.exec.expr.holders.IntHolder;
-import org.apache.drill.exec.expr.holders.LongHolder;
-import org.apache.drill.exec.expr.holders.NullableBooleanHolder;
-import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
-import org.apache.drill.exec.expr.holders.NullableIntHolder;
-import org.apache.drill.exec.expr.holders.NullableLongHolder;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.vector.TypeHelper;
 
 import com.google.common.base.Preconditions;
 import com.sun.codemodel.JBlock;
@@ -200,35 +193,6 @@ public class CodeGenerator<T> {
   }
   
   public JType getHolderType(MajorType t){
-    switch(t.getMode()){
-    case REQUIRED:
-      switch(t.getMinorType()){
-      case BOOLEAN:
-        return model._ref(BooleanHolder.class);
-      case INT:
-        return model._ref(IntHolder.class);
-      case BIGINT:  
-        return model._ref(LongHolder.class);
-      case FLOAT8:
-        return model._ref(Float8Holder.class);
-      
-      }
-      
-    case OPTIONAL:
-      switch(t.getMinorType()){
-      case BOOLEAN:
-        return model._ref(NullableBooleanHolder.class);
-      case INT:
-        return model._ref(NullableIntHolder.class);
-      case BIGINT:  
-        return model._ref(NullableLongHolder.class);
-      case FLOAT8:
-        return model._ref(NullableFloat8Holder.class);
-      }
-
-    }
-    
-    
-    throw new UnsupportedOperationException();
+    return TypeHelper.getHolderType(model, t.getMinorType(), t.getMode());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
index 83f8368..0915fa3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Alternator.java
@@ -11,8 +11,8 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Workspace;
-import org.apache.drill.exec.expr.holders.LongHolder;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.vector.BigIntHolder;
 
 
 
@@ -20,7 +20,7 @@ import org.apache.drill.exec.record.RecordBatch;
 public class Alternator implements DrillFunc{
 
   @Workspace int val;
-  @Output LongHolder out;
+  @Output BigIntHolder out;
   
   public void setup(RecordBatch incoming) {
     val = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 9808bd5..78630ba 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -6,9 +6,9 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.IntHolder;
-import org.apache.drill.exec.expr.holders.LongHolder;
 import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.vector.BigIntHolder;
+import org.apache.drill.exec.vector.IntHolder;
 
 public class MathFunctions{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MathFunctions.class);
@@ -33,9 +33,9 @@ public class MathFunctions{
   @FunctionTemplate(name = "add", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class LongAdd1 implements DrillFunc{
     
-    @Param LongHolder left;
-    @Param LongHolder right;
-    @Output LongHolder out;
+    @Param BigIntHolder left;
+    @Param BigIntHolder right;
+    @Output BigIntHolder out;
 
     public void setup(RecordBatch b){}
     
@@ -45,14 +45,11 @@ public class MathFunctions{
 
   }
   
-  /** 
-   * Define the actual absolute value implementation
-   */
   @FunctionTemplate(name = "negative", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class Negative implements DrillFunc{
     
-    @Param LongHolder input;
-    @Output LongHolder out;
+    @Param BigIntHolder input;
+    @Output BigIntHolder out;
 
     public void setup(RecordBatch b){}
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
deleted file mode 100644
index 70bcd10..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/BooleanHolder.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class BooleanHolder implements ValueHolder{
-  public static final MajorType TYPE = Types.required(MinorType.BOOLEAN); 
-  public int value;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/Float8Holder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/Float8Holder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/Float8Holder.java
deleted file mode 100644
index 78e7cb8..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/Float8Holder.java
+++ /dev/null
@@ -1,14 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-
-public class Float8Holder implements ValueHolder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Float8Holder.class);
-  
-  public static final MajorType TYPE = Types.required(MinorType.FLOAT8);
-  public double value;
-  public int isSet;
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
deleted file mode 100644
index 81c82b9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/IntHolder.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class IntHolder implements ValueHolder{
-  public static final MajorType TYPE = Types.required(MinorType.INT); 
-  public int value;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
deleted file mode 100644
index c73f9ad..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/LongHolder.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class LongHolder implements ValueHolder {
-  public static final MajorType TYPE = Types.required(MinorType.BIGINT);
-  public long value;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
deleted file mode 100644
index f6d2f2e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableBooleanHolder.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class NullableBooleanHolder implements ValueHolder {
-  public static final MajorType TYPE = Types.optional(MinorType.BOOLEAN);
-  public int value;
-  public int isSet;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableFloat8Holder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableFloat8Holder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableFloat8Holder.java
deleted file mode 100644
index f8fa224..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableFloat8Holder.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-
-public class NullableFloat8Holder implements ValueHolder{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFloat8Holder.class);
-  
-  public static final MajorType TYPE = Types.optional(MinorType.FLOAT8);
-  public double value;
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
deleted file mode 100644
index 7eca21e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableIntHolder.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class NullableIntHolder implements ValueHolder {
-  public static final MajorType TYPE = Types.optional(MinorType.INT);
-  public int value;
-  public int isSet;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
deleted file mode 100644
index d1be8f1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/NullableLongHolder.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-
-public final class NullableLongHolder implements ValueHolder {
-  public static final MajorType TYPE = Types.optional(MinorType.BIGINT);
-  public long value;
-  public int isSet;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
deleted file mode 100644
index 554c7d3..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/holders/VarBinaryHolder.java
+++ /dev/null
@@ -1,12 +0,0 @@
-package org.apache.drill.exec.expr.holders;
-
-import io.netty.buffer.ByteBuf;
-
-public class VarBinaryHolder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarBinaryHolder.class);
-  
-  public ByteBuf buffer;
-  public int start;
-  public int length;
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
index 6bddab7..c62d76b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -41,6 +41,38 @@ public abstract class BufferAllocator implements Closeable{
   
   public abstract BufferAllocator getChildAllocator(long initialReservation, long maximumReservation);
   
+  protected abstract boolean pre(int bytes);
+  
+  public PreAllocator getPreAllocator(){
+    return new PreAllocator(); 
+  }
+  
+  public class PreAllocator{
+    int bytes = 0;
+    public boolean preAllocate(int bytes){
+      if(!pre(bytes)) return false;
+      
+      this.bytes += bytes;
+      return true;
+   
+    }
+    
+    public ByteBuf getAllocation(){
+      return buffer(bytes);
+    }
+  }
+  
+  
+  /**
+   * @param bytes
+   * @return
+   */
+  
+  /**
+   * 
+   */
+  
+  
   /**
    * Close and release all buffers generated from this buffer pool.
    */

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
index 8c5b003..027dae6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/DirectBufferAllocator.java
@@ -31,6 +31,16 @@ public class DirectBufferAllocator extends BufferAllocator{
     // TODO: wrap it
     return buffer.directBuffer(size);
   }
+  
+  
+
+  @Override
+  protected boolean pre(int bytes) {
+    // TODO: check allocation
+    return true;
+  }
+
+
 
   @Override
   public long getAllocatedMemory() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
index c8bc3a8..6716a8f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -89,4 +90,14 @@ public interface Exchange extends PhysicalOperator {
    * @return
    */
   public PhysicalOperator getChild();
+  
+  
+  
+  /**
+   * Informs the planner whether or not this particular exchange supports an incoming stream that has an attached selection vector. 
+   * @param mode
+   * @return
+   */
+  @JsonIgnore
+  public boolean supportsSelectionVector();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
index 1f158ce..4600556 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
@@ -79,6 +79,11 @@ public class HashToRandomExchange extends AbstractExchange{
     return new HashToRandomExchange(child, expr);
   }
 
+  @Override
+  public boolean supportsSelectionVector() {
+    return true;
+  }
+
   
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
index 687b28b..4d536b4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -31,7 +31,6 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.FixedWidthVector;
-import org.apache.drill.exec.vector.NonRepeatedMutator;
 import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
@@ -98,23 +97,13 @@ public class MockRecordReader implements RecordReader {
 
     recordsRead += recordSetSize;
     for(ValueVector v : valueVectors){
-      if(v instanceof FixedWidthVector){
-        ((FixedWidthVector)v).allocateNew(recordSetSize);
-      }else if(v instanceof VariableWidthVector){
-        ((VariableWidthVector)v).allocateNew(50*recordSetSize, recordSetSize);
-      }else{
-        throw new UnsupportedOperationException();
-      }
+      AllocationHelper.allocate(v, recordSetSize, 50);
       
       logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
       ValueVector.Mutator m = v.getMutator();
-      m.randomizeData();
+      m.generateTestData();
       
-      if(m instanceof NonRepeatedMutator){
-        ((NonRepeatedMutator)m).setValueCount(recordSetSize);  
-      }else{
-        throw new UnsupportedOperationException();
-      }
+      m.setValueCount(recordSetSize);
       
     }
     return recordSetSize;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
index 56467ce..de40b96 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
@@ -74,6 +74,11 @@ public class UnionExchange extends AbstractExchange{
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
     return new UnionExchange(child);
   }
+
+  @Override
+  public boolean supportsSelectionVector() {
+    return false;
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
index 60e8f42..babc66e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterRecordBatch.java
@@ -6,38 +6,23 @@ import java.util.List;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
-import org.apache.drill.exec.expr.ValueVectorReadExpression;
-import org.apache.drill.exec.expr.ValueVectorWriteExpression;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.impl.VectorHolder;
-import org.apache.drill.exec.physical.impl.project.Projector;
-import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
-import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
-import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.NonRepeatedMutator;
-import org.apache.drill.exec.vector.TypeHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.google.common.base.Preconditions;
@@ -136,11 +121,7 @@ public class FilterRecordBatch implements RecordBatch{
       filter.filterBatch(recordCount);
       for(ValueVector v : this.outputVectors){
         ValueVector.Mutator m = v.getMutator();
-        if(m instanceof NonRepeatedMutator){
-          ((NonRepeatedMutator) m).setValueCount(recordCount);
-        }else{
-          throw new UnsupportedOperationException();
-        }
+        m.setValueCount(recordCount);
       }
       return upstream; // change if upstream changed, otherwise normal.
     default:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/742f4c11/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
index 4092911..92a6994 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterTemplate.java
@@ -19,7 +19,7 @@ public abstract class FilterTemplate implements Filterer{
   public void setup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, TransferPair[] transfers) throws SchemaChangeException{
     this.transfers = transfers;
     this.outgoingSelectionVector = outgoing.getSelectionVector2();
-    this.svMode = incoming.getSchema().getSelectionVector();
+    this.svMode = incoming.getSchema().getSelectionVectorMode();
     
     switch(svMode){
     case NONE: