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/15 22:24:03 UTC

[3/7] git commit: Updated value vectors inheritance model. Moved Mutables to separate Mutator subclasses. Broke VVs into separate files rather than one large class.

Updated value vectors inheritance model.
Moved Mutables to separate Mutator subclasses.
Broke VVs into separate files rather than one large class.


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

Branch: refs/heads/execwork
Commit: 36793bb2f22cd427c8b70d2f5dfe4a4d8a3a6894
Parents: 7075cca
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat Jul 13 21:28:12 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 11:36:32 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/exec/java-exec/pom.xml        |   2 +-
 .../templates/FixedValueVectors.java            | 163 ++++
 .../templates/NullableValueVectors.java         | 151 ++++
 .../templates/RepeatedValueVectors.java         | 158 ++++
 .../ValueVectors/templates/TypeHelper.java      |  28 +-
 .../ValueVectors/templates/ValueVector.java     | 768 -------------------
 .../templates/VariableLengthVectors.java        | 152 ++++
 .../exec/physical/config/MockRecordReader.java  |  17 +-
 .../physical/config/MockScanBatchCreator.java   |   3 -
 .../drill/exec/physical/config/MockScanPOP.java |   2 +-
 .../exec/physical/impl/FilterRecordBatch.java   |   4 +-
 .../drill/exec/physical/impl/OutputMutator.java |   5 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  24 +-
 .../exec/physical/impl/WireRecordBatch.java     |   7 +-
 .../apache/drill/exec/record/BatchSchema.java   |   2 -
 .../apache/drill/exec/record/RecordBatch.java   |   4 +-
 .../drill/exec/record/RecordBatchLoader.java    |  22 +-
 .../apache/drill/exec/record/WritableBatch.java |  10 +-
 .../exec/record/vector/SelectionVector.java     |  14 +-
 .../drill/exec/store/JSONRecordReader.java      |  87 ++-
 .../apache/drill/exec/store/VectorHolder.java   |  11 +-
 .../org/apache/drill/exec/vector/BitVector.java | 123 +++
 .../apache/drill/exec/vector/ValueVector.java   | 192 +++++
 .../physical/impl/TestSimpleFragmentRun.java    |   6 +-
 .../exec/record/vector/TestValueVector.java     | 201 +++--
 .../drill/exec/store/JSONRecordReaderTest.java  |  40 +-
 26 files changed, 1212 insertions(+), 984 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/pom.xml b/sandbox/prototype/exec/java-exec/pom.xml
index 4e4df95..1b6dac0 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -173,7 +173,7 @@
 				<version>1.0</version>
 				<configuration>
 					<cfgFile>src/main/codegen/ValueVectors/config.fmpp</cfgFile>
-					<outputDirectory>target/generated-sources/org/apache/drill/exec/record/vector</outputDirectory>
+					<outputDirectory>target/generated-sources/org/apache/drill/exec/vector</outputDirectory>
 					<templateDirectory>src/main/codegen/ValueVectors/templates</templateDirectory>
 				</configuration>
 				<executions>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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
