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

[5/7] git commit: Create new generated value vectors utilizing fmpp. Includes: - First pass; integrate build system and some cursory implementations - starting to split common logic into base class - implement most of varlen value vector functionality,

Create new generated value vectors utilizing fmpp.  Includes:
- First pass; integrate build system and some cursory implementations
- starting to split common logic into base class
- implement most of varlen value vector functionality, minor cleanup of tdd tags
- added nullable derived class
- Merge changes from JA, minor format cleanup.
- minor fix and cleanup
- added bit vector, removed widthInBits which also allowed removal of FixedBase ctor
- apply TC's fix for resetAllocation()
- added repeated value vectors
- Hooked up templated ValueVectors to codebase.  Removed old ValueVector classes.  Cleanup.
- fix repeated get() and add()
- added some value vector tests.  fixed bugs in VV and some call sites.  generated TypeHelper from FMPP template.  removed unused VV methods
- made base immutable, some debugging
- split mutable/immutable basic VV types. minor refactoring
- fix several allocation bugs
- fix various bugs, only JSONRecordReader test is failing
- fix nullable bit value vector
- make bit vectors use ints to represent the bit value
- remove superfluous logging
- fix value vector getters and setter
- comments and cleanup
- temp disable repeated map JSONReader test
- formatting
- whitespace cleanups


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

Branch: refs/heads/execwork
Commit: 7075cca1f1be45b876ef846762f13d0780627c3a
Parents: 5a5d07f
Author: Ben Becker <be...@gmail.com>
Authored: Tue Jun 18 17:36:11 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jul 15 11:36:32 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/exec/java-exec/pom.xml        |  20 +
 .../src/main/codegen/ValueVectors/config.fmpp   |   3 +
 .../ValueVectors/data/ValueVectorTypes.tdd      | 106 +++
 .../ValueVectors/templates/TypeHelper.java      | 119 +++
 .../ValueVectors/templates/ValueVector.java     | 768 +++++++++++++++++++
 .../exec/physical/config/MockRecordReader.java  |  15 +-
 .../exec/physical/impl/FilterRecordBatch.java   |   2 +-
 .../drill/exec/physical/impl/OutputMutator.java |   2 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  16 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   2 +-
 .../exec/physical/impl/SingleSenderCreator.java |   2 +-
 .../exec/physical/impl/WireRecordBatch.java     |   4 +-
 .../drill/exec/record/MaterializedField.java    |   5 -
 .../apache/drill/exec/record/RecordBatch.java   |   2 +-
 .../drill/exec/record/RecordBatchLoader.java    |  22 +-
 .../apache/drill/exec/record/WritableBatch.java |   9 +-
 .../record/vector/AbstractFixedValueVector.java |  77 --
 .../drill/exec/record/vector/AnyVector.java     |  30 -
 .../exec/record/vector/BaseValueVector.java     | 157 ----
 .../apache/drill/exec/record/vector/Bit.java    | 168 ----
 .../apache/drill/exec/record/vector/Fixed1.java |  43 --
 .../drill/exec/record/vector/Fixed12.java       |  35 -
 .../drill/exec/record/vector/Fixed16.java       |  37 -
 .../apache/drill/exec/record/vector/Fixed2.java |  53 --
 .../apache/drill/exec/record/vector/Fixed4.java |  59 --
 .../apache/drill/exec/record/vector/Fixed8.java |  58 --
 .../drill/exec/record/vector/FixedLen.java      |  45 --
 .../exec/record/vector/NullValueException.java  |   9 +
 .../drill/exec/record/vector/NullableBit.java   |  20 -
 .../exec/record/vector/NullableFixed4.java      |  44 --
 .../exec/record/vector/NullableValueVector.java | 108 ---
 .../exec/record/vector/NullableVarLen4.java     |  21 -
 .../drill/exec/record/vector/RepeatMap.java     |  57 --
 .../exec/record/vector/SelectionVector.java     |   2 +-
 .../drill/exec/record/vector/TypeHelper.java    | 259 -------
 .../drill/exec/record/vector/ValueVector.java   | 118 ---
 .../drill/exec/record/vector/VarLen1.java       |  36 -
 .../drill/exec/record/vector/VarLen2.java       |  36 -
 .../drill/exec/record/vector/VarLen4.java       |  48 --
 .../exec/record/vector/VariableVector.java      | 102 ---
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |   2 +-
 .../drill/exec/store/JSONRecordReader.java      |  30 +-
 .../apache/drill/exec/store/VectorHolder.java   |   6 +-
 .../work/AbstractFragmentRunnerListener.java    |   3 +-
 .../apache/drill/exec/work/FragmentRunner.java  |   3 +-
 .../physical/impl/TestSimpleFragmentRun.java    |  90 ++-
 .../exec/record/vector/TestValueVector.java     | 252 ++++++
 .../drill/exec/store/JSONRecordReaderTest.java  |  36 +-
 48 files changed, 1400 insertions(+), 1741 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 f5ece33..4e4df95 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -167,6 +167,26 @@
 					</execution>
 				</executions>
 			</plugin>
