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

[2/7] Updated value vectors inheritance model. Moved Mutables to separate Mutator subclasses. Broke VVs into separate files rather than one large 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/store/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
index f2f97b7..e637518 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/JSONRecordReader.java
@@ -1,17 +1,15 @@
 package org.apache.drill.exec.store;
 
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.cursors.IntObjectCursor;
-import com.carrotsearch.hppc.cursors.ObjectCursor;
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonToken;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import com.google.common.io.InputSupplier;
-import com.google.common.io.Resources;
+import static com.fasterxml.jackson.core.JsonToken.END_ARRAY;
+import static com.fasterxml.jackson.core.JsonToken.END_OBJECT;
+import static com.fasterxml.jackson.core.JsonToken.FIELD_NAME;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.List;
+
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -21,17 +19,34 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.vector.*;
-import org.apache.drill.exec.schema.*;
+import org.apache.drill.exec.schema.DiffSchema;
+import org.apache.drill.exec.schema.Field;
+import org.apache.drill.exec.schema.IdGenerator;
+import org.apache.drill.exec.schema.ListSchema;
+import org.apache.drill.exec.schema.NamedField;
+import org.apache.drill.exec.schema.ObjectSchema;
+import org.apache.drill.exec.schema.OrderedField;
+import org.apache.drill.exec.schema.RecordSchema;
+import org.apache.drill.exec.schema.SchemaIdGenerator;
 import org.apache.drill.exec.schema.json.jackson.JacksonHelper;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableVarChar4Vector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.ValueVector;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import static com.fasterxml.jackson.core.JsonToken.*;
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+import com.google.common.io.InputSupplier;
+import com.google.common.io.Resources;
 
 public class JSONRecordReader implements RecordReader {
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
@@ -341,42 +356,40 @@ public class JSONRecordReader implements RecordReader {
             switch (minorType) {
                 case INT: {
                     holder.incAndCheckLength(32);
-                    ValueVector.NullableInt int4 = (ValueVector.NullableInt) holder.getValueVector();
-                    if (val == null) {
-                      int4.setNull(index);
-                    } else {
-                      int4.set(index, (Integer) val);
+                    NullableIntVector int4 = (NullableIntVector) holder.getValueVector();
+                    NullableIntVector.Mutator m = int4.getMutator();
+                    if (val != null) {
+                      m.set(index, (Integer) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case FLOAT4: {
                     holder.incAndCheckLength(32);
-                    ValueVector.NullableFloat4 float4 = (ValueVector.NullableFloat4) holder.getValueVector();
-                    if (val == null) {
-                      float4.setNull(index);
-                    } else {
-                      float4.set(index, (Float) val);
+                    NullableFloat4Vector float4 = (NullableFloat4Vector) holder.getValueVector();
+                    NullableFloat4Vector.Mutator m = float4.getMutator();
+                    if (val != null) {
+                      m.set(index, (Float) val);
                     }
                     return holder.hasEnoughSpace(32);
                 }
                 case VARCHAR4: {
                     if (val == null) {
-                        ((ValueVector.NullableVarChar4) holder.getValueVector()).setNull(index);
                         return (index + 1) * 4 <= holder.getLength();
                     } else {
                         byte[] bytes = ((String) val).getBytes(UTF_8);
                         int length = bytes.length;
                         holder.incAndCheckLength(length);
-                        ValueVector.NullableVarChar4 varLen4 = (ValueVector.NullableVarChar4) holder.getValueVector();
-                        varLen4.set(index, bytes);
+                        NullableVarChar4Vector varLen4 = (NullableVarChar4Vector) holder.getValueVector();
+                        NullableVarChar4Vector.Mutator m = varLen4.getMutator();
+                        m.set(index, bytes);
                         return holder.hasEnoughSpace(length);
                     }
                 }
                 case BOOLEAN: {
                     holder.incAndCheckLength(1);
-                    ValueVector.NullableBit bit = (ValueVector.NullableBit) holder.getValueVector();
+                    NullableBitVector bit = (NullableBitVector) holder.getValueVector();
                     if (val != null) {
-                        bit.set(index, (Boolean)val ? 1 : 0);
+                        bit.getMutator().set(index, (Boolean)val ? 1 : 0);
                     }
                     return holder.hasEnoughSpace(1);
                 }
@@ -409,7 +422,7 @@ public class JSONRecordReader implements RecordReader {
             SchemaDefProtos.MajorType type = field.getFieldType();
             int fieldId = field.getFieldId();
             MaterializedField f = MaterializedField.create(new SchemaPath(field.getFieldName()), fieldId, parentFieldId, type);
-            ValueVector.Base v = TypeHelper.getNewVector(f, allocator);
+            ValueVector v = TypeHelper.getNewVector(f, allocator);
             v.allocateNew(batchSize);
             VectorHolder holder = new VectorHolder(batchSize, v);
             valueVectorMap.put(fieldId, holder);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
index 4043913..fa0cbd5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/VectorHolder.java
@@ -18,22 +18,19 @@
 
 package org.apache.drill.exec.store;
 
-import org.apache.drill.exec.memory.BufferAllocator;
-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.vector.ValueVector;
 
 public class VectorHolder {
     private int length;
-    private ValueVector.Base vector;
+    private ValueVector vector;
     private int currentLength;
 
-    VectorHolder(int length, ValueVector.Base vector) {
+    VectorHolder(int length, ValueVector vector) {
         this.length = length;
         this.vector = vector;
     }
 
-    public ValueVector.Base getValueVector() {
+    public ValueVector getValueVector() {
         return vector;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
new file mode 100644
index 0000000..d18a29d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -0,0 +1,123 @@
+package org.apache.drill.exec.vector;
+
+import java.util.Random;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
+/**
+ * Bit implements a vector of bit-width values.  Elements in the vector are accessed
+ * by position from the logical start of the vector.
+ *   The width of each element is 1 bit.
+ *   The equivalent Java primitive is an int containing the value '0' or '1'.
+ *
+ * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+ */
+public final class BitVector extends ValueVector {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
+
+  public BitVector(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 Mutator getMutator() {
+    return new Mutator();
+  }
+
+  /**
+   * 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 class Mutator implements ValueVector.Mutator{
+
+    private Mutator(){}
+    
+    /**
+     * Set the bit at the given index to the specified value.
+     *
+     * @param index   position of the bit to set
+     * @param value   value to set (either 1 or 0)
+     */
+    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 setRecordCount(int recordCount) {
+      BitVector.this.setRecordCount(recordCount);
+    }
+
+    @Override
+    public void randomizeData() {
+      if (data != DeadBuf.DEAD_BUFFER) {
+        Random r = new Random();
+        for (int i = 0; i < data.capacity() - 1; i++) {
+          byte[] bytes = new byte[1];
+          r.nextBytes(bytes);
+          data.setByte(i, bytes[0]);
+        }
+      }
+    }
+  }
+}
\ 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/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
new file mode 100644
index 0000000..718478e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -0,0 +1,192 @@
+/*******************************************************************************
+ * 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.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+
+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;
+
+/**
+ * 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 abstract class ValueVector implements Closeable {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVector.class);
+
+  protected final BufferAllocator allocator;
+  protected ByteBuf data = DeadBuf.DEAD_BUFFER;
+  protected MaterializedField field;
+  protected int recordCount;
+  protected int totalBytes;
+
+  ValueVector(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);
+
+  
+  public abstract Mutator getMutator();
+  
+  /**
+   * 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;
+    }
+  }
+
+  //public abstract <T extends Mutator> T getMutator();
+  
+  /**
+   * Define the number of records that are in this value vector.
+   * @param recordCount Number of records active in this vector.
+   */
+  void setRecordCount(int recordCount) {
+    data.writerIndex(getSizeFromCount(recordCount));
+    this.recordCount = recordCount;
+  }
+
+  /**
+   * For testing only -- randomize the buffer contents
+   */
+  public void randomizeData() { }
+
+  
+  public static interface Mutator{
+    public void randomizeData();
+    public void setRecordCount(int recordCount);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index 3edf283..3fe0622 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -26,10 +26,10 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.vector.ValueVector;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 import com.carrotsearch.hppc.cursors.IntObjectCursor;
@@ -61,7 +61,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       // print headers.
       if (schemaChanged) {
         System.out.println("\n\n========NEW SCHEMA=========\n\n");
-        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
+        for (IntObjectCursor<ValueVector> v : batchLoader) {
 
           if (firstColumn) {
             firstColumn = false;
@@ -80,7 +80,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       for (int i = 0; i < batchLoader.getRecordCount(); i++) {
         boolean first = true;
         recordCount++;
-        for (IntObjectCursor<ValueVector.Base> v : batchLoader) {
+        for (IntObjectCursor<ValueVector> v : batchLoader) {
           if (first) {
             first = false;
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 5924f7d..ae4f644 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -1,19 +1,21 @@
 package org.apache.drill.exec.record.vector;
 
-import io.netty.buffer.ByteBuf;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
-import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.record.MaterializedField;
-
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
-import org.apache.hadoop.io.UTF8;
-import org.junit.Test;
-
 import java.nio.charset.Charset;
 
+import org.apache.drill.exec.memory.DirectBufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableUInt4Vector;
+import org.apache.drill.exec.vector.NullableVarChar2Vector;
+import org.apache.drill.exec.vector.TypeHelper;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.junit.Test;
+
 public class TestValueVector {
 
   DirectBufferAllocator allocator = new DirectBufferAllocator();
@@ -34,15 +36,16 @@ public class TestValueVector {
         MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.MutableUInt4 v = new ValueVector.MutableUInt4(field, allocator);
+    UInt4Vector v = new UInt4Vector(field, allocator);
+    UInt4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -69,16 +72,17 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableVarChar2 v = new ValueVector.NullableVarChar2(field, allocator);
+    NullableVarChar2Vector v = new NullableVarChar2Vector(field, allocator);
+    NullableVarChar2Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Create and set 3 sample strings
     String str1 = new String("AAAAA1");
     String str2 = new String("BBBBBBBBB2");
     String str3 = new String("CCCC3");
-    v.set(0, str1.getBytes(Charset.forName("UTF-8")));
-    v.set(1, str2.getBytes(Charset.forName("UTF-8")));
-    v.set(2, str3.getBytes(Charset.forName("UTF-8")));
+    m.set(0, str1.getBytes(Charset.forName("UTF-8")));
+    m.set(1, str2.getBytes(Charset.forName("UTF-8")));
+    m.set(2, str3.getBytes(Charset.forName("UTF-8")));
 
     // Check the sample strings
     assertEquals(str1, new String(v.get(0), Charset.forName("UTF-8")));
@@ -86,10 +90,16 @@ public class TestValueVector {
     assertEquals(str3, new String(v.get(2), Charset.forName("UTF-8")));
 
     // Ensure null value throws
+    boolean b = false;
     try {
       v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
+    } catch(AssertionError e) { 
+      b = true;
+    }finally{
+      if(!b){
+        assert false;
+      }
+    }
 
   }
 
@@ -110,15 +120,16 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableUInt4 v = new ValueVector.NullableUInt4(field, allocator);
+    NullableUInt4Vector v = new NullableUInt4Vector(field, allocator);
+    NullableUInt4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -126,22 +137,39 @@ public class TestValueVector {
     assertEquals(104, v.get(1023));
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }      
+    }
+
+    
     v.allocateNew(2048);
-    try {
-      v.get(0);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
-    v.set(0, 100);
-    v.set(1, 101);
-    v.set(100, 102);
-    v.set(1022, 103);
-    v.set(1023, 104);
+    {
+      boolean b = false;
+      try {
+        v.get(0);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
+    
+    m.set(0, 100);
+    m.set(1, 101);
+    m.set(100, 102);
+    m.set(1022, 103);
+    m.set(1023, 104);
     assertEquals(100, v.get(0));
     assertEquals(101, v.get(1));
     assertEquals(102, v.get(100));
@@ -149,10 +177,19 @@ public class TestValueVector {
     assertEquals(104, v.get(1023));
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
+    
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
 
   }
 
@@ -172,16 +209,16 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.NullableFloat4 v = (ValueVector.NullableFloat4) TypeHelper.getNewVector(field, allocator);
-
+    NullableFloat4Vector v = (NullableFloat4Vector) TypeHelper.getNewVector(field, allocator);
+    NullableFloat4Vector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 100.1f);
-    v.set(1, 101.2f);
-    v.set(100, 102.3f);
-    v.set(1022, 103.4f);
-    v.set(1023, 104.5f);
+    m.set(0, 100.1f);
+    m.set(1, 101.2f);
+    m.set(100, 102.3f);
+    m.set(1022, 103.4f);
+    m.set(1023, 104.5f);
     assertEquals(100.1f, v.get(0), 0);
     assertEquals(101.2f, v.get(1), 0);
     assertEquals(102.3f, v.get(100), 0);
@@ -189,17 +226,32 @@ public class TestValueVector {
     assertEquals(104.5f, v.get(1023), 0);
 
     // Ensure null values throw
-    try {
-      v.get(3);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(3);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
+    
     v.allocateNew(2048);
-    try {
-      v.get(0);
-      assertFalse(false);
-    } catch(NullValueException e) { }
-
+    {
+      boolean b = false;
+      try {
+        v.get(0);
+      } catch(AssertionError e) { 
+        b = true;
+      }finally{
+        if(!b){
+          assert false;
+        }
+      }   
+    }
   }
 
   @Test
@@ -218,30 +270,31 @@ public class TestValueVector {
     MaterializedField field = MaterializedField.create(defBuilder.build());
 
     // Create a new value vector for 1024 integers
-    ValueVector.MutableBit v = new ValueVector.MutableBit(field, allocator);
+    BitVector v = new BitVector(field, allocator);
+    BitVector.Mutator m = v.getMutator();
     v.allocateNew(1024);
 
     // Put and set a few values
-    v.set(0, 1);
-    v.set(1, 0);
-    v.set(100, 0);
-    v.set(1022, 1);
+    m.set(0, 1);
+    m.set(1, 0);
+    m.set(100, 0);
+    m.set(1022, 1);
     assertEquals(1, v.get(0));
     assertEquals(0, v.get(1));
     assertEquals(0, v.get(100));
     assertEquals(1, v.get(1022));
 
     // test setting the same value twice
-    v.set(0, 1);
-    v.set(0, 1);
-    v.set(1, 0);
-    v.set(1, 0);
+    m.set(0, 1);
+    m.set(0, 1);
+    m.set(1, 0);
+    m.set(1, 0);
     assertEquals(1, v.get(0));
     assertEquals(0, v.get(1));
 
     // test toggling the values
-    v.set(0, 0);
-    v.set(1, 1);
+    m.set(0, 0);
+    m.set(1, 1);
     assertEquals(0, v.get(0));
     assertEquals(1, v.get(1));
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/36793bb2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
index cef40ff..117414c 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/JSONRecordReaderTest.java
@@ -1,8 +1,16 @@
 package org.apache.drill.exec.store;
 
-import com.beust.jcommander.internal.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+
 import mockit.Expectations;
 import mockit.Injectable;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.DirectBufferAllocator;
@@ -10,17 +18,11 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.SchemaDefProtos;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.beust.jcommander.internal.Lists;
 
 public class JSONRecordReaderTest {
     private static final Charset UTF_8 = Charset.forName("UTF-8");
@@ -31,7 +33,7 @@ public class JSONRecordReaderTest {
 
     class MockOutputMutator implements OutputMutator {
         List<Integer> removedFields = Lists.newArrayList();
-        List<ValueVector.Base> addFields = Lists.newArrayList();
+        List<ValueVector> addFields = Lists.newArrayList();
 
         @Override
         public void removeField(int fieldId) throws SchemaChangeException {
@@ -39,7 +41,7 @@ public class JSONRecordReaderTest {
         }
 
         @Override
-        public void addField(int fieldId, ValueVector.Base vector) throws SchemaChangeException {
+        public void addField(int fieldId, ValueVector vector) throws SchemaChangeException {
             addFields.add(vector);
         }
 
@@ -51,16 +53,16 @@ public class JSONRecordReaderTest {
             return removedFields;
         }
 
-        List<ValueVector.Base> getAddFields() {
+        List<ValueVector> getAddFields() {
             return addFields;
         }
     }
 
-    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name) {
         assertField(valueVector, index, expectedMinorType, value, name, 0);
     }
 
-    private <T> void assertField(ValueVector.Base valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
+    private <T> void assertField(ValueVector valueVector, int index, SchemaDefProtos.MinorType expectedMinorType, T value, String name, int parentFieldId) {
         UserBitShared.FieldMetadata metadata = valueVector.getMetadata();
         SchemaDefProtos.FieldDef def = metadata.getDef();
         assertEquals(expectedMinorType, def.getMajorType().getMinorType());
@@ -90,7 +92,7 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_1.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(3, addFields.size());
@@ -116,7 +118,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"));
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
 
         jr.setup(mutator);
         assertEquals(3, jr.next());
@@ -142,7 +144,7 @@ public class JSONRecordReaderTest {
         assertEquals(0, jr.next());
     }
 
-    @Test
+    @Test @Ignore
     public void testChangedSchemaInTwoBatches(@Injectable final FragmentContext context) throws IOException, ExecutionSetupException {
         new Expectations() {
             {
@@ -153,7 +155,7 @@ public class JSONRecordReaderTest {
 
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_2.json"), 64); // batch only fits 1 int
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         List<Integer> removedFields = mutator.getRemovedFields();
 
         jr.setup(mutator);
@@ -201,7 +203,7 @@ public class JSONRecordReaderTest {
         JSONRecordReader jr = new JSONRecordReader(context, getResource("scan_json_test_3.json"));
 
         MockOutputMutator mutator = new MockOutputMutator();
-        List<ValueVector.Base> addFields = mutator.getAddFields();
+        List<ValueVector> addFields = mutator.getAddFields();
         jr.setup(mutator);
         assertEquals(2, jr.next());
         assertEquals(5, addFields.size());