new file mode 100644
index 0000000..09dd5d8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/FixedValueVectors.java
@@ -0,0 +1,163 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+
+<#if type.major == "Fixed">
+<@pp.changeOutputFile name="${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+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.MsgPack2Vector;
+
+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
+ * vector sequentially, but may be randomly accessed.
+ *   The width of each element is ${type.width} byte(s)
+ *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class ${minor.class}Vector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
+
+  public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+  }
+
+  /**
+   * 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) {
+    totalBytes = valueCount * ${type.width};
+    allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return (int) Math.ceil(totalBytes);
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  @Override
+  public int getSizeFromCount(int valueCount) {
+    return valueCount * ${type.width};
+  }
+
+  public Mutator getMutator() {
+    return new Mutator();
+  }
+
+ <#if (type.width > 8)>
+
+  public ${minor.javaType!type.javaType} get(int index) {
+    ByteBuf dst = allocator.buffer(${type.width});
+    data.getBytes(index * ${type.width}, dst, 0, ${type.width});
+    return dst;
+  }
+
+  @Override
+  public Object getObject(int index) {
+    ByteBuf dst = allocator.buffer(${type.width});
+    data.getBytes(index, dst, 0, ${type.width});
+    return dst;
+  }
+
+
+ <#else> <#-- type.width <= 8 -->
+
+  public ${minor.javaType!type.javaType} get(int index) {
+    return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+  }
+
+  public Object getObject(int index) {
+    return get(index);
+  }
+
+
+ </#if> <#-- type.width -->
+ 
+ 
+ /**
+  * ${minor.class}.Mutator implements a mutable vector of fixed 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.
+  *   The width of each element is ${type.width} byte(s)
+  *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+  *
+  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+  */
+  public class Mutator implements ValueVector.Mutator{
+
+    private Mutator(){};
+   /**
+    * Set the element at the given index to the given value.  Note that widths smaller than
+    * 32 bits are handled by the ByteBuf interface.
+    *
+    * @param index   position of the bit to set
+    * @param value   value to set
+    */
+  <#if (type.width > 8)>
+   public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+     data.setBytes(index * ${type.width}, 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}){
+         byte[] bytes = new byte[${type.width}];
+         r.nextBytes(bytes);
+         data.setByte(i, bytes[0]);
+       }
+     }
+   }
+   
+  <#else> <#-- type.width <= 8 -->
+   public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, 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>());
+       }
+     }
+   }
+  </#if> <#-- type.width -->
+  
+   @Override
+   public void setRecordCount(int recordCount) {
+     ${minor.class}Vector.this.setRecordCount(recordCount);
+   }
+
+
+
+  
+ }
+}
+
+</#if> <#-- type.major -->
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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
new file mode 100644
index 0000000..c7de73f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/NullableValueVectors.java
@@ -0,0 +1,151 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+<@pp.changeOutputFile name="Nullable${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+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.UInt2Vector;
+import org.apache.drill.exec.vector.UInt4Vector;
+
+/**
+ * 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
+ * from the base class (if not null).
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class Nullable${minor.class}Vector extends ValueVector {
+
+  private final BitVector bits;
+  private final ${minor.class}Vector values;
+
+  public Nullable${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    bits = new BitVector(null, allocator);
+    values = new ${minor.class}Vector(null, allocator);
+  }
+
+  /**
+   * Get the element at the specified position.
+   *
+   * @param   index   position of the value
+   * @return  value of the element, if not null
+   * @throws  NullValueException if the value is null
+   */
+  public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
+    assert !isNull(index);
+    return values.get(index);
+  }
+
+
+  public boolean isNull(int index) {
+    return bits.get(index) == 0;
+  }
+
+  public int isSet(int index){
+    return bits.get(index);
+  }
+  
+  /**
+   * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
+   *
+   * @param valueCount   The number of values which may be contained by this vector.
+   */
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    values.allocateNew(totalBytes, sourceBuffer, valueCount);
+    bits.allocateNew(valueCount);
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return bits.getAllocatedSize() + values.getAllocatedSize();
+  }
+
+  /**
+   * 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) + bits.getSizeFromCount(valueCount);
+  }
+  
+  public int getSizeFromCount(int valueCount){
+    return getTotalSizeFromCount(valueCount);
+  }
+
+  @Override
+  public MaterializedField getField() {
+    return field;
+  }
+
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{bits.data, values.data};
+  }
+
+
+  @Override
+  public Object getObject(int index) {
+    return isNull(index) ? null : values.getObject(index);
+  }
+  
+  public Mutator getMutator(){
+    return new Mutator();
+  }
+  
+  public class Mutator implements ValueVector.Mutator{
+
+    private final BitVector.Mutator bitMutator;
+    private final ${minor.class}Vector.Mutator valueMutator;
+    
+    private Mutator(){
+      bitMutator = bits.getMutator();
+      valueMutator = values.getMutator();
+    }
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
+      setNotNull(index);
+      valueMutator.set(index, value);
+    }
+
+    public void setNull(int index) {
+      bitMutator.set(index, 0);
+    }
+
+    private void setNotNull(int index) {
+      bitMutator.set(index, 1);
+    }
+    
+    @Override
+    public void setRecordCount(int recordCount) {
+      Nullable${minor.class}Vector.this.setRecordCount(recordCount);
+      bits.setRecordCount(recordCount);
+    }
+    
+    public void randomizeData(){
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+}
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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
new file mode 100644
index 0000000..4acc4cc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/RepeatedValueVectors.java
@@ -0,0 +1,158 @@
+import org.apache.drill.exec.vector.UInt2Vector;
+import org.apache.drill.exec.vector.UInt4Vector;
+
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+<@pp.changeOutputFile name="Repeated${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+@SuppressWarnings("unused")
+/**
+ * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or
+ * repeated protobuf field).  The implementation uses two additional value vectors; one to convert
+ * the index offset to the underlying element offset, and another to store the number of values
+ * in the vector.
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+
+ public final class Repeated${minor.class}Vector extends ValueVector {
+
+  private final UInt2Vector countVector;    // number of repeated elements in each record
+  private final UInt4Vector offsetVector;   // offsets to start of each record
+  private final ${minor.class}Vector valuesVector;
+
+  public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.countVector = new UInt2Vector(null, allocator);
+    this.offsetVector = new UInt4Vector(null, allocator);
+    this.valuesVector = new ${minor.class}Vector(null, allocator);
+  }
+
+  public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    super.allocateNew(totalBytes, sourceBuffer, valueCount);
+    countVector.allocateNew(valueCount);
+    offsetVector.allocateNew(valueCount);
+  }
+
+
+  /**
+   * Get a value for the given record.  Each element in the repeated field is accessed by
+   * the positionIndex param.
+   *
+   * @param  index           record containing the repeated field
+   * @param  positionIndex   position within the repeated field
+   * @return element at the given position in the given record
+   */
+  public <#if type.major == "VarLen">byte[]
+         <#else>${minor.javaType!type.javaType}
+         </#if> get(int index, int positionIndex) {
+
+    assert positionIndex < countVector.get(index);
+    return valuesVector.get(offsetVector.get(index) + positionIndex);
+  }
+
+  public MaterializedField getField() {
+    return field;
+  }
+
+  /**
+   * 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 valuesVector.getSizeFromCount(valueCount) +
+           countVector.getSizeFromCount(valueCount) +
+           offsetVector.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 valuesVector.getAllocatedSize() +
+           countVector.getAllocatedSize() +
+           offsetVector.getAllocatedSize();
+  }
+
+  /**
+   * Get the elements at the given index.
+   */
+  public int getCount(int index) {
+    return countVector.get(index);
+  }
+
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{countVector.data, offsetVector.data, data};
+  }
+
+  public Object getObject(int index) {
+    return data.slice(index, getSizeFromCount(countVector.get(index)));
+  }
+
+  public Mutator getMutator(){
+    return new Mutator();
+  }
+  
+  public class Mutator implements ValueVector.Mutator{
+
+    
+    private final UInt2Vector.Mutator countMutator;
+    private final ${minor.class}Vector.Mutator valuesMutator;
+    private final UInt4Vector.Mutator offsetMutator;
+    
+    private Mutator(){
+      this.countMutator = countVector.getMutator();
+      this.offsetMutator = offsetVector.getMutator();
+      this.valuesMutator = valuesVector.getMutator();
+    }
+
+    /**
+     * Add an element to the given record index.  This is similar to the set() method in other
+     * value vectors, except that it permits setting multiple values for a single record.
+     *
+     * @param index   record of the element to add
+     * @param value   value to add to the given row
+     */
+    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
+                               <#elseif type.major == "VarLen"> byte[]
+                               <#else> int
+                               </#if> value) {
+      countMutator.set(index, countVector.get(index) + 1);
+      offsetMutator.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
+      valuesMutator.set(offsetVector.get(index), value);
+    }
+    
+    public void setRecordCount(int recordCount) {
+      valuesMutator.setRecordCount(recordCount);
+      offsetMutator.setRecordCount(recordCount);
+      countMutator.setRecordCount(recordCount);
+    }
+    
+    public void randomizeData(){
+      throw new UnsupportedOperationException();
+    }
+    
+  }
+}
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 8dfd3af..b03b842 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
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  ******************************************************************************/
-package org.apache.drill.exec.record.vector;
+package org.apache.drill.exec.vector;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
@@ -57,21 +57,21 @@ public class TypeHelper {
       case BOOLEAN:
         switch (mode) {
           case REQUIRED:
-            return ValueVector.${minor.class}.class;
+            return ${minor.class}Vector.class;
           case OPTIONAL:
-            return ValueVector.Nullable${minor.class}.class;
+            return Nullable${minor.class}Vector.class;
           case REPEATED:
-            return ValueVector.Repeated${minor.class}.class;
+            return Repeated${minor.class}Vector.class;
         }
     <#else>
       case ${minor.class?upper_case}:
         switch (mode) {
           case REQUIRED:
-            return ValueVector.${minor.class}.class;
+            return ${minor.class}Vector.class;
           case OPTIONAL:
-            return ValueVector.Nullable${minor.class}.class;
+            return Nullable${minor.class}Vector.class;
           case REPEATED:
-            return ValueVector.Repeated${minor.class}.class;
+            return Repeated${minor.class}Vector.class;
         }
     </#if>
   </#list>
@@ -83,7 +83,7 @@ public class TypeHelper {
   }
 
 
-  public static ValueVector.Base getNewVector(MaterializedField field, BufferAllocator allocator){
+  public static ValueVector getNewVector(MaterializedField field, BufferAllocator allocator){
     MajorType type = field.getType();
 
     switch (type.getMinorType()) {
@@ -93,11 +93,11 @@ public class TypeHelper {
       case ${minor.class?upper_case}:
         switch (type.getMode()) {
           case REQUIRED:
-            return new ValueVector.${minor.class}(field, allocator);
+            return new ${minor.class}Vector(field, allocator);
           case OPTIONAL:
-            return new ValueVector.Nullable${minor.class}(field, allocator);
+            return new Nullable${minor.class}Vector(field, allocator);
           case REPEATED:
-            return new ValueVector.Repeated${minor.class}(field, allocator);
+            return new Repeated${minor.class}Vector(field, allocator);
         }
     </#if>
   </#list>
@@ -105,11 +105,11 @@ public class TypeHelper {
       case BOOLEAN:
         switch (type.getMode()) {
           case REQUIRED:
-            return new ValueVector.Bit(field, allocator);
+            return new BitVector(field, allocator);
           case OPTIONAL:
-            return new ValueVector.NullableBit(field, allocator);
+            return new NullableBitVector(field, allocator);
           case REPEATED:
-            return new ValueVector.RepeatedBit(field, allocator);
+            return new RepeatedBitVector(field, allocator);
         }
     }
     // All ValueVector types have been handled.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
deleted file mode 100644
index e0e895b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
+++ /dev/null
@@ -1,768 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.record.vector;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import io.netty.buffer.ByteBuf;
-import java.io.Closeable;
-import java.util.Random;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.record.DeadBuf;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * 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 class ValueVector {
-
-  /**
-   * ValueVector.Base implements common logic for all immutable value vectors.
-   */
-  public abstract static class Base implements Closeable {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Base.class);
-
-    protected final BufferAllocator allocator;
-    protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-    protected MaterializedField field;
-    protected int recordCount;
-    protected int totalBytes;
-
-    public Base(MaterializedField field, BufferAllocator allocator) {
-      this.allocator = allocator;
-      this.field = field;
-    }
-
-    /**
-     * 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();
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Takes derived
-     * type specs into account.
-     */
-    public abstract int getSizeFromCount(int valueCount);
-
-    /**
-     * Get the Java Object representation of the element at the specified position
-     *
-     * @param index   Index of the value to get
-     */
-    public abstract Object getObject(int index);
-
-    /**
-     * 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
-     * that this buffer changes regularly thus external classes shouldn't hold a reference to
-     * it (unless they change it).
-     *
-     * @return The underlying ByteBuf.
-     */
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{data};
-    }
-
-    /**
-     * Returns the maximum number of values contained within this vector.
-     * @return Vector size
-     */
-    public int capacity() {
-      return getRecordCount();
-    }
-
-    /**
-     * Release supporting resources.
-     */
-    @Override
-    public void close() {
-      clear();
-    }
-
-    /**
-     * Get information about how this field is materialized.
-     * @return
-     */
-    public MaterializedField getField() {
-      return field;
-    }
-
-    /**
-     * Get the number of records allocated for this value vector.
-     * @return number of allocated records
-     */
-    public int getRecordCount() {
-      return recordCount;
-    }
-
-    /**
-     * Get the metadata for this field.
-     * @return
-     */
-    public FieldMetadata getMetadata() {
-      int len = 0;
-      for(ByteBuf b : getBuffers()){
-        len += b.writerIndex();
-      }
-      return FieldMetadata.newBuilder()
-               .setDef(getField().getDef())
-               .setValueCount(getRecordCount())
-               .setBufferLength(len)
-               .build();
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param totalBytes   Optional desired size of the underlying buffer.  Specifying 0 will
-     *                     estimate the size based on valueCount.
-     * @param sourceBuffer Optional ByteBuf to use for storage (null will allocate automatically).
-     * @param valueCount   Number of values in the vector.
-     */
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      clear();
-      this.recordCount = valueCount;
-      this.totalBytes = totalBytes > 0 ? totalBytes : getSizeFromCount(valueCount);
-      this.data = (sourceBuffer != null) ? sourceBuffer : allocator.buffer(this.totalBytes);
-      this.data.retain();
-      data.readerIndex(0);
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount
-     *          The number of elements which can be contained within this vector.
-     */
-    public void allocateNew(int valueCount) {
-      allocateNew(0, null, valueCount);
-    }
-
-    /**
-     * Release the underlying ByteBuf and reset the ValueVector
-     */
-    protected void clear() {
-      if (data != DeadBuf.DEAD_BUFFER) {
-        data.release();
-        data = DeadBuf.DEAD_BUFFER;
-        recordCount = 0;
-        totalBytes = 0;
-      }
-    }
-
-    /**
-     * Define the number of records that are in this value vector.
-     * @param recordCount Number of records active in this vector.
-     */
-    public void setRecordCount(int recordCount) {
-      data.writerIndex(getSizeFromCount(recordCount));
-      this.recordCount = recordCount;
-    }
-
-    /**
-     * For testing only -- randomize the buffer contents
-     */
-    public void randomizeData() { }
-
-  }
-
-  /**
-   * 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 equivilent Java primitive is an int containing the value '0' or '1'.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Bit extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
-
-    public Bit(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * 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) {
-      // 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;
-    }
-
-    @Override
-    public Object getObject(int index) {
-      return new Boolean(get(index) != 0);
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.
-     */
-    @Override
-    public int getSizeFromCount(int valueCount) {
-      return (int) Math.ceil(valueCount / 8);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return totalBytes;
-    }
-
-    public MutableBit getMutable() {
-      return (MutableBit)this;
-    }
-
-    /**
-     * 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.
-     */
-    @Override
-    public void allocateNew(int valueCount) {
-      allocateNew(getSizeFromCount(valueCount), null, valueCount);
-      for (int i = 0; i < getSizeFromCount(valueCount); i++) {
-        data.setByte(i, 0);
-      }
-    }
-
-  }
-
-  /**
-   * 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 static class MutableBit extends Bit {
-
-    public MutableBit(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * 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)
-     */
-    public 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))) {
-        // false, and bit was previously set
-        currentByte -= (byte) Math.pow(2, (index % 8));
-      }
-      data.setByte((int) Math.floor(index/8), currentByte);
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.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]);
-        }
-      }
-    }
-  }
-
-<#list types as type>
- <#list type.minor as minor>
-  <#if type.major == "Fixed">
-
-  /**
-   * ${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
-   * vector sequentially, but may be randomly accessed.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class ${minor.class} extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
-
-    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * 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) {
-      totalBytes = valueCount * ${type.width};
-      allocateNew(totalBytes, allocator.buffer(totalBytes), valueCount);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return (int) Math.ceil(totalBytes);
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    @Override
-    public int getSizeFromCount(int valueCount) {
-      return valueCount * ${type.width};
-    }
-
-    public Mutable${minor.class} getMutable() {
-      return (Mutable${minor.class})this;
-    }
-
-   <#if (type.width > 8)>
-
-    public ${minor.javaType!type.javaType} get(int index) {
-      ByteBuf dst = allocator.buffer(${type.width});
-      data.getBytes(index * ${type.width}, dst, 0, ${type.width});
-      return dst;
-    }
-
-    @Override
-    public Object getObject(int index) {
-      ByteBuf dst = allocator.buffer(${type.width});
-      data.getBytes(index, dst, 0, ${type.width});
-      return dst;
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.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]);
-        }
-      }
-    }
-
-   <#else> <#-- type.width <= 8 -->
-
-    public ${minor.javaType!type.javaType} get(int index) {
-      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-    }
-
-    public Object getObject(int index) {
-      return get(index);
-    }
-
-    @Override
-    public void randomizeData() {
-      if (this.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>());
-        }
-      }
-    }
-
-   </#if> <#-- type.width -->
-  }
-
-  /**
-   * Mutable${minor.class} implements a mutable vector of fixed 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.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-   public static class Mutable${minor.class} extends ${minor.class} {
-
-    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Set the element at the given index to the given value.  Note that widths smaller than
-     * 32 bits are handled by the ByteBuf interface.
-     *
-     * @param index   position of the bit to set
-     * @param value   value to set
-     */
-   <#if (type.width > 8)>
-    public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.setBytes(index * ${type.width}, value);
-   <#else> <#-- type.width <= 8 -->
-    public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
-   </#if> <#-- type.width -->
-    }
-  }
-
-  <#elseif type.major == "VarLen">
-
-  /**
-   * ${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
-   * is used to convert an element's position to it's offset from the start of the (0-based)
-   * ByteBuf.  Size is inferred by adjacent elements.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class ${minor.class} extends Base {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}.class);
-
-    protected final MutableUInt${type.width} lengthVector;
-
-    public ${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      this.lengthVector = new MutableUInt${type.width}(null, allocator);
-    }
-
-    public byte[] get(int index) {
-      checkArgument(index >= 0);
-      int startIdx = 0;
-      int size = 0;
-      if (index == 0) {
-        size = lengthVector.get(1);
-      } else {
-        startIdx = lengthVector.get(index);
-        size = lengthVector.get(index + 1) - startIdx;
-      }
-      checkState(size >= 0);
-      byte[] dst = new byte[size];
-      data.getBytes(startIdx, dst, 0, size);
-      return dst;
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return lengthVector.getAllocatedSize() + totalBytes;
-    }
-
-    /**
-     * Get the size requirement (in bytes) for the given number of values.  Only accurate
-     * for fixed width value vectors.
-     */
-    public int getSizeFromCount(int valueCount) {
-      return valueCount * ${type.width};
-    }
-
-    @Override
-    protected void clear() {
-      super.clear();
-      lengthVector.clear();
-    }
-
-    /**
-     * 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 totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      lengthVector.allocateNew(valueCount);
-    }
-
-    @Override
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{lengthVector.data, data};
-    }
-
-    public Object getObject(int index) {
-      return get(index);
-    }
-
-    public Mutable${minor.class} getMutable() {
-      return (Mutable${minor.class})this;
-    }
-  }
-
-  /**
-   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
-   * is used to convert an element's position to it's offset from the start of the (0-based)
-   * ByteBuf.  Size is inferred by adjacent elements.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivilent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Mutable${minor.class} extends ${minor.class} {
-
-    public Mutable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-    }
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     */
-    public void set(int index, byte[] bytes) {
-      checkArgument(index >= 0);
-      if (index == 0) {
-        lengthVector.set(0, 0);
-        lengthVector.set(1, bytes.length);
-        data.setBytes(0, bytes);
-      }
-      else {
-        int currentOffset = lengthVector.get(index);
-        // set the end offset of the buffer
-        lengthVector.set(index + 1, currentOffset + bytes.length);
-        data.setBytes(currentOffset, bytes);
-      }
-    }
-
-    @Override
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      lengthVector.setRecordCount(recordCount);
-    }
-
-  }
-
-  </#if> <#-- type.major -->
-
-  /**
-   * 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
-   * from the base class (if not null).
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public static class Nullable${minor.class} extends Mutable${minor.class} {
-
-    protected MutableBit bits;
-
-    public Nullable${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      bits = new MutableBit(null, allocator);
-    }
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     */
-    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
-      setNotNull(index);
-      super.set(index, value);
-    }
-
-    /**
-     * Get the element at the specified position.
-     *
-     * @param   index   position of the value
-     * @return  value of the element, if not null
-     * @throws  NullValueException if the value is null
-     */
-    public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
-      if (isNull(index))
-        throw new NullValueException(index);
-      return super.get(index);
-    }
-
-    public void setNull(int index) {
-      bits.set(index, 0);
-    }
-
-    private void setNotNull(int index) {
-      bits.set(index, 1);
-    }
-
-    public boolean isNull(int index) {
-      return bits.get(index) == 0;
-    }
-
-    /**
-     * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
-     *
-     * @param valueCount   The number of values which may be contained by this vector.
-     */
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      bits.allocateNew(valueCount);
-    }
-
-    @Override
-    public int getAllocatedSize() {
-      return bits.getAllocatedSize() + super.getAllocatedSize();
-    }
-
-    /**
-     * 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 getSizeFromCount(valueCount) + bits.getSizeFromCount(valueCount);
-    }
-
-    @Override
-    public MaterializedField getField() {
-      return field;
-    }
-
-    @Override
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{bits.data, super.data};
-    }
-
-    @Override
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      bits.setRecordCount(recordCount);
-    }
-
-    @Override
-    public Object getObject(int index) {
-      return isNull(index) ? null : super.getObject(index);
-    }
-  }
-
-  /**
-   * Repeated${minor.class} implements a vector with multple values per row (e.g. JSON array or
-   * repeated protobuf field).  The implementation uses two additional value vectors; one to convert
-   * the index offset to the underlying element offset, and another to store the number of values
-   * in the vector.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-   public static class Repeated${minor.class} extends Mutable${minor.class} {
-
-    private MutableUInt4 countVector;    // number of repeated elements in each record
-    private MutableUInt4 offsetVector;   // offsets to start of each record
-
-    public Repeated${minor.class}(MaterializedField field, BufferAllocator allocator) {
-      super(field, allocator);
-      countVector = new MutableUInt4(null, allocator);
-      offsetVector = new MutableUInt4(null, allocator);
-    }
-
-    public void allocateNew(int totalBytes, ByteBuf sourceBuffer, int valueCount) {
-      super.allocateNew(totalBytes, sourceBuffer, valueCount);
-      countVector.allocateNew(valueCount);
-      offsetVector.allocateNew(valueCount);
-    }
-
-    /**
-     * Add an element to the given record index.  This is similar to the set() method in other
-     * value vectors, except that it permits setting multiple values for a single record.
-     *
-     * @param index   record of the element to add
-     * @param value   value to add to the given row
-     */
-    public void add(int index, <#if (type.width > 4)> ${minor.javaType!type.javaType}
-                               <#elseif type.major == "VarLen"> byte[]
-                               <#else> int
-                               </#if> value) {
-      countVector.set(index, countVector.get(index) + 1);
-      offsetVector.set(index, offsetVector.get(index - 1) + countVector.get(index-1));
-      super.set(offsetVector.get(index), value);
-    }
-
-    /**
-     * Get a value for the given record.  Each element in the repeated field is accessed by
-     * the positionIndex param.
-     *
-     * @param  index           record containing the repeated field
-     * @param  positionIndex   position within the repeated field
-     * @return element at the given position in the given record
-     */
-    public <#if type.major == "VarLen">byte[]
-           <#else>${minor.javaType!type.javaType}
-           </#if> get(int index, int positionIndex) {
-
-      assert positionIndex < countVector.get(index);
-      return super.get(offsetVector.get(index) + positionIndex);
-    }
-
-    public MaterializedField getField() {
-      return field;
-    }
-
-    /**
-     * 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 getSizeFromCount(valueCount) +
-             countVector.getSizeFromCount(valueCount) +
-             offsetVector.getSizeFromCount(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 super.getAllocatedSize() +
-             countVector.getAllocatedSize() +
-             offsetVector.getAllocatedSize();
-    }
-
-    /**
-     * Get the elements at the given index.
-     */
-    public int getCount(int index) {
-      return countVector.get(index);
-    }
-
-    public void setRecordCount(int recordCount) {
-      super.setRecordCount(recordCount);
-      offsetVector.setRecordCount(recordCount);
-      countVector.setRecordCount(recordCount);
-    }
-
-    public ByteBuf[] getBuffers() {
-      return new ByteBuf[]{countVector.data, offsetVector.data, data};
-    }
-
-    public Object getObject(int index) {
-      return data.slice(index, getSizeFromCount(countVector.get(index)));
-    }
-
-  }
- </#list>
-</#list>
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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
new file mode 100644
index 0000000..954836a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/VariableLengthVectors.java
@@ -0,0 +1,152 @@
+<@pp.dropOutputFile />
+<#list types as type>
+<#list type.minor as minor>
+
+<#if type.major == "VarLen">
+<@pp.changeOutputFile name="${minor.class}Vector.java" />
+package org.apache.drill.exec.vector;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
+ * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+ * is used to convert an element's position to it's offset from the start of the (0-based)
+ * ByteBuf.  Size is inferred by adjacent elements.
+ *   The width of each element is ${type.width} byte(s)
+ *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+@SuppressWarnings("unused")
+public final class ${minor.class}Vector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
+
+  private final UInt${type.width}Vector lengthVector;
+  private final UInt${type.width}Vector.Mutator lengthVectorMutator;
+
+  public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
+    super(field, allocator);
+    this.lengthVector = new UInt${type.width}Vector(null, allocator);
+    this.lengthVectorMutator = lengthVector.getMutator();
+  }
+
+  public byte[] get(int index) {
+    checkArgument(index >= 0);
+    int startIdx = 0;
+    int size = 0;
+    if (index == 0) {
+      size = lengthVector.get(1);
+    } else {
+      startIdx = lengthVector.get(index);
+      size = lengthVector.get(index + 1) - startIdx;
+    }
+    checkState(size >= 0);
+    byte[] dst = new byte[size];
+    data.getBytes(startIdx, dst, 0, size);
+    return dst;
+  }
+
+  @Override
+  public int getAllocatedSize() {
+    return lengthVector.getAllocatedSize() + totalBytes;
+  }
+
+  /**
+   * Get the size requirement (in bytes) for the given number of values.  Only accurate
+   * for fixed width value vectors.
+   */
+  public int getSizeFromCount(int valueCount) {
+    return valueCount * ${type.width};
+  }
+
+  @Override
+  protected void clear() {
+    super.clear();
+    lengthVector.clear();
+  }
+
+  /**
+   * 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 totalBytes, ByteBuf sourceBuffer, int valueCount) {
+    super.allocateNew(totalBytes, sourceBuffer, valueCount);
+    lengthVector.allocateNew(valueCount);
+  }
+
+  @Override
+  public ByteBuf[] getBuffers() {
+    return new ByteBuf[]{lengthVector.data, data};
+  }
+
+  public Object getObject(int index) {
+    return get(index);
+  }
+
+  public Mutator getMutator() {
+    return new Mutator();
+  }
+  
+  
+  /**
+   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
+   * are accessed by position from the logical start of the vector.  A fixed width lengthVector
+   * is used to convert an element's position to it's offset from the start of the (0-based)
+   * ByteBuf.  Size is inferred by adjacent elements.
+   *   The width of each element is ${type.width} byte(s)
+   *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+   *
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   */
+  public class Mutator implements ValueVector.Mutator{
+
+    /**
+     * Set the variable length element at the specified index to the supplied byte array.
+     *
+     * @param index   position of the bit to set
+     * @param bytes   array of bytes to write
+     */
+    public void set(int index, byte[] bytes) {
+      checkArgument(index >= 0);
+      if (index == 0) {
+        lengthVectorMutator.set(0, 0);
+        lengthVectorMutator.set(1, bytes.length);
+        data.setBytes(0, bytes);
+      } else {
+        int currentOffset = lengthVector.get(index);
+        // set the end offset of the buffer
+        lengthVectorMutator.set(index + 1, currentOffset + bytes.length);
+        data.setBytes(currentOffset, bytes);
+      }
+    }
+
+    @Override
+    public void setRecordCount(int recordCount) {
+      ${minor.class}Vector.this.setRecordCount(recordCount);
+      lengthVector.setRecordCount(recordCount);
+    }
+
+    @Override
+    public void randomizeData(){}
+  }
+  
+}
+
+
+</#if> <#-- type.major -->
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 65584db..cd3371d 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
@@ -27,9 +27,9 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class MockRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
@@ -37,7 +37,7 @@ public class MockRecordReader implements RecordReader {
   private OutputMutator output;
   private MockScanEntry config;
   private FragmentContext context;
-  private ValueVector.Base[] valueVectors;
+  private ValueVector[] valueVectors;
   private int recordsRead;
 
   public MockRecordReader(FragmentContext context, MockScanEntry config) {
@@ -53,12 +53,12 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector.Base getVector(int fieldId, String name, MajorType type, int length) {
+  private ValueVector getVector(int fieldId, String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
     if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
     
     MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
-    ValueVector.Base v;
+    ValueVector v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     v.allocateNew(length);
     return v;
@@ -70,7 +70,7 @@ public class MockRecordReader implements RecordReader {
     try {
       this.output = output;
       int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector.Base[config.getTypes().length];
+      valueVectors = new ValueVector[config.getTypes().length];
       int batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
@@ -88,10 +88,11 @@ public class MockRecordReader implements RecordReader {
   public int next() {
     int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
     recordsRead += recordSetSize;
-    for(ValueVector.Base v : valueVectors){
+    for(ValueVector v : valueVectors){
       logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
       v.randomizeData();
-      v.setRecordCount(recordSetSize);
+      
+      v.getMutator().setRecordCount(recordSetSize);
     }
     return recordSetSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
index b821d6e..bfc19af 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -24,10 +24,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
 import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
index 4a3a606..40227e5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
-import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.vector.TypeHelper;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
index b7b7d93..6440d98 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -22,7 +22,7 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.vector.SelectionVector;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public abstract class FilterRecordBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
@@ -58,7 +58,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index e96c1be..b3b9f5f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -18,11 +18,10 @@
 package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException ;
+  public void addField(int fieldId, ValueVector vector) throws SchemaChangeException ;
   public void setNewSchema() throws SchemaChangeException ;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 822d828..2f3e1fe 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -17,27 +17,21 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import io.netty.buffer.ByteBuf;
-
 import java.util.Iterator;
-import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
-import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.InvalidValueAccessor;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.SchemaBuilder;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-import com.google.common.collect.Lists;
 
 /**
  * Record batch used for a particular scan. Operators against one or more
@@ -45,7 +39,7 @@ import com.google.common.collect.Lists;
 public class ScanBatch implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
 
-  private IntObjectOpenHashMap<ValueVector.Base> fields = new IntObjectOpenHashMap<ValueVector.Base>();
+  private IntObjectOpenHashMap<ValueVector> fields = new IntObjectOpenHashMap<ValueVector>();
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -89,9 +83,9 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector.Base>() {
+    fields.forEach(new IntObjectProcedure<ValueVector>() {
       @Override
-      public void apply(int key, ValueVector.Base value) {
+      public void apply(int key, ValueVector value) {
         value.close();
       }
     });
@@ -99,9 +93,9 @@ public class ScanBatch implements RecordBatch {
 
   @SuppressWarnings("unchecked")
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
-    ValueVector.Base vector = this.fields.lget();
+    ValueVector vector = this.fields.lget();
     if (vector.getClass().isAssignableFrom(clazz)) {
       return (T) vector;
     } else {
@@ -143,14 +137,14 @@ public class ScanBatch implements RecordBatch {
     
     public void removeField(int fieldId) throws SchemaChangeException {
       schemaChanged();
-      ValueVector.Base v = fields.remove(fieldId);
+      ValueVector v = fields.remove(fieldId);
       if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
       v.close();
     }
 
-    public void addField(int fieldId, ValueVector.Base vector) {
+    public void addField(int fieldId, ValueVector vector) {
       schemaChanged();
-      ValueVector.Base v = fields.put(fieldId, vector);
+      ValueVector v = fields.put(fieldId, vector);
       vector.getField();
       builder.addField(vector.getField());
       if (v != null) v.close();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
index be32d1f..fcbd272 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -17,8 +17,6 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.impl;
 
-import java.util.Iterator;
-
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
@@ -29,8 +27,7 @@ import org.apache.drill.exec.record.RawFragmentBatchProvider;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.RecordBatch.IterOutcome;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class WireRecordBatch implements RecordBatch{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
@@ -68,7 +65,7 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return batchLoader.getValueVector(fieldId, clazz);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 05b1cc7..b26e742 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
@@ -20,8 +20,6 @@ package org.apache.drill.exec.record;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.exec.record.vector.ValueVector;
-
 
 public class BatchSchema implements Iterable<MaterializedField> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index 042c40c..c6b7888 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.record;
 
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 /**
  * A record batch contains a set of field values for a particular range of records. In the case of a record batch
@@ -68,7 +68,7 @@ public interface RecordBatch {
   public void kill();
 
 
-  public abstract <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
+  public abstract <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
 
 //  public abstract void getDictReader(int fieldId, Class<?> clazz) throws InvalidValueAccessor;
 //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index ea1de73..be43026 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -27,16 +27,16 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.record.vector.TypeHelper;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
 
-public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.Base>>{
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private IntObjectOpenHashMap<ValueVector.Base> vectors = new IntObjectOpenHashMap<ValueVector.Base>();
+  private IntObjectOpenHashMap<ValueVector> vectors = new IntObjectOpenHashMap<ValueVector>();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -61,14 +61,14 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
     
-    IntObjectOpenHashMap<ValueVector.Base> newVectors = new IntObjectOpenHashMap<ValueVector.Base>();
+    IntObjectOpenHashMap<ValueVector> newVectors = new IntObjectOpenHashMap<ValueVector>();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector.Base v = vectors.remove(fieldDef.getFieldId());
+      ValueVector v = vectors.remove(fieldDef.getFieldId());
       if (v != null) {
         if (v.getField().getDef().equals(fieldDef)) {
           v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
@@ -89,7 +89,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     
     if(!vectors.isEmpty()){
       schemaChanged = true;
-      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
+      for(IntObjectCursor<ValueVector> cursor : newVectors){
         cursor.value.close();
       }
       
@@ -98,7 +98,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
     if(schemaChanged){
       // rebuild the schema.
       SchemaBuilder b = BatchSchema.newBuilder();
-      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
+      for(IntObjectCursor<ValueVector> cursor : newVectors){
         b.addField(cursor.value.getField());
       }
       b.setSelectionVector(false);
@@ -110,8 +110,8 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
   }
 
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    ValueVector.Base v = vectors.get(fieldId);
+  public <T extends ValueVector> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    ValueVector v = vectors.get(fieldId);
     assert v != null;
     if (v.getClass() != clazz)
       throw new InvalidValueAccessor(String.format(
@@ -130,7 +130,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.B
   }
 
   @Override
-  public Iterator<IntObjectCursor<ValueVector.Base>> iterator() {
+  public Iterator<IntObjectCursor<ValueVector>> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/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 67c6cb9..a367b6d 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
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.procedures.IntObjectProcedure;
@@ -58,7 +58,7 @@ public class WritableBatch {
     return buffers;
   }
   
-//  public static WritableBatch get(ValueVector.Base[] vectors){
+//  public static WritableBatch get(ValueVector[] vectors){
 //    WritableCreator c = new WritableCreator();
 //    for(int i =0; i < vectors.length; i++){
 //      c.apply(i, vectors[i]);
@@ -67,14 +67,14 @@ public class WritableBatch {
 //  }
 //  
   
-  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector.Base> fields){
+  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector> fields){
     WritableCreator creator = new WritableCreator(recordCount);
     fields.forEach(creator);
     return creator.get();
     
   }
   
-  private static class WritableCreator implements IntObjectProcedure<ValueVector.Base>{
+  private static class WritableCreator implements IntObjectProcedure<ValueVector>{
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
@@ -87,7 +87,7 @@ public class WritableBatch {
     }
     
     @Override
-    public void apply(int key, ValueVector.Base value) {
+    public void apply(int key, ValueVector value) {
       metadata.add(value.getMetadata());
       for(ByteBuf b : value.getBuffers()){
         buffers.add(b);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index e79a525..02b75ce 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -17,19 +17,25 @@
  ******************************************************************************/
 package org.apache.drill.exec.record.vector;
 
-import io.netty.buffer.ByteBufAllocator;
-
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
  * Convenience/Clarification Fixed2 wrapper.
  */
-public class SelectionVector extends ValueVector.UInt2 {
+public class SelectionVector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
 
   public SelectionVector(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator);
+
+  }
+
+  public int capacity() {
+    return -1;
+  }
+
+  public void allocateNew(int count) {
+
   }
 
 }