+			<plugin>
+				<groupId>com.googlecode.fmpp-maven-plugin</groupId>
+				<artifactId>fmpp-maven-plugin</artifactId>
+				<version>1.0</version>
+				<configuration>
+					<cfgFile>src/main/codegen/ValueVectors/config.fmpp</cfgFile>
+					<outputDirectory>target/generated-sources/org/apache/drill/exec/record/vector</outputDirectory>
+					<templateDirectory>src/main/codegen/ValueVectors/templates</templateDirectory>
+				</configuration>
+				<executions>
+					<execution>
+						<id>generate-sources</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>generate</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
 			<!-- <plugin> -->
 			<!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
 			<!-- <artifactId>protobuf-maven-plugin</artifactId> -->

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
new file mode 100644
index 0000000..da05e2d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/config.fmpp
@@ -0,0 +1,3 @@
+data: {
+    tdd(../data/ValueVectorTypes.tdd)
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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
new file mode 100644
index 0000000..42153b6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/data/ValueVectorTypes.tdd
@@ -0,0 +1,106 @@
+{
+  types: [
+    {
+      major: "Fixed",
+      width: 1,
+      javaType: "byte",
+      minor: [
+        { class: "TinyInt" },
+        { class: "UInt1" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 2,
+      javaType: "short",
+      minor: [
+        { class: "SmallInt" },
+        { class: "UInt2" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 4,
+      javaType: "int",
+      minor: [
+        { class: "Int" },
+        { class: "UInt4" },
+        { class: "Decimal4", maxPrecisionDigits: 8, scale: 1, javaType: "float"},
+        { class: "Float4", javaType: "float" },
+        { class: "Date" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 8,
+      javaType: "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: "Time" },
+        { class: "TimeStamp" },
+        { class: "DateTime" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 12,
+      javaType: "ByteBuf",
+      minor: [
+        { class: "Decimal12", maxPrecisionDigits: 28, scale: 1},
+        { class: "TimeTZ" },
+        { class: "Interval" }
+      ]
+    },
+    {
+      major: "Fixed",
+      width: 16,
+      javaType: "ByteBuf",
+      minor: [
+        { class: "Decimal16", maxPrecisionDigits: 37, scale: 1}
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 1,
+      javaType: "byte",
+      minor: [
+        { class: "VarBinary1" },
+        { class: "VarChar1" }
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 2,
+      javaType: "short",
+      minor: [
+        { class: "VarBinary2" },
+        { class: "VarChar2" },
+        { class: "Proto2" },
+        { class: "MsgPack2" }
+      ]
+    },
+    {
+      major: "VarLen",
+      width: 4,
+      javaType: "int",
+      minor: [
+        { class: "VarBinary4" },
+        { class: "VarChar4" }
+        { class: "Proto4" },
+        { class: "MsgPack4" }
+      ]
+    },
+    {
+      major: "Bit",
+      width: 1,
+      javaType: "int",
+      minor: [
+        { class: "Bit" }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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
new file mode 100644
index 0000000..8dfd3af
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/TypeHelper.java
@@ -0,0 +1,119 @@
+/*******************************************************************************
+ * 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 org.apache.drill.exec.memory.BufferAllocator;
+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.MaterializedField;
+
+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;
+
+  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>
+  </#list>
+</#list>
+      case BOOLEAN: return 1;
+      case FIXEDCHAR: return major.getWidth();
+      case FIXEDBINARY: return major.getWidth();
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
+    switch (type) {
+<#list types as type>
+  <#list type.minor as minor>
+    <#if minor.class == "Bit">
+      case BOOLEAN:
+        switch (mode) {
+          case REQUIRED:
+            return ValueVector.${minor.class}.class;
+          case OPTIONAL:
+            return ValueVector.Nullable${minor.class}.class;
+          case REPEATED:
+            return ValueVector.Repeated${minor.class}.class;
+        }
+    <#else>
+      case ${minor.class?upper_case}:
+        switch (mode) {
+          case REQUIRED:
+            return ValueVector.${minor.class}.class;
+          case OPTIONAL:
+            return ValueVector.Nullable${minor.class}.class;
+          case REPEATED:
+            return ValueVector.Repeated${minor.class}.class;
+        }
+    </#if>
+  </#list>
+</#list>
+    default:
+      break;
+    }
+    throw new UnsupportedOperationException();
+  }
+
+
+  public static ValueVector.Base getNewVector(MaterializedField field, BufferAllocator allocator){
+    MajorType type = field.getType();
+
+    switch (type.getMinorType()) {
+<#list types as type>
+  <#list type.minor as minor>
+    <#if minor.class != "Bit">
+      case ${minor.class?upper_case}:
+        switch (type.getMode()) {
+          case REQUIRED:
+            return new ValueVector.${minor.class}(field, allocator);
+          case OPTIONAL:
+            return new ValueVector.Nullable${minor.class}(field, allocator);
+          case REPEATED:
+            return new ValueVector.Repeated${minor.class}(field, allocator);
+        }
+    </#if>
+  </#list>
+</#list>
+      case BOOLEAN:
+        switch (type.getMode()) {
+          case REQUIRED:
+            return new ValueVector.Bit(field, allocator);
+          case OPTIONAL:
+            return new ValueVector.NullableBit(field, allocator);
+          case REPEATED:
+            return new ValueVector.RepeatedBit(field, allocator);
+        }
+    }
+    // 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/7075cca1/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
new file mode 100644
index 0000000..e0e895b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/codegen/ValueVectors/templates/ValueVector.java
@@ -0,0 +1,768 @@
+/*******************************************************************************
+ * 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/7075cca1/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 6a1eba4..65584db 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
@@ -37,7 +37,7 @@ public class MockRecordReader implements RecordReader {
   private OutputMutator output;
   private MockScanEntry config;
   private FragmentContext context;
-  private ValueVector<?>[] valueVectors;
+  private ValueVector.Base[] valueVectors;
   private int recordsRead;
 
   public MockRecordReader(FragmentContext context, MockScanEntry config) {
@@ -53,16 +53,14 @@ public class MockRecordReader implements RecordReader {
     return x;
   }
 
-  private ValueVector<?> getVector(int fieldId, String name, MajorType type, int length) {
+  private ValueVector.Base 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<?> v;
+    ValueVector.Base v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     v.allocateNew(length);
-    
     return v;
 
   }
@@ -72,7 +70,7 @@ public class MockRecordReader implements RecordReader {
     try {
       this.output = output;
       int estimateRowSize = getEstimatedRecordSize(config.getTypes());
-      valueVectors = new ValueVector<?>[config.getTypes().length];
+      valueVectors = new ValueVector.Base[config.getTypes().length];
       int batchRecordCount = 250000 / estimateRowSize;
 
       for (int i = 0; i < config.getTypes().length; i++) {
@@ -90,7 +88,8 @@ public class MockRecordReader implements RecordReader {
   public int next() {
     int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
     recordsRead += recordSetSize;
-    for(ValueVector<?> v : valueVectors){
+    for(ValueVector.Base v : valueVectors){
+      logger.debug("MockRecordReader:  Generating random data for VV of type " + v.getClass().getName());
       v.randomizeData();
       v.setRecordCount(recordSetSize);
     }
@@ -103,7 +102,7 @@ public class MockRecordReader implements RecordReader {
       try {
         output.removeField(valueVectors[i].getField().getFieldId());
       } catch (SchemaChangeException e) {
-        logger.warn("Failure while trying tremove field.", e);
+        logger.warn("Failure while trying to remove field.", e);
       }
       valueVectors[i].close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 6592ca1..b7b7d93 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
@@ -58,7 +58,7 @@ public abstract class FilterRecordBatch implements RecordBatch {
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 ce0cf66..e96c1be 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
@@ -23,6 +23,6 @@ import org.apache.drill.exec.record.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(int fieldId) throws SchemaChangeException;
-  public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
+  public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException ;
   public void setNewSchema() throws SchemaChangeException ;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 33c1e29..822d828 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
@@ -45,7 +45,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<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
+  private IntObjectOpenHashMap<ValueVector.Base> fields = new IntObjectOpenHashMap<ValueVector.Base>();
   private BatchSchema schema;
   private int recordCount;
   private boolean schemaChanged = true;
@@ -89,9 +89,9 @@ public class ScanBatch implements RecordBatch {
   }
 
   private void releaseAssets() {
-    fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
+    fields.forEach(new IntObjectProcedure<ValueVector.Base>() {
       @Override
-      public void apply(int key, ValueVector<?> value) {
+      public void apply(int key, ValueVector.Base value) {
         value.close();
       }
     });
@@ -99,9 +99,9 @@ public class ScanBatch implements RecordBatch {
 
   @SuppressWarnings("unchecked")
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> 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<?> vector = this.fields.lget();
+    ValueVector.Base vector = this.fields.lget();
     if (vector.getClass().isAssignableFrom(clazz)) {
       return (T) vector;
     } else {
@@ -143,14 +143,14 @@ public class ScanBatch implements RecordBatch {
     
     public void removeField(int fieldId) throws SchemaChangeException {
       schemaChanged();
-      ValueVector<?> v = fields.remove(fieldId);
+      ValueVector.Base 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<?> vector) {
+    public void addField(int fieldId, ValueVector.Base vector) {
       schemaChanged();
-      ValueVector<?> v = fields.put(fieldId, vector);
+      ValueVector.Base 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/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index c20538d..3819036 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -73,7 +73,7 @@ public class ScreenCreator implements RootCreator<Screen>{
         stop();
         return false;
       }
-      
+
       IterOutcome outcome = incoming.next();
       logger.debug("Screen Outcome {}", outcome);
       switch(outcome){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index b7d4c7e..a2b9865 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -103,7 +103,7 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
       @Override
       public void success(Ack value) {
         if(value.getOk()) return;
-        
+
         logger.error("Downstream fragment was not accepted.  Stopping future sends.");
         // if we didn't get ack ok, we'll need to kill the query.
         context.fail(new RpcException("A downstream fragment batch wasn't accepted.  This fragment thus fails."));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 b41b0cd..be32d1f 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
@@ -68,7 +68,7 @@ public class WireRecordBatch implements RecordBatch{
   }
 
   @Override
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
     return batchLoader.getValueVector(fieldId, clazz);
   }
 
@@ -76,7 +76,7 @@ public class WireRecordBatch implements RecordBatch{
   public IterOutcome next() {
     RawFragmentBatch batch = fragProvider.getNext();
     try{
-      if(batch == null) return IterOutcome.NONE;
+      if (batch == null) return IterOutcome.NONE;
 
       logger.debug("Next received batch {}", batch);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 09427ef..d1858f1 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
@@ -28,7 +28,6 @@ import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
-import org.apache.drill.exec.record.vector.TypeHelper;
 
 public class MaterializedField implements Comparable<MaterializedField> {
   private final FieldDef def;
@@ -107,10 +106,6 @@ public class MaterializedField implements Comparable<MaterializedField> {
     return def.getMajorType().getMode();
   }
 
-  public Class<?> getValueClass() {
-    return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
-  }
-
   public boolean matches(SchemaPath path) {
     Iterator<NamePart> iter = def.getNameList().iterator();
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 3e4ded2..042c40c 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
@@ -68,7 +68,7 @@ public interface RecordBatch {
   public void kill();
 
 
-  public abstract <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor;
+  public abstract <T extends ValueVector.Base> 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/7075cca1/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 d990198..ea1de73 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
@@ -33,10 +33,10 @@ import org.apache.drill.exec.record.vector.ValueVector;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
 
-public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?>>>{
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector.Base>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
 
-  private IntObjectOpenHashMap<ValueVector<?>> vectors = new IntObjectOpenHashMap<ValueVector<?>>();
+  private IntObjectOpenHashMap<ValueVector.Base> vectors = new IntObjectOpenHashMap<ValueVector.Base>();
   private final BufferAllocator allocator;
   private int recordCount; 
   private BatchSchema schema;
@@ -61,17 +61,17 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
     this.recordCount = def.getRecordCount();
     boolean schemaChanged = false;
     
-    IntObjectOpenHashMap<ValueVector<?>> newVectors = new IntObjectOpenHashMap<ValueVector<?>>();
+    IntObjectOpenHashMap<ValueVector.Base> newVectors = new IntObjectOpenHashMap<ValueVector.Base>();
 
     List<FieldMetadata> fields = def.getFieldList();
     
     int bufOffset = 0;
     for (FieldMetadata fmd : fields) {
       FieldDef fieldDef = fmd.getDef();
-      ValueVector<?> v = vectors.remove(fieldDef.getFieldId());
+      ValueVector.Base v = vectors.remove(fieldDef.getFieldId());
       if (v != null) {
         if (v.getField().getDef().equals(fieldDef)) {
-          v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+          v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
           newVectors.put(fieldDef.getFieldId(), v);
           continue;
         } else {
@@ -83,13 +83,13 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
       schemaChanged = true;
       MaterializedField m = new MaterializedField(fieldDef);
       v = TypeHelper.getNewVector(m, allocator);
-      v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+      v.allocateNew(fmd.getBufferLength(), buf.slice(bufOffset, fmd.getBufferLength()), recordCount);
       newVectors.put(fieldDef.getFieldId(), v);
     }
     
     if(!vectors.isEmpty()){
       schemaChanged = true;
-      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
         cursor.value.close();
       }
       
@@ -98,7 +98,7 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
     if(schemaChanged){
       // rebuild the schema.
       SchemaBuilder b = BatchSchema.newBuilder();
-      for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+      for(IntObjectCursor<ValueVector.Base> cursor : newVectors){
         b.addField(cursor.value.getField());
       }
       b.setSelectionVector(false);
@@ -110,8 +110,8 @@ public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?
   }
 
   @SuppressWarnings("unchecked")
-  public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
-    ValueVector<?> v = vectors.get(fieldId);
+  public <T extends ValueVector.Base> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+    ValueVector.Base 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<?
   }
 
   @Override
-  public Iterator<IntObjectCursor<ValueVector<?>>> iterator() {
+  public Iterator<IntObjectCursor<ValueVector.Base>> iterator() {
     return this.vectors.iterator();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/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 788c731..67c6cb9 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
@@ -58,7 +58,7 @@ public class WritableBatch {
     return buffers;
   }
   
-//  public static WritableBatch get(ValueVector<?>[] vectors){
+//  public static WritableBatch get(ValueVector.Base[] 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<?>> fields){
+  public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector.Base> fields){
     WritableCreator creator = new WritableCreator(recordCount);
     fields.forEach(creator);
     return creator.get();
     
   }
   
-  private static class WritableCreator implements IntObjectProcedure<ValueVector<?>>{
+  private static class WritableCreator implements IntObjectProcedure<ValueVector.Base>{
     
     List<ByteBuf> buffers = Lists.newArrayList();
     List<FieldMetadata> metadata = Lists.newArrayList();
@@ -87,7 +87,7 @@ public class WritableBatch {
     }
     
     @Override
-    public void apply(int key, ValueVector<?> value) {
+    public void apply(int key, ValueVector.Base value) {
       metadata.add(value.getMetadata());
       for(ByteBuf b : value.getBuffers()){
         buffers.add(b);
@@ -97,7 +97,6 @@ public class WritableBatch {
       value.allocateNew(value.capacity());
     }
 
-
     public WritableBatch get(){
       RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
       WritableBatch b = new WritableBatch(batchDef, buffers);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
deleted file mode 100644
index 8748285..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ /dev/null
@@ -1,77 +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 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.MaterializedField;
-
-/**
- * Abstract class that fixed value vectors are derived from.
- */
-abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> extends BaseValueVector<T> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFixedValueVector.class);
-
-  protected final int widthInBits;
-
-  protected int longWords = 0;
-  
-  public AbstractFixedValueVector(MaterializedField field, BufferAllocator allocator, int widthInBits) {
-    super(field, allocator);
-    this.widthInBits = widthInBits;
-  }
-
-    public int getWidthInBits() {
-        return widthInBits;
-    }
-
-    @Override
-  protected int getAllocationSize(int valueCount) {
-    return (int) Math.ceil(valueCount*widthInBits*1.0/8);
-  }
-  
-  @Override
-  protected void childResetAllocation(int valueCount, ByteBuf buf) {
-    this.longWords = valueCount/8;
-  }
-
-  @Override
-  protected void childCloneMetadata(T other) {
-    other.longWords = this.longWords;
-  }
-
-  @Override
-  protected void childClear() {
-    longWords = 0;
-  }
-
-  @Override
-  public void setRecordCount(int recordCount) {
-    this.data.writerIndex(recordCount*(widthInBits/8));
-    super.setRecordCount(recordCount);
-  }
-
-
-
-
-
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
deleted file mode 100644
index 6becfcd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AnyVector.java
+++ /dev/null
@@ -1,30 +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;
-
-/**
- * A Vector that holds each value with type information. In the case of complex types, an any vector will inline the
- * complex type within the value space.  The complex value will be stored in 
- */
-public class AnyVector {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnyVector.class);
-
-  
-   
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
deleted file mode 100644
index 69cd628..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ /dev/null
@@ -1,157 +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 io.netty.buffer.ByteBuf;
-
-import java.util.Random;
-
-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;
-
-public abstract class BaseValueVector<T extends BaseValueVector<T>> implements ValueVector<T>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
-  
-  protected final BufferAllocator allocator;
-  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
-  protected int maxValueCount = 0;
-  protected final MaterializedField field;
-  private int recordCount;
-  
-  public BaseValueVector(MaterializedField field, BufferAllocator allocator) {
-    this.allocator = allocator;
-    this.field = field;
-  }
-
-  public final void allocateNew(int valueCount){
-    int allocationSize = getAllocationSize(valueCount);
-    ByteBuf newBuf = allocator.buffer(allocationSize);
-    resetAllocation(valueCount, newBuf);
-  }
-
-  protected abstract int getAllocationSize(int maxValueCount);
-  protected abstract void childResetAllocation(int valueCount, ByteBuf buf);
-  protected abstract void childCloneMetadata(T other);
-  protected abstract void childClear();
-  
-  /**
-   * Update the current buffer allocation utilize the provided allocation.
-   * @param maxValueCount
-   * @param buf
-   */
-  protected final void resetAllocation(int maxValueCount, ByteBuf buf){
-    clear();
-    buf.retain();
-    this.maxValueCount = maxValueCount;
-    this.data = buf;
-    childResetAllocation(maxValueCount, buf);
-  }
-  
-  public final void cloneMetadata(T other){
-    other.maxValueCount = this.maxValueCount;
-  }
-  
-  
-  @Override
-  public final void cloneInto(T vector) {
-    vector.allocateNew(maxValueCount);
-    data.writeBytes(vector.data);
-    cloneMetadata(vector);
-    childResetAllocation(maxValueCount, vector.data);
-  }
-  
-  @Override
-  public final void transferTo(T vector) {
-    vector.data = this.data;
-    cloneMetadata(vector);
-    childResetAllocation(maxValueCount, data);
-    clear();
-  }
-
-  protected final void clear(){
-    if(this.data != DeadBuf.DEAD_BUFFER){
-      this.data.release();
-      this.data = DeadBuf.DEAD_BUFFER;
-      this.maxValueCount = 0;
-    }
-    childClear();
-  }
-  
-  /**
-   * Give the length of the value vector in bytes.
-   * 
-   * @return
-   */
-  public int capacity() {
-    return maxValueCount;
-  }
-  
-  @Override
-  public void close() {
-    clear();
-  }
-
-  @Override
-  public ByteBuf[] getBuffers() {
-    return new ByteBuf[]{data};
-  }
-  
-  public MaterializedField getField(){
-    return field;
-  }
-  
-  
-  public int getRecordCount() {
-    return recordCount;
-  }
-
-  public void setRecordCount(int recordCount) {
-    this.recordCount = recordCount;
-  }
-
-  @Override
-  public FieldMetadata getMetadata() {
-    int len = 0;
-    for(ByteBuf b : getBuffers()){
-      len += b.writerIndex();
-    }
-    return FieldMetadata.newBuilder().setDef(getField().getDef()).setValueCount(getRecordCount()).setBufferLength(len).build();
-  }
-  
-  @Override
-  public void setTo(FieldMetadata metadata, ByteBuf data) {
-//    logger.debug("Updating value vector to {}, {}", metadata, data);
-    clear();
-    resetAllocation(metadata.getValueCount(), data);
-  }
-
-  @Override
-  public void randomizeData() {
-    if(this.data != DeadBuf.DEAD_BUFFER){
-      Random r = new Random();
-      for(int i =0; i < data.capacity()-8; i+=8){
-        data.setLong(i, r.nextLong());
-      }
-    }
-    
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
deleted file mode 100644
index 533e3bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
+++ /dev/null
@@ -1,168 +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 org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.RecordField.ValueMode;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.hadoop.io.SequenceFile;
-
-/**
- * Describes a vector which holds a number of true/false values.
- */
-public class Bit extends AbstractFixedValueVector<Bit> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
-
-  public Bit(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 1);
-  }
-
-  
-//  /** Returns true or false for the specified bit index.
-//   * The index should be less than the OpenBitSet size
-//   */
-//  public boolean get(int index) {
-//    assert index >= 0 && index < this.valueCount;
-//    int i = index >> 3;               // div 8
-//    // signed shift will keep a negative index and force an
-//    // array-index-out-of-bounds-exception, removing the need for an explicit check.
-//    int bit = index & 0x3f;           // mod 64
-//    long bitmask = 1L << bit;
-//    return (data.getLong(i) & bitmask) != 0;
-//  }
-  
-  public int getBit(int index) {
-    
-    assert index >= 0 && index < this.maxValueCount;
-    int i = 8*(index >> 6); // div 8
-    int bit = index & 0x3f; // mod 64
-    return ((int) (data.getLong(i) >>> bit)) & 0x01;
-  }
-  
-  /** Sets the bit at the specified index.
-   * The index should be less than the OpenBitSet size.
-   */
-   public void set(int index) {
-     assert index >= 0 && index < this.maxValueCount;
-     int wordNum = index >> 3;   
-     int bit = index & 0x3f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) | bitmask);
-   }
-   
-   public void clear(int index) {
-     assert index >= 0 && index < this.maxValueCount;
-     int wordNum = index >> 3;
-     int bit = index & 0x03f;
-     long bitmask = 1L << bit;
-     data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
-   }
-   
-   
-   
-   /** Clears a range of bits.  Clearing past the end does not change the size of the set.
-   *
-   * @param startBitIndex lower index
-   * @param lastBitIndex one-past the last bit to clear
-   */
-  private void clear2(int startBitIndex, int lastBitIndex) {
-    if (lastBitIndex <= startBitIndex) return;
-
-    int firstWordStart = (startBitIndex>>3);
-    if (firstWordStart >= this.longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int lastWordStart   = ((lastBitIndex-1)>>3);
-
-    long startmask = -1L << startBitIndex;
-    long endmask = -1L >>> -lastBitIndex;  // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-
-    if (firstWordStart == lastWordStart) {
-      data.setLong(firstWordStart,  data.getLong(firstWordStart) & (startmask | endmask));
-      return;
-    }
-    data.setLong(firstWordStart,  data.getLong(firstWordStart) & startmask);
-
-    int middle = Math.min(this.longWords, lastWordStart);
-    
-    for(int i =firstWordStart+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    if (lastWordStart < this.longWords) {
-      data.setLong(lastWordStart,  data.getLong(lastWordStart) & endmask);
-    }
-  }
-  
-  public void setAllFalse(){
-    clear(0, maxValueCount);
-  }
-
-  
-  public void clear(int startIndex, int endIndex) {
-    if (endIndex <= startIndex) return;
-
-    int startWord = (startIndex >> 6);
-    if (startWord >= longWords) return;
-
-    // since endIndex is one past the end, this is index of the last
-    // word to be changed.
-    int endWord = ((endIndex - 1) >> 6);
-
-    long startmask = -1L << startIndex;
-    long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
-    // invert masks since we are clearing
-    startmask = ~startmask;
-    endmask = ~endmask;
-    
-    int startWordPos = startWord * 8;
-    if (startWord == endWord) {
-      data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
-      return;
-    }
-
-    int endWordPos = endWord * 8;
-
-    data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
-
-    int middle = Math.min(longWords, endWord)*8;
-    
-    
-    for(int i =startWordPos+8; i < middle; i += 8){
-      data.setLong(i, 0L);
-    }
-    
-    if (endWordPos < startWordPos) {
-      data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
-    }
-  }
-
-
-  @Override
-  public Object getObject(int index) {
-    return this.getBit(index);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
deleted file mode 100644
index 82c86d1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
+++ /dev/null
@@ -1,43 +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 org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed1 extends AbstractFixedValueVector<Fixed1>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed1.class);
-
-  public Fixed1(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 8);
-  }
-  
-  public void setByte(int index, byte b){
-    data.setByte(index, b);
-  }
-
-  public byte getByte(int index){
-    return data.getByte(index);
-  }
-  
-  @Override
-  public Object getObject(int index) {
-    return getByte(index);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
deleted file mode 100644
index c5f641a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
+++ /dev/null
@@ -1,35 +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 org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed12 extends AbstractFixedValueVector<Fixed12>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed12.class);
-
-  public Fixed12(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 12*8);
-  }
-
-  
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
deleted file mode 100644
index 649832b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
+++ /dev/null
@@ -1,37 +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 org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed16 extends AbstractFixedValueVector<Fixed16>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed16.class);
-
-  public Fixed16(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 16*8);
-  }
-
-  @Override
-  public Object getObject(int index) {
-    return null;
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7075cca1/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
deleted file mode 100644
index bd0e313..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
+++ /dev/null
@@ -1,53 +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 org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class Fixed2 extends AbstractFixedValueVector<Fixed2>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed2.class);
-
-  public Fixed2(MaterializedField field, BufferAllocator allocator) {
-    super(field, allocator, 2*8);
-  }
-
-  public final void setSmallInt(int index, short value){
-    index*=2;
-    data.setShort(index, value);
-  }
-  
-  public final short getSmallInt(int index){
-    index*=2;
-    return data.getShort(index);
-  }
-  
-  public final void setUInt2(int index, short value){
-    setSmallInt(index, value);
-  }
-  
-  public final short getUInt2(int index){
-    return getSmallInt(index);
-  }
-  
-  @Override
-  public Object getObject(int index) {
-    return getSmallInt(index);
-  }
-  
-}