You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2017/04/18 02:05:51 UTC

[1/3] drill git commit: Test-specific column accessor implementation. Provides a simplified, unified set of access methods for value vectors specifically for wrting simple, compact unit test code.

Repository: drill
Updated Branches:
  refs/heads/master 72903d014 -> 381eab668


http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
new file mode 100644
index 0000000..019d3be
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
@@ -0,0 +1,122 @@
+/*
+ * 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.accessor.impl;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors;
+import org.apache.drill.exec.vector.accessor.impl.AbstractArrayReader.ArrayColumnReader;
+import org.apache.drill.exec.vector.accessor.impl.AbstractArrayWriter.ArrayColumnWriter;
+
+/**
+ * Gather generated accessor classes into a set of class
+ * tables to allow rapid run-time creation of accessors.
+ * The caller is responsible for binding the accessor to
+ * a vector and a row index.
+ */
+
+public class ColumnAccessorFactory {
+
+  private static Class<? extends AbstractColumnWriter> columnWriters[][] = buildColumnWriters();
+  private static Class<? extends AbstractColumnReader> columnReaders[][] = buildColumnReaders();
+  private static Class<? extends AbstractArrayWriter> arrayWriters[] = buildArrayWriters();
+  private static Class<? extends AbstractArrayReader> arrayReaders[] = buildArrayReaders();
+
+  @SuppressWarnings("unchecked")
+  private static Class<? extends AbstractColumnWriter>[][] buildColumnWriters() {
+    int typeCount = MinorType.values().length;
+    int modeCount = DataMode.values().length;
+    Class<? extends AbstractColumnWriter> writers[][] = new Class[typeCount][];
+    for (int i = 0; i < typeCount; i++) {
+      writers[i] = new Class[modeCount];
+    }
+
+    ColumnAccessors.defineWriters(writers);
+    return writers;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Class<? extends AbstractColumnReader>[][] buildColumnReaders() {
+    int typeCount = MinorType.values().length;
+    int modeCount = DataMode.values().length;
+    Class<? extends AbstractColumnReader> readers[][] = new Class[typeCount][];
+    for (int i = 0; i < typeCount; i++) {
+      readers[i] = new Class[modeCount];
+    }
+
+    ColumnAccessors.defineReaders(readers);
+    return readers;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Class<? extends AbstractArrayWriter>[] buildArrayWriters() {
+    int typeCount = MinorType.values().length;
+    Class<? extends AbstractArrayWriter> writers[] = new Class[typeCount];
+    ColumnAccessors.defineArrayWriters(writers);
+    return writers;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Class<? extends AbstractArrayReader>[] buildArrayReaders() {
+    int typeCount = MinorType.values().length;
+    Class<? extends AbstractArrayReader> readers[] = new Class[typeCount];
+    ColumnAccessors.defineArrayReaders(readers);
+    return readers;
+  }
+
+  public static AbstractColumnWriter newWriter(MajorType type) {
+    try {
+      if (type.getMode() == DataMode.REPEATED) {
+        Class<? extends AbstractArrayWriter> writerClass = arrayWriters[type.getMinorType().ordinal()];
+        if (writerClass == null) {
+          throw new UnsupportedOperationException();
+        }
+        return new ArrayColumnWriter(writerClass.newInstance());
+      } else {
+        Class<? extends AbstractColumnWriter> writerClass = columnWriters[type.getMinorType().ordinal()][type.getMode().ordinal()];
+        if (writerClass == null) {
+          throw new UnsupportedOperationException();
+        }
+        return writerClass.newInstance();
+      }
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public static AbstractColumnReader newReader(MajorType type) {
+    try {
+      if (type.getMode() == DataMode.REPEATED) {
+        Class<? extends AbstractArrayReader> readerClass = arrayReaders[type.getMinorType().ordinal()];
+        if (readerClass == null) {
+          throw new UnsupportedOperationException();
+        }
+        return new ArrayColumnReader(readerClass.newInstance());
+      } else {
+        Class<? extends AbstractColumnReader> readerClass = columnReaders[type.getMinorType().ordinal()][type.getMode().ordinal()];
+        if (readerClass == null) {
+          throw new UnsupportedOperationException();
+        }
+        return readerClass.newInstance();
+      }
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
new file mode 100644
index 0000000..041023b
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.vector.accessor.impl;
+
+import org.apache.drill.exec.vector.accessor.ColumnReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+
+/**
+ * Reader for a tuple (a row or a map.) Provides access to each
+ * column using either a name or a numeric index.
+ */
+
+public class TupleReaderImpl extends AbstractTupleAccessor implements TupleReader {
+
+  private final AbstractColumnReader readers[];
+
+  public TupleReaderImpl(TupleSchema schema, AbstractColumnReader readers[]) {
+    super(schema);
+    this.readers = readers;
+  }
+
+  @Override
+  public ColumnReader column(int colIndex) {
+    return readers[colIndex];
+  }
+
+  @Override
+  public ColumnReader column(String colName) {
+    int index = schema.columnIndex(colName);
+    if (index == -1) {
+      return null; }
+    return readers[index];
+  }
+
+  @Override
+  public Object get(int colIndex) {
+    ColumnReader colReader = column(colIndex);
+    if (colReader.isNull()) {
+      return null; }
+    switch (colReader.valueType()) {
+    case BYTES:
+      return colReader.getBytes();
+    case DOUBLE:
+      return colReader.getDouble();
+    case INTEGER:
+      return colReader.getInt();
+    case LONG:
+      return colReader.getLong();
+    case STRING:
+      return colReader.getString();
+    default:
+      throw new IllegalArgumentException("Unsupported type " + colReader.valueType());
+    }
+  }
+
+  @Override
+  public String getAsString(int colIndex) {
+    ColumnReader colReader = column(colIndex);
+    if (colReader.isNull()) {
+      return "null";
+    }
+    switch (colReader.valueType()) {
+    case BYTES:
+      StringBuilder buf = new StringBuilder()
+          .append("[");
+      byte value[] = colReader.getBytes();
+      int len = Math.min(value.length, 20);
+      for (int i = 0; i < len;  i++) {
+        if (i > 0) {
+          buf.append(", ");
+        }
+        buf.append((int) value[i]);
+      }
+      if (value.length > len) {
+        buf.append("...");
+      }
+      buf.append("]");
+      return buf.toString();
+    case DOUBLE:
+      return Double.toString(colReader.getDouble());
+    case INTEGER:
+      return Integer.toString(colReader.getInt());
+    case LONG:
+      return Long.toString(colReader.getLong());
+    case STRING:
+      return "\"" + colReader.getString() + "\"";
+    case DECIMAL:
+      return colReader.getDecimal().toPlainString();
+    default:
+      throw new IllegalArgumentException("Unsupported type " + colReader.valueType());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
new file mode 100644
index 0000000..015b099
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
@@ -0,0 +1,162 @@
+/*
+ * 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.accessor.impl;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.accessor.AccessorUtilities;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnAccessor.ValueType;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.joda.time.Period;
+
+/**
+ * Implementation for a writer for a tuple (a row or a map.) Provides access to each
+ * column using either a name or a numeric index.
+ */
+
+public class TupleWriterImpl extends AbstractTupleAccessor implements TupleWriter {
+
+  private final AbstractColumnWriter writers[];
+
+  public TupleWriterImpl(TupleSchema schema, AbstractColumnWriter writers[]) {
+    super(schema);
+    this.writers = writers;
+  }
+
+  public void start() {
+    for (int i = 0; i < writers.length;  i++) {
+      writers[i].start();
+    }
+  }
+
+  @Override
+  public ColumnWriter column(int colIndex) {
+    return writers[colIndex];
+  }
+
+  @Override
+  public ColumnWriter column(String colName) {
+    int index = schema.columnIndex(colName);
+    if (index == -1) {
+      return null; }
+    return writers[index];
+  }
+
+  @Override
+  public void set(int colIndex, Object value) {
+    ColumnWriter colWriter = column(colIndex);
+    if (value == null) {
+      // Arrays have no null concept, just an empty array.
+      if (colWriter.valueType() != ValueType.ARRAY) {
+        colWriter.setNull();
+      }
+    } else if (value instanceof Integer) {
+      colWriter.setInt((Integer) value);
+    } else if (value instanceof Long) {
+      colWriter.setLong((Long) value);
+    } else if (value instanceof String) {
+      colWriter.setString((String) value);
+    } else if (value instanceof BigDecimal) {
+      colWriter.setDecimal((BigDecimal) value);
+    } else if (value instanceof Period) {
+      colWriter.setPeriod((Period) value);
+    } else if (value instanceof byte[]) {
+      colWriter.setBytes((byte[]) value);
+    } else if (value instanceof Byte) {
+      colWriter.setInt((Byte) value);
+    } else if (value instanceof Short) {
+      colWriter.setInt((Short) value);
+    } else if (value instanceof Double) {
+      colWriter.setDouble((Double) value);
+    } else if (value instanceof Float) {
+      colWriter.setDouble((Float) value);
+    } else if (value.getClass().getName().startsWith("[")) {
+      setArray(colIndex, value);
+    } else {
+      throw new IllegalArgumentException("Unsupported type " +
+                value.getClass().getSimpleName() + " for column " + colIndex);
+    }
+  }
+
+  public void setArray(int colIndex, Object value) {
+    if (value == null) {
+      // Assume null means a 0-element array since Drill does
+      // not support null for the whole array.
+
+      return;
+    }
+    String objClass = value.getClass().getName();
+    if (!objClass.startsWith("[")) {
+      throw new IllegalArgumentException("Argument is not an array");
+    }
+
+    ColumnWriter colWriter = column(colIndex);
+    if (colWriter.valueType() != ValueType.ARRAY) {
+      throw new IllegalArgumentException("Column is not an array");
+    }
+
+    ArrayWriter arrayWriter = colWriter.array();
+
+    // Figure out type
+
+    char second = objClass.charAt( 1 );
+    switch ( second ) {
+    case  'B':
+      AccessorUtilities.setByteArray(arrayWriter, (byte[]) value );
+      break;
+    case  'S':
+      AccessorUtilities.setShortArray(arrayWriter, (short[]) value );
+      break;
+    case  'I':
+      AccessorUtilities.setIntArray(arrayWriter, (int[]) value );
+      break;
+    case  'J':
+      AccessorUtilities.setLongArray(arrayWriter, (long[]) value );
+      break;
+    case  'F':
+      AccessorUtilities.setFloatArray(arrayWriter, (float[]) value );
+      break;
+    case  'D':
+      AccessorUtilities.setDoubleArray(arrayWriter, (double[]) value );
+      break;
+    case  'Z':
+      AccessorUtilities.setBooleanArray(arrayWriter, (boolean[]) value );
+      break;
+    case 'L':
+     int posn = objClass.indexOf(';');
+
+      // If the array is of type Object, then we have no type info.
+
+      String memberClassName = objClass.substring( 2, posn );
+      if (memberClassName.equals(String.class.getName())) {
+        AccessorUtilities.setStringArray(arrayWriter, (String[]) value );
+      } else if (memberClassName.equals(Period.class.getName())) {
+        AccessorUtilities.setPeriodArray(arrayWriter, (Period[]) value );
+      } else if (memberClassName.equals(BigDecimal.class.getName())) {
+        AccessorUtilities.setBigDecimalArray(arrayWriter, (BigDecimal[]) value );
+      } else {
+        throw new IllegalArgumentException( "Unknown Java array type: " + memberClassName );
+      }
+      break;
+    default:
+      throw new IllegalArgumentException( "Unknown Java array type: " + second );
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/package-info.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/package-info.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/package-info.java
new file mode 100644
index 0000000..92c2a16
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/**
+ * Provides implementations of the API defined in the parent package.
+ * For the most part, code here provides base classes for the generated
+ * accessors. The code here implements the common bits, so that the
+ * generated code can be as simple as possible. While there may be some
+ * slight performance benefits from repeated code, this code is designed
+ * for testing, so simplicity is more important that the last tiny bit
+ * of performance.
+ */
+package org.apache.drill.exec.vector.accessor.impl;

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/package-info.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/package-info.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/package-info.java
new file mode 100644
index 0000000..f51c1a9
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/package-info.java
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+/**
+ * Provides a light-weight, simplified set of column readers and writers that
+ * can be plugged into a variety of row-level readers and writers. The classes
+ * and interfaces here form a framework for accessing rows and columns, but do
+ * not provide the code to build accessors for a given row batch. This code is
+ * meant to be generic, but the first (and, thus far, only) use is with the test
+ * framework for the java-exec project. That one implementation is specific to
+ * unit tests, but the accessor framework could easily be used for other
+ * purposes as well.
+ * <p>
+ * Drill provides a set of column readers and writers. Compared to those, this
+ * set:
+ * <ul>
+ * <li>Works with all Drill data types. The other set works only with repeated
+ * and nullable types.</li>
+ * <li>Is a generic interface. The other set is bound tightly to the
+ * {@link ScanBatch} class.</li>
+ * <li>Uses generic types such as <tt>getInt()</tt> for most numeric types. The
+ * other set has accessors specific to each of the ~30 data types which Drill
+ * supports.</li>
+ * </ul>
+ * The key difference is that this set is designed for developer ease-of-use, a
+ * primary requirement for unit tests. The other set is designed to be used in
+ * machine-generated or write-once code and so can be much more complex.
+ * <p>
+ * That is, the accessors here are optimized for test code: they trade
+ * convenience for a slight decrease in speed (the performance hit comes from
+ * the extra level of indirection which hides the complex, type-specific code
+ * otherwise required.)
+ * <p>
+ * {@link ColumnReader} and {@link ColumnWriter} are the core abstractions: they
+ * provide simplified access to the myriad of Drill column types via a
+ * simplified, uniform API. {@link TupleReader} and {@link TupleWriter} provide
+ * a simplified API to rows or maps (both of which are tuples in Drill.)
+ * {@link AccessorUtilities} provides a number of data conversion tools.
+ * <p>
+ * Overview of the code structure:
+ * <dl>
+ * <dt>TupleWriter, TupleReader</dt>
+ * <dd>In relational terms, a tuple is an ordered collection of values, where
+ * the meaning of the order is provided by a schema (usually a name/type pair.)
+ * It turns out that Drill rows and maps are both tuples. The tuple classes
+ * provide the means to work with a tuple: get the schema, get a column by name
+ * or by position. Note that Drill code normally references columns by name.
+ * But, doing so is slower than access by position (index). To provide efficient
+ * code, the tuple classes assume that the implementation imposes a column
+ * ordering which can be exposed via the indexes.</dd>
+ * <dt>ColumnAccessor</dt>
+ * <dd>A generic base class for column readers and writers that provides the
+ * column data type.</dd>
+ * <dt>ColumnWriter, ColumnReader</dt>
+ * <dd>A uniform interface implemented for each column type ("major type" in
+ * Drill terminology). The scalar types: Nullable (Drill optional) and
+ * non-nullable (Drill required) fields use the same interface. Arrays (Drill
+ * repeated) are special. To handle the array aspect, even array fields use the
+ * same interface, but the <tt>getArray</tt> method returns another layer of
+ * accessor (writer or reader) specific for arrays.
+ * <p>
+ * Both the column reader and writer use a reduced set of data types to access
+ * values. Drill provides about 38 different types, but they can be mapped to a
+ * smaller set for programmatic access. For example, the signed byte, short,
+ * int; and the unsigned 8-bit, and 16-bit values can all be mapped to ints for
+ * get/set. The result is a much simpler set of get/set methods compared to the
+ * underlying set of vector types.</dt>
+ * <dt>ArrayWriter, ArrayReader
+ * <dt>
+ * <dd>The interface for the array accessors as described above. Of particular
+ * note is the difference in the form of the methods. The writer has only a
+ * <tt>setInt()</tt> method, no index. The methods assume write-only, write-once
+ * semantics: each set adds a new value. The reader, by contrast has a
+ * <tt>getInt(int index)</tt> method: read access is random.</tt>
+ * <dt>ScalarWriter<dt>
+ * <dd>Because of the form of the array writer, both the array writer and
+ * column writer have the same method signatures. To avoid repeating these
+ * methods, they are factored out into the common <tt>ScalarWriter</tt>
+ * interface.</dd>
+ * <dt>ColumnAccessors (templates)</dt>
+ * <dd>The Freemarker-based template used to generate the actual accessor
+ * implementations.</dd>
+ * <dt>ColumnAccessors (accessors)</dt>
+ * <dd>The generated accessors: one for each combination of write/read, data
+ * (minor) type and cardinality (data model).
+ * <dd>
+ * <dt>RowIndex</dt>
+ * <dd>This nested class binds the accessor to the current row position for the
+ * entire record batch. That is, you don't ask for the value of column a for row
+ * 5, then the value of column b for row 5, etc. as with the "raw" vectors.
+ * Instead, the implementation sets the row position (with, say an interator.)
+ * Then, all columns implicitly return values for the current row.
+ * <p>
+ * Different implementations of the row index handle the case of no selection
+ * vector, a selection vector 2, or a selection vector 4.</dd>
+ * <dt>VectorAccessor</dt>
+ * <dd>The readers can work with single batches or "hyper"
+ * batches. A hyper batch occurs in operators such as sort where an operator
+ * references a collection of batches as if they were one huge batch. In this
+ * case, each column consists of a "stack" of vectors. The vector accessor picks
+ * out one vector from the stack for each row. Vector accessors are used only
+ * for hyper batches; single batches work directly with the corresponding
+ * vector.
+ * <p>
+ * You can think of the (row index + vector accessor, column index) as forming a
+ * coordinate pair. The row index provides the y index (vertical position along
+ * the rows.) The vector accessor maps the row position to a vector when needed.
+ * The column index picks out the x coordinate (horizontal position along the
+ * columns.)</dt>
+ * </dl>
+ */
+
+package org.apache.drill.exec.vector.accessor;

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
index 1664b0a..8a54535 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
@@ -266,5 +266,9 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
       final int childValueCount = valueCount == 0 ? 0 : offsets.getAccessor().get(valueCount);
       vector.getMutator().setValueCount(childValueCount);
     }
+
+    public int getInnerValueCountAt(int index) {
+      return offsets.getAccessor().get(index+1) - offsets.getAccessor().get(index);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
index 3707ff0..d930728 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
@@ -48,7 +48,6 @@ import org.apache.drill.exec.vector.complex.impl.NullReader;
 import org.apache.drill.exec.vector.complex.impl.RepeatedMapReaderImpl;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
 public class RepeatedMapVector extends AbstractMapVector


[2/3] drill git commit: Test-specific column accessor implementation. Provides a simplified, unified set of access methods for value vectors specifically for wrting simple, compact unit test code.

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
index a87fe4c..914d68d 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -32,25 +32,25 @@ import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
 
 public class DecimalUtility extends CoreDecimalUtility{
 
-    public final static int MAX_DIGITS = 9;
-    public final static int MAX_DIGITS_INT = 10;
-    public final static int MAX_DIGITS_BIGINT = 19;
-    public final static int DIGITS_BASE = 1000000000;
-    public final static int DIGITS_MAX = 999999999;
-    public final static int INTEGER_SIZE = (Integer.SIZE/8);
-
-    public final static String[] decimalToString = {"",
-            "0",
-            "00",
-            "000",
-            "0000",
-            "00000",
-            "000000",
-            "0000000",
-            "00000000",
-            "000000000"};
-
-    public final static long[] scale_long_constants = {
+  public final static int MAX_DIGITS = 9;
+  public final static int MAX_DIGITS_INT = 10;
+  public final static int MAX_DIGITS_BIGINT = 19;
+  public final static int DIGITS_BASE = 1000000000;
+  public final static int DIGITS_MAX = 999999999;
+  public final static int INTEGER_SIZE = (Integer.SIZE/8);
+
+  public final static String[] decimalToString = {"",
+          "0",
+          "00",
+          "000",
+          "0000",
+          "00000",
+          "000000",
+          "0000000",
+          "00000000",
+          "000000000"};
+
+  public final static long[] scale_long_constants = {
         1,
         10,
         100,
@@ -71,99 +71,99 @@ public class DecimalUtility extends CoreDecimalUtility{
         100000000000000000l,
         1000000000000000000l};
 
-    /*
-     * Simple function that returns the static precomputed
-     * power of ten, instead of using Math.pow
-     */
-    public static long getPowerOfTen(int power) {
-      assert power >= 0 && power < scale_long_constants.length;
-      return scale_long_constants[(power)];
-    }
-
-    /*
-     * Math.pow returns a double and while multiplying with large digits
-     * in the decimal data type we encounter noise. So instead of multiplying
-     * with Math.pow we use the static constants to perform the multiplication
-     */
-    public static long adjustScaleMultiply(long input, int factor) {
-      int index = Math.abs(factor);
-      assert index >= 0 && index < scale_long_constants.length;
-      if (factor >= 0) {
-        return input * scale_long_constants[index];
-      } else {
-        return input / scale_long_constants[index];
-      }
-    }
+  /*
+   * Simple function that returns the static precomputed
+   * power of ten, instead of using Math.pow
+   */
+  public static long getPowerOfTen(int power) {
+    assert power >= 0 && power < scale_long_constants.length;
+    return scale_long_constants[(power)];
+  }
 
-    public static long adjustScaleDivide(long input, int factor) {
-      int index = Math.abs(factor);
-      assert index >= 0 && index < scale_long_constants.length;
-      if (factor >= 0) {
-        return input / scale_long_constants[index];
-      } else {
-        return input * scale_long_constants[index];
-      }
+  /*
+   * Math.pow returns a double and while multiplying with large digits
+   * in the decimal data type we encounter noise. So instead of multiplying
+   * with Math.pow we use the static constants to perform the multiplication
+   */
+  public static long adjustScaleMultiply(long input, int factor) {
+    int index = Math.abs(factor);
+    assert index >= 0 && index < scale_long_constants.length;
+    if (factor >= 0) {
+      return input * scale_long_constants[index];
+    } else {
+      return input / scale_long_constants[index];
     }
+  }
 
-    /* Given the number of actual digits this function returns the
-     * number of indexes it will occupy in the array of integers
-     * which are stored in base 1 billion
-     */
-    public static int roundUp(int ndigits) {
-        return (ndigits + MAX_DIGITS - 1)/MAX_DIGITS;
+  public static long adjustScaleDivide(long input, int factor) {
+    int index = Math.abs(factor);
+    assert index >= 0 && index < scale_long_constants.length;
+    if (factor >= 0) {
+      return input / scale_long_constants[index];
+    } else {
+      return input * scale_long_constants[index];
     }
+  }
 
-    /* Returns a string representation of the given integer
-     * If the length of the given integer is less than the
-     * passed length, this function will prepend zeroes to the string
-     */
-    public static StringBuilder toStringWithZeroes(int number, int desiredLength) {
-        String value = ((Integer) number).toString();
-        int length = value.length();
+  /* Given the number of actual digits this function returns the
+   * number of indexes it will occupy in the array of integers
+   * which are stored in base 1 billion
+   */
+  public static int roundUp(int ndigits) {
+      return (ndigits + MAX_DIGITS - 1)/MAX_DIGITS;
+  }
 
-        StringBuilder str = new StringBuilder();
-        str.append(decimalToString[desiredLength - length]);
-        str.append(value);
+  /** Returns a string representation of the given integer
+   * If the length of the given integer is less than the
+   * passed length, this function will prepend zeroes to the string
+   */
+  public static StringBuilder toStringWithZeroes(int number, int desiredLength) {
+    String value = ((Integer) number).toString();
+    int length = value.length();
 
-        return str;
-    }
+    StringBuilder str = new StringBuilder();
+    str.append(decimalToString[desiredLength - length]);
+    str.append(value);
 
-    public static StringBuilder toStringWithZeroes(long number, int desiredLength) {
-        String value = ((Long) number).toString();
-        int length = value.length();
+    return str;
+  }
 
-        StringBuilder str = new StringBuilder();
+  public static StringBuilder toStringWithZeroes(long number, int desiredLength) {
+    String value = ((Long) number).toString();
+    int length = value.length();
 
-        // Desired length can be > MAX_DIGITS
-        int zeroesLength = desiredLength - length;
-        while (zeroesLength > MAX_DIGITS) {
-            str.append(decimalToString[MAX_DIGITS]);
-            zeroesLength -= MAX_DIGITS;
-        }
-        str.append(decimalToString[zeroesLength]);
-        str.append(value);
+    StringBuilder str = new StringBuilder();
 
-        return str;
+    // Desired length can be > MAX_DIGITS
+    int zeroesLength = desiredLength - length;
+    while (zeroesLength > MAX_DIGITS) {
+        str.append(decimalToString[MAX_DIGITS]);
+        zeroesLength -= MAX_DIGITS;
     }
+    str.append(decimalToString[zeroesLength]);
+    str.append(value);
+
+    return str;
+  }
 
   public static BigDecimal getBigDecimalFromIntermediate(ByteBuf data, int startIndex, int nDecimalDigits, int scale) {
 
-        // In the intermediate representation we don't pad the scale with zeroes, so set truncate = false
-        return getBigDecimalFromDrillBuf(data, startIndex, nDecimalDigits, scale, false);
-    }
+    // In the intermediate representation we don't pad the scale with zeroes, so set truncate = false
+    return getBigDecimalFromDrillBuf(data, startIndex, nDecimalDigits, scale, false);
+  }
 
-    public static BigDecimal getBigDecimalFromSparse(DrillBuf data, int startIndex, int nDecimalDigits, int scale) {
+  public static BigDecimal getBigDecimalFromSparse(DrillBuf data, int startIndex, int nDecimalDigits, int scale) {
 
-        // In the sparse representation we pad the scale with zeroes for ease of arithmetic, need to truncate
-        return getBigDecimalFromDrillBuf(data, startIndex, nDecimalDigits, scale, true);
-    }
+    // In the sparse representation we pad the scale with zeroes for ease of arithmetic, need to truncate
+    return getBigDecimalFromDrillBuf(data, startIndex, nDecimalDigits, scale, true);
+  }
 
-    public static BigDecimal getBigDecimalFromDrillBuf(DrillBuf bytebuf, int start, int length, int scale) {
-      byte[] value = new byte[length];
-      bytebuf.getBytes(start, value, 0, length);
-      BigInteger unscaledValue = new BigInteger(value);
-      return new BigDecimal(unscaledValue, scale);
-    }
+  public static BigDecimal getBigDecimalFromDrillBuf(DrillBuf bytebuf, int start, int length, int scale) {
+    byte[] value = new byte[length];
+    bytebuf.getBytes(start, value, 0, length);
+    BigInteger unscaledValue = new BigInteger(value);
+    return new BigDecimal(unscaledValue, scale);
+  }
 
   public static BigDecimal getBigDecimalFromByteBuffer(ByteBuffer bytebuf, int start, int length, int scale) {
     byte[] value = new byte[length];
@@ -172,111 +172,111 @@ public class DecimalUtility extends CoreDecimalUtility{
     return new BigDecimal(unscaledValue, scale);
   }
 
-    /* Create a BigDecimal object using the data in the DrillBuf.
-     * This function assumes that data is provided in a non-dense format
-     * It works on both sparse and intermediate representations.
-     */
+  /** Create a BigDecimal object using the data in the DrillBuf.
+   * This function assumes that data is provided in a non-dense format
+   * It works on both sparse and intermediate representations.
+   */
   public static BigDecimal getBigDecimalFromDrillBuf(ByteBuf data, int startIndex, int nDecimalDigits, int scale,
       boolean truncateScale) {
 
-        // For sparse decimal type we have padded zeroes at the end, strip them while converting to BigDecimal.
-        int actualDigits;
+    // For sparse decimal type we have padded zeroes at the end, strip them while converting to BigDecimal.
+    int actualDigits;
 
-        // Initialize the BigDecimal, first digit in the DrillBuf has the sign so mask it out
-        BigInteger decimalDigits = BigInteger.valueOf((data.getInt(startIndex)) & 0x7FFFFFFF);
+    // Initialize the BigDecimal, first digit in the DrillBuf has the sign so mask it out
+    BigInteger decimalDigits = BigInteger.valueOf((data.getInt(startIndex)) & 0x7FFFFFFF);
 
-        BigInteger base = BigInteger.valueOf(DIGITS_BASE);
+    BigInteger base = BigInteger.valueOf(DIGITS_BASE);
 
-        for (int i = 1; i < nDecimalDigits; i++) {
+    for (int i = 1; i < nDecimalDigits; i++) {
 
-            BigInteger temp = BigInteger.valueOf(data.getInt(startIndex + (i * INTEGER_SIZE)));
-            decimalDigits = decimalDigits.multiply(base);
-            decimalDigits = decimalDigits.add(temp);
-        }
+        BigInteger temp = BigInteger.valueOf(data.getInt(startIndex + (i * INTEGER_SIZE)));
+        decimalDigits = decimalDigits.multiply(base);
+        decimalDigits = decimalDigits.add(temp);
+    }
 
-        // Truncate any additional padding we might have added
-        if (truncateScale == true && scale > 0 && (actualDigits = scale % MAX_DIGITS) != 0) {
-            BigInteger truncate = BigInteger.valueOf((int)Math.pow(10, (MAX_DIGITS - actualDigits)));
-            decimalDigits = decimalDigits.divide(truncate);
-        }
+    // Truncate any additional padding we might have added
+    if (truncateScale == true && scale > 0 && (actualDigits = scale % MAX_DIGITS) != 0) {
+        BigInteger truncate = BigInteger.valueOf((int)Math.pow(10, (MAX_DIGITS - actualDigits)));
+        decimalDigits = decimalDigits.divide(truncate);
+    }
 
-        // set the sign
-        if ((data.getInt(startIndex) & 0x80000000) != 0) {
-            decimalDigits = decimalDigits.negate();
-        }
+    // set the sign
+    if ((data.getInt(startIndex) & 0x80000000) != 0) {
+        decimalDigits = decimalDigits.negate();
+    }
 
-        BigDecimal decimal = new BigDecimal(decimalDigits, scale);
+    BigDecimal decimal = new BigDecimal(decimalDigits, scale);
 
-        return decimal;
-    }
+    return decimal;
+  }
+
+  /* This function returns a BigDecimal object from the dense decimal representation.
+   * First step is to convert the dense representation into an intermediate representation
+   * and then invoke getBigDecimalFromDrillBuf() to get the BigDecimal object
+   */
+  public static BigDecimal getBigDecimalFromDense(DrillBuf data, int startIndex, int nDecimalDigits, int scale, int maxPrecision, int width) {
 
-    /* This function returns a BigDecimal object from the dense decimal representation.
-     * First step is to convert the dense representation into an intermediate representation
-     * and then invoke getBigDecimalFromDrillBuf() to get the BigDecimal object
+    /* This method converts the dense representation to
+     * an intermediate representation. The intermediate
+     * representation has one more integer than the dense
+     * representation.
      */
-    public static BigDecimal getBigDecimalFromDense(DrillBuf data, int startIndex, int nDecimalDigits, int scale, int maxPrecision, int width) {
-
-        /* This method converts the dense representation to
-         * an intermediate representation. The intermediate
-         * representation has one more integer than the dense
-         * representation.
-         */
-        byte[] intermediateBytes = new byte[((nDecimalDigits + 1) * INTEGER_SIZE)];
-
-        // Start storing from the least significant byte of the first integer
-        int intermediateIndex = 3;
-
-        int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
-        int[] reverseMask = {0xFC, 0xF0, 0xC0, 0x00};
-
-        int maskIndex;
-        int shiftOrder;
-        byte shiftBits;
-
-        // TODO: Some of the logic here is common with casting from Dense to Sparse types, factor out common code
-        if (maxPrecision == 38) {
-            maskIndex = 0;
-            shiftOrder = 6;
-            shiftBits = 0x00;
-            intermediateBytes[intermediateIndex++] = (byte) (data.getByte(startIndex) & 0x7F);
-        } else if (maxPrecision == 28) {
-            maskIndex = 1;
-            shiftOrder = 4;
-            shiftBits = (byte) ((data.getByte(startIndex) & 0x03) << shiftOrder);
-            intermediateBytes[intermediateIndex++] = (byte) (((data.getByte(startIndex) & 0x3C) & 0xFF) >>> 2);
-        } else {
-            throw new UnsupportedOperationException("Dense types with max precision 38 and 28 are only supported");
-        }
+    byte[] intermediateBytes = new byte[((nDecimalDigits + 1) * INTEGER_SIZE)];
+
+    // Start storing from the least significant byte of the first integer
+    int intermediateIndex = 3;
+
+    int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
+    int[] reverseMask = {0xFC, 0xF0, 0xC0, 0x00};
+
+    int maskIndex;
+    int shiftOrder;
+    byte shiftBits;
+
+    // TODO: Some of the logic here is common with casting from Dense to Sparse types, factor out common code
+    if (maxPrecision == 38) {
+        maskIndex = 0;
+        shiftOrder = 6;
+        shiftBits = 0x00;
+        intermediateBytes[intermediateIndex++] = (byte) (data.getByte(startIndex) & 0x7F);
+    } else if (maxPrecision == 28) {
+        maskIndex = 1;
+        shiftOrder = 4;
+        shiftBits = (byte) ((data.getByte(startIndex) & 0x03) << shiftOrder);
+        intermediateBytes[intermediateIndex++] = (byte) (((data.getByte(startIndex) & 0x3C) & 0xFF) >>> 2);
+    } else {
+        throw new UnsupportedOperationException("Dense types with max precision 38 and 28 are only supported");
+    }
 
-        int inputIndex = 1;
-        boolean sign = false;
+    int inputIndex = 1;
+    boolean sign = false;
 
-        if ((data.getByte(startIndex) & 0x80) != 0) {
-            sign = true;
-        }
+    if ((data.getByte(startIndex) & 0x80) != 0) {
+        sign = true;
+    }
 
-        while (inputIndex < width) {
+    while (inputIndex < width) {
 
-            intermediateBytes[intermediateIndex] = (byte) ((shiftBits) | (((data.getByte(startIndex + inputIndex) & reverseMask[maskIndex]) & 0xFF) >>> (8 - shiftOrder)));
+      intermediateBytes[intermediateIndex] = (byte) ((shiftBits) | (((data.getByte(startIndex + inputIndex) & reverseMask[maskIndex]) & 0xFF) >>> (8 - shiftOrder)));
 
-            shiftBits = (byte) ((data.getByte(startIndex + inputIndex) & mask[maskIndex]) << shiftOrder);
+      shiftBits = (byte) ((data.getByte(startIndex + inputIndex) & mask[maskIndex]) << shiftOrder);
 
-            inputIndex++;
-            intermediateIndex++;
+      inputIndex++;
+      intermediateIndex++;
 
-            if (((inputIndex - 1) % INTEGER_SIZE) == 0) {
-                shiftBits = (byte) ((shiftBits & 0xFF) >>> 2);
-                maskIndex++;
-                shiftOrder -= 2;
-            }
+      if (((inputIndex - 1) % INTEGER_SIZE) == 0) {
+          shiftBits = (byte) ((shiftBits & 0xFF) >>> 2);
+          maskIndex++;
+          shiftOrder -= 2;
+      }
 
-        }
-        /* copy the last byte */
-        intermediateBytes[intermediateIndex] = shiftBits;
+    }
+    /* copy the last byte */
+    intermediateBytes[intermediateIndex] = shiftBits;
 
-        if (sign == true) {
-            intermediateBytes[0] = (byte) (intermediateBytes[0] | 0x80);
-        }
+    if (sign == true) {
+        intermediateBytes[0] = (byte) (intermediateBytes[0] | 0x80);
+    }
 
     final ByteBuf intermediate = UnpooledByteBufAllocator.DEFAULT.buffer(intermediateBytes.length);
     try {
@@ -288,300 +288,305 @@ public class DecimalUtility extends CoreDecimalUtility{
       intermediate.release();
     }
 
-    }
+  }
 
-    /*
-     * Function converts the BigDecimal and stores it in out internal sparse representation
-     */
+  /**
+   * Function converts the BigDecimal and stores it in out internal sparse representation
+   */
   public static void getSparseFromBigDecimal(BigDecimal input, ByteBuf data, int startIndex, int scale, int precision,
       int nDecimalDigits) {
 
-        // Initialize the buffer
-        for (int i = 0; i < nDecimalDigits; i++) {
-          data.setInt(startIndex + (i * INTEGER_SIZE), 0);
-        }
-
-        boolean sign = false;
+    // Initialize the buffer
+    for (int i = 0; i < nDecimalDigits; i++) {
+      data.setInt(startIndex + (i * INTEGER_SIZE), 0);
+    }
 
-        if (input.signum() == -1) {
-            // negative input
-            sign = true;
-            input = input.abs();
-        }
+    boolean sign = false;
 
-        // Truncate the input as per the scale provided
-        input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
+    if (input.signum() == -1) {
+        // negative input
+        sign = true;
+        input = input.abs();
+    }
 
-        // Separate out the integer part
-        BigDecimal integerPart = input.setScale(0, BigDecimal.ROUND_DOWN);
+    // Truncate the input as per the scale provided
+    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
 
-        int destIndex = nDecimalDigits - roundUp(scale) - 1;
+    // Separate out the integer part
+    BigDecimal integerPart = input.setScale(0, BigDecimal.ROUND_DOWN);
 
-        // we use base 1 billion integer digits for out integernal representation
-        BigDecimal base = new BigDecimal(DIGITS_BASE);
+    int destIndex = nDecimalDigits - roundUp(scale) - 1;
 
-        while (integerPart.compareTo(BigDecimal.ZERO) == 1) {
-            // store the modulo as the integer value
-            data.setInt(startIndex + (destIndex * INTEGER_SIZE), (integerPart.remainder(base)).intValue());
-            destIndex--;
-            // Divide by base 1 billion
-            integerPart = (integerPart.divide(base)).setScale(0, BigDecimal.ROUND_DOWN);
-        }
+    // we use base 1 billion integer digits for out integernal representation
+    BigDecimal base = new BigDecimal(DIGITS_BASE);
 
-        /* Sparse representation contains padding of additional zeroes
-         * so each digit contains MAX_DIGITS for ease of arithmetic
-         */
-        int actualDigits;
-        if ((actualDigits = (scale % MAX_DIGITS)) != 0) {
-            // Pad additional zeroes
-            scale = scale + (MAX_DIGITS - actualDigits);
-            input = input.setScale(scale, BigDecimal.ROUND_DOWN);
-        }
+    while (integerPart.compareTo(BigDecimal.ZERO) == 1) {
+        // store the modulo as the integer value
+        data.setInt(startIndex + (destIndex * INTEGER_SIZE), (integerPart.remainder(base)).intValue());
+        destIndex--;
+        // Divide by base 1 billion
+        integerPart = (integerPart.divide(base)).setScale(0, BigDecimal.ROUND_DOWN);
+    }
 
-        //separate out the fractional part
-        BigDecimal fractionalPart = input.remainder(BigDecimal.ONE).movePointRight(scale);
+    /* Sparse representation contains padding of additional zeroes
+     * so each digit contains MAX_DIGITS for ease of arithmetic
+     */
+    int actualDigits;
+    if ((actualDigits = (scale % MAX_DIGITS)) != 0) {
+        // Pad additional zeroes
+        scale = scale + (MAX_DIGITS - actualDigits);
+        input = input.setScale(scale, BigDecimal.ROUND_DOWN);
+    }
 
-        destIndex = nDecimalDigits - 1;
+    //separate out the fractional part
+    BigDecimal fractionalPart = input.remainder(BigDecimal.ONE).movePointRight(scale);
 
-        while (scale > 0) {
-            // Get next set of MAX_DIGITS (9) store it in the DrillBuf
-            fractionalPart = fractionalPart.movePointLeft(MAX_DIGITS);
-            BigDecimal temp = fractionalPart.remainder(BigDecimal.ONE);
+    destIndex = nDecimalDigits - 1;
 
-            data.setInt(startIndex + (destIndex * INTEGER_SIZE), (temp.unscaledValue().intValue()));
-            destIndex--;
+    while (scale > 0) {
+        // Get next set of MAX_DIGITS (9) store it in the DrillBuf
+        fractionalPart = fractionalPart.movePointLeft(MAX_DIGITS);
+        BigDecimal temp = fractionalPart.remainder(BigDecimal.ONE);
 
-            fractionalPart = fractionalPart.setScale(0, BigDecimal.ROUND_DOWN);
-            scale -= MAX_DIGITS;
-        }
+        data.setInt(startIndex + (destIndex * INTEGER_SIZE), (temp.unscaledValue().intValue()));
+        destIndex--;
 
-        // Set the negative sign
-        if (sign == true) {
-            data.setInt(startIndex, data.getInt(startIndex) | 0x80000000);
-        }
+        fractionalPart = fractionalPart.setScale(0, BigDecimal.ROUND_DOWN);
+        scale -= MAX_DIGITS;
+    }
 
+    // Set the negative sign
+    if (sign == true) {
+        data.setInt(startIndex, data.getInt(startIndex) | 0x80000000);
     }
+  }
 
+  public static long getDecimal18FromBigDecimal(BigDecimal input, int scale, int precision) {
+    // Truncate or pad to set the input to the correct scale
+    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
 
-    public static long getDecimal18FromBigDecimal(BigDecimal input, int scale, int precision) {
-        // Truncate or pad to set the input to the correct scale
-        input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
+    return input.unscaledValue().longValue();
+  }
 
-        return (input.unscaledValue().longValue());
-    }
+  public static int getDecimal9FromBigDecimal(BigDecimal input, int scale, int precision) {
+    // Truncate or pad to set the input to the correct scale
+    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
 
-    public static BigDecimal getBigDecimalFromPrimitiveTypes(int input, int scale, int precision) {
-      return BigDecimal.valueOf(input, scale);
-    }
+    return input.unscaledValue().intValue();
+  }
 
-    public static BigDecimal getBigDecimalFromPrimitiveTypes(long input, int scale, int precision) {
-      return BigDecimal.valueOf(input, scale);
-    }
+  public static BigDecimal getBigDecimalFromPrimitiveTypes(int input, int scale, int precision) {
+    return BigDecimal.valueOf(input, scale);
+  }
+
+  public static BigDecimal getBigDecimalFromPrimitiveTypes(long input, int scale, int precision) {
+    return BigDecimal.valueOf(input, scale);
+  }
 
+  public static int compareDenseBytes(DrillBuf left, int leftStart, boolean leftSign, DrillBuf right, int rightStart, boolean rightSign, int width) {
 
-    public static int compareDenseBytes(DrillBuf left, int leftStart, boolean leftSign, DrillBuf right, int rightStart, boolean rightSign, int width) {
+    int invert = 1;
 
-      int invert = 1;
+    /* If signs are different then simply look at the
+     * sign of the two inputs and determine which is greater
+     */
+    if (leftSign != rightSign) {
 
-      /* If signs are different then simply look at the
-       * sign of the two inputs and determine which is greater
+      return((leftSign == true) ? -1 : 1);
+    } else if(leftSign == true) {
+      /* Both inputs are negative, at the end we will
+       * have to invert the comparison
        */
-      if (leftSign != rightSign) {
+      invert = -1;
+    }
 
-        return((leftSign == true) ? -1 : 1);
-      } else if(leftSign == true) {
-        /* Both inputs are negative, at the end we will
-         * have to invert the comparison
-         */
-        invert = -1;
-      }
+    int cmp = 0;
 
-      int cmp = 0;
-
-      for (int i = 0; i < width; i++) {
-        byte leftByte  = left.getByte(leftStart + i);
-        byte rightByte = right.getByte(rightStart + i);
-        // Unsigned byte comparison
-        if ((leftByte & 0xFF) > (rightByte & 0xFF)) {
-          cmp = 1;
-          break;
-        } else if ((leftByte & 0xFF) < (rightByte & 0xFF)) {
-          cmp = -1;
-          break;
-        }
+    for (int i = 0; i < width; i++) {
+      byte leftByte  = left.getByte(leftStart + i);
+      byte rightByte = right.getByte(rightStart + i);
+      // Unsigned byte comparison
+      if ((leftByte & 0xFF) > (rightByte & 0xFF)) {
+        cmp = 1;
+        break;
+      } else if ((leftByte & 0xFF) < (rightByte & 0xFF)) {
+        cmp = -1;
+        break;
       }
-      cmp *= invert; // invert the comparison if both were negative values
-
-      return cmp;
     }
+    cmp *= invert; // invert the comparison if both were negative values
 
-    public static int getIntegerFromSparseBuffer(DrillBuf buffer, int start, int index) {
-      int value = buffer.getInt(start + (index * 4));
+    return cmp;
+  }
 
-      if (index == 0) {
-        /* the first byte contains sign bit, return value without it */
-        value = (value & 0x7FFFFFFF);
-      }
-      return value;
-    }
+  public static int getIntegerFromSparseBuffer(DrillBuf buffer, int start, int index) {
+    int value = buffer.getInt(start + (index * 4));
 
-    public static void setInteger(DrillBuf buffer, int start, int index, int value) {
-      buffer.setInt(start + (index * 4), value);
+    if (index == 0) {
+      /* the first byte contains sign bit, return value without it */
+      value = (value & 0x7FFFFFFF);
     }
+    return value;
+  }
 
-    public static int compareSparseBytes(DrillBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, DrillBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits, boolean absCompare) {
+  public static void setInteger(DrillBuf buffer, int start, int index, int value) {
+    buffer.setInt(start + (index * 4), value);
+  }
 
-      int invert = 1;
+  public static int compareSparseBytes(DrillBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, DrillBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits, boolean absCompare) {
 
-      if (absCompare == false) {
-        if (leftSign != rightSign) {
-          return (leftSign == true) ? -1 : 1;
-        }
+    int invert = 1;
 
-        // Both values are negative invert the outcome of the comparison
-        if (leftSign == true) {
-          invert = -1;
-        }
+    if (absCompare == false) {
+      if (leftSign != rightSign) {
+        return (leftSign == true) ? -1 : 1;
       }
 
-      int cmp = compareSparseBytesInner(left, leftStart, leftSign, leftScale, leftPrecision, right, rightStart, rightSign, rightPrecision, rightScale, width, nDecimalDigits);
-      return cmp * invert;
+      // Both values are negative invert the outcome of the comparison
+      if (leftSign == true) {
+        invert = -1;
+      }
     }
-    public static int compareSparseBytesInner(DrillBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, DrillBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits) {
-      /* compute the number of integer digits in each decimal */
-      int leftInt  = leftPrecision - leftScale;
-      int rightInt = rightPrecision - rightScale;
 
-      /* compute the number of indexes required for storing integer digits */
-      int leftIntRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(leftInt);
-      int rightIntRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(rightInt);
+    int cmp = compareSparseBytesInner(left, leftStart, leftSign, leftScale, leftPrecision, right, rightStart, rightSign, rightPrecision, rightScale, width, nDecimalDigits);
+    return cmp * invert;
+  }
 
-      /* compute number of indexes required for storing scale */
-      int leftScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(leftScale);
-      int rightScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(rightScale);
+  public static int compareSparseBytesInner(DrillBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, DrillBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits) {
+    /* compute the number of integer digits in each decimal */
+    int leftInt  = leftPrecision - leftScale;
+    int rightInt = rightPrecision - rightScale;
 
-      /* compute index of the most significant integer digits */
-      int leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
-      int rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
+    /* compute the number of indexes required for storing integer digits */
+    int leftIntRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(leftInt);
+    int rightIntRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(rightInt);
 
-      int leftStopIndex = nDecimalDigits - leftScaleRoundedUp;
-      int rightStopIndex = nDecimalDigits - rightScaleRoundedUp;
+    /* compute number of indexes required for storing scale */
+    int leftScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(leftScale);
+    int rightScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(rightScale);
 
-      /* Discard the zeroes in the integer part */
-      while (leftIndex1 < leftStopIndex) {
-        if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
-          break;
-        }
+    /* compute index of the most significant integer digits */
+    int leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
+    int rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
 
-        /* Digit in this location is zero, decrement the actual number
-         * of integer digits
-         */
-        leftIntRoundedUp--;
-        leftIndex1++;
-      }
+    int leftStopIndex = nDecimalDigits - leftScaleRoundedUp;
+    int rightStopIndex = nDecimalDigits - rightScaleRoundedUp;
 
-      /* If we reached the stop index then the number of integers is zero */
-      if (leftIndex1 == leftStopIndex) {
-        leftIntRoundedUp = 0;
+    /* Discard the zeroes in the integer part */
+    while (leftIndex1 < leftStopIndex) {
+      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
+        break;
       }
 
-      while (rightIndex1 < rightStopIndex) {
-        if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
-          break;
-        }
+      /* Digit in this location is zero, decrement the actual number
+       * of integer digits
+       */
+      leftIntRoundedUp--;
+      leftIndex1++;
+    }
 
-        /* Digit in this location is zero, decrement the actual number
-         * of integer digits
-         */
-        rightIntRoundedUp--;
-        rightIndex1++;
-      }
+    /* If we reached the stop index then the number of integers is zero */
+    if (leftIndex1 == leftStopIndex) {
+      leftIntRoundedUp = 0;
+    }
 
-      if (rightIndex1 == rightStopIndex) {
-        rightIntRoundedUp = 0;
+    while (rightIndex1 < rightStopIndex) {
+      if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
+        break;
       }
 
-      /* We have the accurate number of non-zero integer digits,
-       * if the number of integer digits are different then we can determine
-       * which decimal is larger and needn't go down to comparing individual values
+      /* Digit in this location is zero, decrement the actual number
+       * of integer digits
        */
-      if (leftIntRoundedUp > rightIntRoundedUp) {
-        return 1;
-      }
-      else if (rightIntRoundedUp > leftIntRoundedUp) {
-        return -1;
-      }
+      rightIntRoundedUp--;
+      rightIndex1++;
+    }
 
-      /* The number of integer digits are the same, set the each index
-       * to the first non-zero integer and compare each digit
-       */
-      leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
-      rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
+    if (rightIndex1 == rightStopIndex) {
+      rightIntRoundedUp = 0;
+    }
 
-      while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
-        if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
-          return 1;
-        }
-        else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
-          return -1;
-        }
+    /* We have the accurate number of non-zero integer digits,
+     * if the number of integer digits are different then we can determine
+     * which decimal is larger and needn't go down to comparing individual values
+     */
+    if (leftIntRoundedUp > rightIntRoundedUp) {
+      return 1;
+    }
+    else if (rightIntRoundedUp > leftIntRoundedUp) {
+      return -1;
+    }
+
+    /* The number of integer digits are the same, set the each index
+     * to the first non-zero integer and compare each digit
+     */
+    leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
+    rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
 
-        leftIndex1++;
-        rightIndex1++;
+    while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
+      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
+        return 1;
+      }
+      else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
+        return -1;
       }
 
-      /* The integer part of both the decimal's are equal, now compare
-       * each individual fractional part. Set the index to be at the
-       * beginning of the fractional part
-       */
-      leftIndex1 = leftStopIndex;
-      rightIndex1 = rightStopIndex;
+      leftIndex1++;
+      rightIndex1++;
+    }
 
-      /* Stop indexes will be the end of the array */
-      leftStopIndex = nDecimalDigits;
-      rightStopIndex = nDecimalDigits;
+    /* The integer part of both the decimal's are equal, now compare
+     * each individual fractional part. Set the index to be at the
+     * beginning of the fractional part
+     */
+    leftIndex1 = leftStopIndex;
+    rightIndex1 = rightStopIndex;
 
-      /* compare the two fractional parts of the decimal */
-      while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
-        if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
-          return 1;
-        }
-        else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
-          return -1;
-        }
+    /* Stop indexes will be the end of the array */
+    leftStopIndex = nDecimalDigits;
+    rightStopIndex = nDecimalDigits;
 
-        leftIndex1++;
-        rightIndex1++;
+    /* compare the two fractional parts of the decimal */
+    while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
+      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
+        return 1;
       }
-
-      /* Till now the fractional part of the decimals are equal, check
-       * if one of the decimal has fractional part that is remaining
-       * and is non-zero
-       */
-      while (leftIndex1 < leftStopIndex) {
-        if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
-          return 1;
-        }
-        leftIndex1++;
+      else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
+        return -1;
       }
 
-      while(rightIndex1 < rightStopIndex) {
-        if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
-          return -1;
-        }
-        rightIndex1++;
-      }
+      leftIndex1++;
+      rightIndex1++;
+    }
 
-      /* Both decimal values are equal */
-      return 0;
+    /* Till now the fractional part of the decimals are equal, check
+     * if one of the decimal has fractional part that is remaining
+     * and is non-zero
+     */
+    while (leftIndex1 < leftStopIndex) {
+      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
+        return 1;
+      }
+      leftIndex1++;
     }
 
-    public static BigDecimal getBigDecimalFromByteArray(byte[] bytes, int start, int length, int scale) {
-      byte[] value = Arrays.copyOfRange(bytes, start, start + length);
-      BigInteger unscaledValue = new BigInteger(value);
-      return new BigDecimal(unscaledValue, scale);
+    while(rightIndex1 < rightStopIndex) {
+      if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
+        return -1;
+      }
+      rightIndex1++;
     }
 
+    /* Both decimal values are equal */
+    return 0;
+  }
+
+  public static BigDecimal getBigDecimalFromByteArray(byte[] bytes, int start, int length, int scale) {
+    byte[] value = Arrays.copyOfRange(bytes, start, start + length);
+    BigInteger unscaledValue = new BigInteger(value);
+    return new BigDecimal(unscaledValue, scale);
+  }
+
   public static void roundDecimal(DrillBuf result, int start, int nDecimalDigits, int desiredScale, int currentScale) {
     int newScaleRoundedUp  = org.apache.drill.exec.util.DecimalUtility.roundUp(desiredScale);
     int origScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(currentScale);
@@ -724,8 +729,6 @@ public class DecimalUtility extends CoreDecimalUtility{
 
     // compare byte by byte
     int n = 0;
-    int lPos = lStart;
-    int rPos = 0;
     while (n < length/4) {
       int leftInt = Decimal38SparseHolder.getInteger(n, lStart, left);
       int rightInt = ByteFunctionHelpers.getInteger(right, n);

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index 0321fa8..a0d5f65 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -28,8 +28,6 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public abstract class BaseValueVector implements ValueVector {
 //  private static final Logger logger = LoggerFactory.getLogger(BaseValueVector.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/AccessorUtilities.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/AccessorUtilities.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/AccessorUtilities.java
new file mode 100644
index 0000000..708d0db
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/AccessorUtilities.java
@@ -0,0 +1,125 @@
+/*
+ * 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.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Duration;
+import org.joda.time.Period;
+
+public class AccessorUtilities {
+
+  private AccessorUtilities() { }
+
+  public static void setFromInt(ColumnWriter writer, int value) {
+    switch (writer.valueType()) {
+    case BYTES:
+      writer.setBytes(Integer.toHexString(value).getBytes());
+      break;
+    case DOUBLE:
+      writer.setDouble(value);
+      break;
+    case INTEGER:
+      writer.setInt(value);
+      break;
+    case LONG:
+      writer.setLong(value);
+      break;
+    case STRING:
+      writer.setString(Integer.toString(value));
+      break;
+    case DECIMAL:
+      writer.setDecimal(BigDecimal.valueOf(value));
+      break;
+    case PERIOD:
+      writer.setPeriod(Duration.millis(value).toPeriod());
+      break;
+    default:
+      throw new IllegalStateException("Unknown writer type: " + writer.valueType());
+    }
+  }
+
+  public static int sv4Batch(int sv4Index) {
+    return sv4Index >>> 16;
+  }
+
+  public static int sv4Index(int sv4Index) {
+    return sv4Index & 0xFFFF;
+  }
+
+  public static void setBooleanArray(ArrayWriter arrayWriter, boolean[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setInt(value[i] ? 1 : 0);
+    }
+  }
+
+  public static void setByteArray(ArrayWriter arrayWriter, byte[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setInt(value[i]);
+    }
+  }
+
+  public static void setShortArray(ArrayWriter arrayWriter, short[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setInt(value[i]);
+    }
+  }
+
+  public static void setIntArray(ArrayWriter arrayWriter, int[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setInt(value[i]);
+    }
+  }
+
+  public static void setLongArray(ArrayWriter arrayWriter, long[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setLong(value[i]);
+    }
+  }
+
+  public static void setFloatArray(ArrayWriter arrayWriter, float[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setDouble(value[i]);
+    }
+  }
+
+  public static void setDoubleArray(ArrayWriter arrayWriter, double[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setDouble(value[i]);
+    }
+  }
+
+  public static void setStringArray(ArrayWriter arrayWriter, String[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setString(value[i]);
+    }
+  }
+
+  public static void setPeriodArray(ArrayWriter arrayWriter, Period[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setPeriod(value[i]);
+    }
+  }
+
+  public static void setBigDecimalArray(ArrayWriter arrayWriter,
+      BigDecimal[] value) {
+    for (int i = 0; i < value.length; i++) {
+      arrayWriter.setDecimal(value[i]);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayReader.java
new file mode 100644
index 0000000..040dcda
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayReader.java
@@ -0,0 +1,52 @@
+/*
+ * 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.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Period;
+
+/**
+ * Interface to access the values of an array column. In general, each
+ * vector implements just one of the get methods. Check the vector type
+ * to know which method to use. Though, generally, when writing test
+ * code, the type is known to the test writer.
+ * <p>
+ * Arrays allow random access to the values within the array. The index
+ * passed to each method is the index into the array for the current
+ * row and column. (This means that arrays are three dimensional:
+ * the usual (row, column) dimensions plus an array index dimension:
+ * (row, column, array index).
+ * <p>
+ * Note that the <tt>isNull()</tt> method is provided for completeness,
+ * but no Drill array allows null values at present.
+ */
+
+public interface ArrayReader extends ColumnAccessor {
+  int size();
+  boolean isNull(int index);
+  int getInt(int index);
+  long getLong(int index);
+  double getDouble(int index);
+  String getString(int index);
+  byte[] getBytes(int index);
+  BigDecimal getDecimal(int index);
+  Period getPeriod(int index);
+  TupleReader map(int index);
+  ArrayReader array(int index);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayWriter.java
new file mode 100644
index 0000000..16ff89e
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ArrayWriter.java
@@ -0,0 +1,42 @@
+/*
+ * 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.accessor;
+
+/**
+ * Writer for values into an array. Array writes are write-once,
+ * sequential: each call to a <tt>setFoo()</tt> method writes a
+ * value and advances the array index.
+ * <p>
+ * {@see ArrayReader}
+ */
+
+public interface ArrayWriter extends ColumnAccessor, ScalarWriter {
+
+  int size();
+
+  /**
+   * Determine if the next position is valid for writing. Will be invalid
+   * if the writer hits a size or other limit.
+   *
+   * @return true if another item is available and the reader is positioned
+   * at that item, false if no more items are available and the reader
+   * is no longer valid
+   */
+
+  boolean valid();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnAccessor.java
new file mode 100644
index 0000000..44cd48a
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnAccessor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.accessor;
+
+/**
+ * Common base interface for columns readers and writers. Provides
+ * the access type for the column. Note that multiple Drill types and
+ * data modes map to the same access type.
+ */
+
+public interface ColumnAccessor {
+  public enum ValueType {
+    INTEGER, LONG, DOUBLE, STRING, BYTES, DECIMAL, PERIOD, ARRAY, MAP
+  }
+
+  /**
+   * Describe the type of the value. This is a compression of the
+   * value vector type: it describes which method will return the
+   * vector value.
+   * @return the value type which indicates which get method
+   * is valid for the column
+   */
+
+  ColumnAccessor.ValueType valueType();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
new file mode 100644
index 0000000..860a866
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
@@ -0,0 +1,63 @@
+/*
+ * 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.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Period;
+
+/**
+ * Defines a reader to obtain values from value vectors using
+ * a simple, uniform interface. Vector values are mapped to
+ * their "natural" representations: the representation closest
+ * to the actual vector value. For date and time values, this
+ * generally means a numeric value. Applications can then map
+ * this value to Java objects as desired. Decimal types all
+ * map to BigDecimal as that is the only way in Java to
+ * represent large decimal values.
+ * <p>
+ * In general, a column maps to just one value. However, derived
+ * classes may choose to provide type conversions if convenient.
+ * An exception is thrown if a call is made to a method that
+ * is not supported by the column type.
+ * <p>
+ * Values of scalars are provided directly, using the get method
+ * for the target type. Maps and arrays are structured types and
+ * require another level of reader abstraction to access each value
+ * in the structure.
+ */
+
+public interface ColumnReader extends ColumnAccessor {
+
+  /**
+   * Report if the column is null. Non-nullable columns always
+   * return <tt>false</tt>.
+   * @return true if the column value is null, false if the
+   * value is set
+   */
+  boolean isNull();
+  int getInt();
+  long getLong();
+  double getDouble();
+  String getString();
+  byte[] getBytes();
+  BigDecimal getDecimal();
+  Period getPeriod();
+  TupleReader map();
+  ArrayReader array();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriter.java
new file mode 100644
index 0000000..0cc691c
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.accessor;
+
+/**
+ * Defines a writer to set values for value vectors using
+ * a simple, uniform interface. Vector values are mapped to
+ * their "natural" representations: the representation closest
+ * to the actual vector value. For date and time values, this
+ * generally means a numeric value. Applications can then map
+ * this value to Java objects as desired. Decimal types all
+ * map to BigDecimal as that is the only way in Java to
+ * represent large decimal values.
+ * <p>
+ * In general, a column maps to just one value. However, derived
+ * classes may choose to provide type conversions if convenient.
+ * An exception is thrown if a call is made to a method that
+ * is not supported by the column type.
+ * <p>
+ * Values of scalars are set directly, using the get method
+ * for the target type. Maps and arrays are structured types and
+ * require another level of writer abstraction to access each value
+ * in the structure.
+ */
+
+public interface ColumnWriter extends ColumnAccessor, ScalarWriter {
+  void setNull();
+  TupleWriter map();
+  ArrayWriter array();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
new file mode 100644
index 0000000..5cbe80a
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarWriter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.accessor;
+
+import java.math.BigDecimal;
+
+import org.joda.time.Period;
+
+/**
+ * Methods common to the {@link ColumnWriter} and
+ * {@link ArrayWriter} interfaces.
+ */
+
+public interface ScalarWriter {
+  void setInt(int value);
+  void setLong(long value);
+  void setDouble(double value);
+  void setString(String value);
+  void setBytes(byte[] value);
+  void setDecimal(BigDecimal value);
+  void setPeriod(Period value);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
new file mode 100644
index 0000000..2ebb32c
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.accessor;
+
+import org.apache.drill.exec.record.MaterializedField;
+
+/**
+ * Provides access to a "tuple". In Drill, both rows and maps are
+ * tuples: both are an ordered collection of values, defined by a
+ * schema. Each tuple has a schema that defines the column ordering
+ * for indexed access. Each tuple also provides methods to get column
+ * accessors by name or index.
+ */
+
+public interface TupleAccessor {
+
+  /**
+   * Flattened view of the schema as needed for row-based access of scalar
+   * members. The scalar view presents scalar fields: those that can be set
+   * or retrieved. A separate map view presents map vectors. The scalar
+   * view is the one used by row set readers and writers. Column indexes
+   * are into the flattened view, with maps removed and map members flattened
+   * into the top-level name space with compound names.
+   */
+
+  public interface TupleSchema {
+    /**
+     * Return a column schema given an indexed into the flattened row structure.
+     *
+     * @param index index of the row in the flattened structure
+     * @return schema of the column
+     */
+
+    MaterializedField column(int index);
+
+    MaterializedField column(String name);
+
+    int columnIndex(String name);
+
+    int count();
+  }
+
+  TupleSchema schema();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
new file mode 100644
index 0000000..57425af
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleReader.java
@@ -0,0 +1,33 @@
+/*
+ * 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.accessor;
+
+/**
+ * Interface for reading from tuples (rows or maps). Provides
+ * a column reader for each column that can be obtained either
+ * by name or column index (as defined in the tuple schema.)
+ * Also provides two generic methods to get the value as a
+ * Java object or as a string.
+ */
+
+public interface TupleReader extends TupleAccessor {
+  ColumnReader column(int colIndex);
+  ColumnReader column(String colName);
+  Object get(int colIndex);
+  String getAsString(int colIndex);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
new file mode 100644
index 0000000..59eca79
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.accessor;
+
+/**
+ * Interface for writing to rows via a column writer.
+ * Column writers can be obtained by name or index. Column
+ * indexes are defined by the tuple schema. Also provides
+ * a convenience method to set the column value from a Java
+ * object. The caller is responsible for providing the
+ * correct object type for each column. (The object type
+ * must match the column accessor type.)
+ */
+
+public interface TupleWriter extends TupleAccessor {
+  ColumnWriter column(int colIndex);
+  ColumnWriter column(String colName);
+  void set(int colIndex, Object value);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
new file mode 100644
index 0000000..deea7f8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
@@ -0,0 +1,128 @@
+/*
+ * 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.accessor.impl;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader.VectorAccessor;
+import org.joda.time.Period;
+
+/**
+ * Reader for an array-valued column. This reader provides access to specific
+ * array members via an array index. This is an abstract base class;
+ * subclasses are generated for each repeated value vector type.
+ */
+
+public abstract class AbstractArrayReader extends AbstractColumnAccessor implements ArrayReader {
+
+  /**
+   * Column reader that provides access to an array column by returning a
+   * separate reader specifically for that array. That is, reading a column
+   * is a two-part process:<pre><code>
+   * tupleReader.column("arrayCol").array().getInt(2);</code></pre>
+   * This pattern is used to avoid overloading the column reader with
+   * both scalar and array access. Also, this pattern mimics the way
+   * that nested tuples (Drill maps) are handled.
+   */
+
+  public static class ArrayColumnReader extends AbstractColumnReader {
+
+    private final AbstractArrayReader arrayReader;
+
+    public ArrayColumnReader(AbstractArrayReader arrayReader) {
+      this.arrayReader = arrayReader;
+    }
+
+    @Override
+    public ValueType valueType() {
+       return ValueType.ARRAY;
+    }
+
+    @Override
+    public void bind(RowIndex rowIndex, ValueVector vector) {
+      arrayReader.bind(rowIndex, vector);
+      vectorIndex = rowIndex;
+    }
+
+    @Override
+    public ArrayReader array() {
+      return arrayReader;
+    }
+  }
+
+  protected VectorAccessor vectorAccessor;
+
+  public void bind(RowIndex rowIndex, MaterializedField field, VectorAccessor va) {
+    bind(rowIndex);
+    vectorAccessor = va;
+  }
+
+  @Override
+  public boolean isNull(int index) {
+    return false;
+  }
+
+  @Override
+  public int getInt(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getLong(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double getDouble(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getString(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte[] getBytes(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BigDecimal getDecimal(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Period getPeriod(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleReader map(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ArrayReader array(int index) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
new file mode 100644
index 0000000..d1d1263
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
@@ -0,0 +1,127 @@
+/*
+ * 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.accessor.impl;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
+import org.joda.time.Period;
+
+/**
+ * Writer for an array-valued column. This writer appends values: once a value
+ * is written, it cannot be changed. As a result, writer methods have no item index;
+ * each set advances the array to the next position. This is an abstract base class;
+ * subclasses are generated for each repeated value vector type.
+ */
+
+public abstract class AbstractArrayWriter extends AbstractColumnAccessor implements ArrayWriter {
+
+  /**
+   * Column writer that provides access to an array column by returning a
+   * separate writer specifically for that array. That is, writing an array
+   * is a two-part process:<pre><code>
+   * tupleWriter.column("arrayCol").array().setInt(2);</code></pre>
+   * This pattern is used to avoid overloading the column reader with
+   * both scalar and array access. Also, this pattern mimics the way
+   * that nested tuples (Drill maps) are handled.
+   */
+
+  public static class ArrayColumnWriter extends AbstractColumnWriter {
+
+    private final AbstractArrayWriter arrayWriter;
+
+    public ArrayColumnWriter(AbstractArrayWriter arrayWriter) {
+      this.arrayWriter = arrayWriter;
+    }
+
+    @Override
+    public ValueType valueType() {
+      return ValueType.ARRAY;
+    }
+
+    @Override
+    public void bind(RowIndex rowIndex, ValueVector vector) {
+      arrayWriter.bind(rowIndex, vector);
+      vectorIndex = rowIndex;
+    }
+
+    @Override
+    public ArrayWriter array() {
+      return arrayWriter;
+    }
+
+    /**
+     * Arrays require a start step for each row, regardless of
+     * whether any values are written for that row.
+     */
+
+    public void start() {
+      arrayWriter.mutator().startNewValue(vectorIndex.index());
+    }
+  }
+
+  protected abstract BaseRepeatedValueVector.BaseRepeatedMutator mutator();
+
+  @Override
+  public int size() {
+    return mutator().getInnerValueCountAt(vectorIndex.index());
+  }
+
+  @Override
+  public boolean valid() {
+    // Not implemented yet
+    return true;
+  }
+
+  @Override
+  public void setInt(int value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setLong(long value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setDouble(double value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setString(String value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setBytes(byte[] value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setDecimal(BigDecimal value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setPeriod(Period value) {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
new file mode 100644
index 0000000..5b751c5
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
@@ -0,0 +1,43 @@
+/*
+ * 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.accessor.impl;
+
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Abstract base class for column readers and writers that
+ * implements the mechanism for binding accessors to a row
+ * index. The row index is implicit: index a row, then
+ * column accessors pull out columns from that row.
+ */
+
+public abstract class AbstractColumnAccessor {
+
+  public interface RowIndex {
+    int batch();
+    int index();
+  }
+
+  protected RowIndex vectorIndex;
+
+  protected void bind(RowIndex rowIndex) {
+    this.vectorIndex = rowIndex;
+  }
+
+  public abstract void bind(RowIndex rowIndex, ValueVector vector);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
new file mode 100644
index 0000000..1ef2243
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
@@ -0,0 +1,98 @@
+/*
+ * 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.accessor.impl;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ColumnReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.joda.time.Period;
+
+/**
+ * Column reader implementation that acts as the basis for the
+ * generated, vector-specific implementations. All set methods
+ * throw an exception; subclasses simply override the supported
+ * method(s).
+ */
+
+public abstract class AbstractColumnReader extends AbstractColumnAccessor implements ColumnReader {
+
+  public interface VectorAccessor {
+    ValueVector vector();
+  }
+
+  protected VectorAccessor vectorAccessor;
+
+  public void bind(RowIndex rowIndex, MaterializedField field, VectorAccessor va) {
+    bind(rowIndex);
+    vectorAccessor = va;
+  }
+
+  @Override
+  public boolean isNull() {
+    return false;
+  }
+
+  @Override
+  public int getInt() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long getLong() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double getDouble() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String getString() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public byte[] getBytes() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BigDecimal getDecimal() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Period getPeriod() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleReader map() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ArrayReader array() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
new file mode 100644
index 0000000..5071e03
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
@@ -0,0 +1,87 @@
+/*
+ * 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.accessor.impl;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.joda.time.Period;
+
+/**
+ * Column writer implementation that acts as the basis for the
+ * generated, vector-specific implementations. All set methods
+ * throw an exception; subclasses simply override the supported
+ * method(s).
+ */
+
+public abstract class AbstractColumnWriter extends AbstractColumnAccessor implements ColumnWriter {
+
+  public void start() { }
+
+  @Override
+  public void setNull() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setInt(int value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setLong(long value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setDouble(double value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setString(String value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setBytes(byte[] value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setDecimal(BigDecimal value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setPeriod(Period value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleWriter map() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ArrayWriter array() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
new file mode 100644
index 0000000..98ea6ac
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
@@ -0,0 +1,38 @@
+/*
+ * 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.accessor.impl;
+
+import org.apache.drill.exec.vector.accessor.TupleAccessor;
+
+/**
+ * Common base class for tuple readers and writers.
+ */
+
+public abstract class AbstractTupleAccessor implements TupleAccessor {
+
+  protected final TupleSchema schema;
+
+  public AbstractTupleAccessor(TupleSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  public TupleSchema schema() {
+    return schema;
+  }
+}


[3/3] drill git commit: Test-specific column accessor implementation. Provides a simplified, unified set of access methods for value vectors specifically for wrting simple, compact unit test code.

Posted by am...@apache.org.
Test-specific column accessor implementation. Provides a simplified, unified set of access methods for value vectors specifically for wrting simple, compact unit test code.

* Interfaces for column readers and writers
* Interfaces for tuple (row and map) readers and writers
* Generated implementations
* Base implementation used by the generated code
* Factory class to create the proper reader or writer given a major
type (type and cardinality)
* Utilities for generic access, type conversions, etc.

Many vector types can be mapped to an int for get and set. One key
exception are the decimal types: decimals, by definition, require a
different representation. In Java, that is `BigDecimal`. Added get, set
and setSafe accessors as required for each decimal type that uses
`BigDecimal` to hold data.

The generated code builds on the `valueVectorTypes.tdd` file, adding
additional properties needed to generate the accessors.

The PR also includes a number of code cleanups done while reviewing
existing code. In particular `DecimalUtility` was very roughly
formatted and thus hard to follow.

Supports Drill\u2019s interval types (INTERVAL, INTERVALDAY,
INTERVALYEAR) in the form of the Joda interval class.

Adds support for Map vectors. Maps are treated as nested tuples and are
expanded out to create a flattened row in the schema. The accessors
then access rows using the flattened column index or the combined name
(\u201ca.b\u201d).

Supports arrays via a writer interface that appends values as written,
and an indexed, random-access reader interface.

Removed HTTP log parser from JDBC jar to keep the JDBC jar from getting
too big.

close apache/drill#783


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

Branch: refs/heads/master
Commit: 381eab668480062471f5bd3897a04a4b322f5d9a
Parents: 72903d0
Author: Paul Rogers <pr...@maprtech.com>
Authored: Fri Mar 10 23:03:23 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Mon Apr 17 09:57:49 2017 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml                           |   4 +
 .../src/main/codegen/data/ValueVectorTypes.tdd  |  61 +-
 .../src/main/codegen/includes/license.ftl       |   5 +-
 .../src/main/codegen/includes/vv_imports.ftl    |  24 +-
 .../codegen/templates/AbstractFieldReader.java  |  34 +-
 .../main/codegen/templates/ColumnAccessors.java | 331 ++++++++
 .../codegen/templates/FixedValueVectors.java    | 301 +++----
 .../codegen/templates/NullableValueVectors.java |  40 +-
 .../codegen/templates/RepeatedValueVectors.java |  40 +-
 .../templates/VariableLengthVectors.java        |   2 +-
 .../drill/exec/record/MaterializedField.java    |  59 +-
 .../apache/drill/exec/util/DecimalUtility.java  | 803 ++++++++++---------
 .../drill/exec/vector/BaseValueVector.java      |   2 -
 .../exec/vector/accessor/AccessorUtilities.java | 125 +++
 .../drill/exec/vector/accessor/ArrayReader.java |  52 ++
 .../drill/exec/vector/accessor/ArrayWriter.java |  42 +
 .../exec/vector/accessor/ColumnAccessor.java    |  40 +
 .../exec/vector/accessor/ColumnReader.java      |  63 ++
 .../exec/vector/accessor/ColumnWriter.java      |  45 ++
 .../exec/vector/accessor/ScalarWriter.java      |  37 +
 .../exec/vector/accessor/TupleAccessor.java     |  59 ++
 .../drill/exec/vector/accessor/TupleReader.java |  33 +
 .../drill/exec/vector/accessor/TupleWriter.java |  34 +
 .../accessor/impl/AbstractArrayReader.java      | 128 +++
 .../accessor/impl/AbstractArrayWriter.java      | 127 +++
 .../accessor/impl/AbstractColumnAccessor.java   |  43 +
 .../accessor/impl/AbstractColumnReader.java     |  98 +++
 .../accessor/impl/AbstractColumnWriter.java     |  87 ++
 .../accessor/impl/AbstractTupleAccessor.java    |  38 +
 .../accessor/impl/ColumnAccessorFactory.java    | 122 +++
 .../vector/accessor/impl/TupleReaderImpl.java   | 108 +++
 .../vector/accessor/impl/TupleWriterImpl.java   | 162 ++++
 .../exec/vector/accessor/impl/package-info.java |  27 +
 .../exec/vector/accessor/package-info.java      | 127 +++
 .../vector/complex/BaseRepeatedValueVector.java |   4 +
 .../exec/vector/complex/RepeatedMapVector.java  |   1 -
 36 files changed, 2631 insertions(+), 677 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 1b015e9..17af111 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -143,6 +143,10 @@
           <artifactId>hazelcast</artifactId>
           <groupId>com.hazelcast</groupId>
         </exclusion>
+        <exclusion>
+	      <groupId>nl.basjes.parse.httpdlog</groupId>
+	      <artifactId>httpdlog-parser</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
index 26bf02d..4d719b4 100644
--- a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
+++ b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
@@ -26,6 +26,8 @@
       width: 1,
       javaType: "byte",
       boxedType: "Byte",
+      accessorType: "int",
+      accessorCast: "set",
       fields: [{name: "value", type: "byte"}],
       minor: [
         { class: "TinyInt", valueHolder: "IntHolder" },
@@ -37,15 +39,19 @@
       width: 2,
       javaType: "char",
       boxedType: "Character",
+      accessorType: "int",
       fields: [{name: "value", type: "char"}],
       minor: [
-        { class: "UInt2", valueHolder: "UInt2Holder"}
+        { class: "UInt2", valueHolder: "UInt2Holder", accessorCast: "set"}
       ]
-    },    {
+    },
+    {
       major: "Fixed",
       width: 2,
       javaType: "short",
       boxedType: "Short",
+      accessorType: "int",
+      accessorCast: "set",
       fields: [{name: "value", type: "short"}],
       minor: [
         { class: "SmallInt", valueHolder: "Int2Holder"},
@@ -60,10 +66,13 @@
       minor: [
         { class: "Int", valueHolder: "IntHolder"},
         { class: "UInt4", valueHolder: "UInt4Holder" },
-        { class: "Float4", javaType: "float" , boxedType: "Float", fields: [{name: "value", type: "float"}]},
-        { class: "Time", javaType: "int", friendlyType: "DateTime" },
+        { class: "Float4", javaType: "float" , boxedType: "Float", accessorType: "double", accessorCast: "set",
+          fields: [{name: "value", type: "float"}]},
+        { class: "Time", javaType: "int", friendlyType: "DateTime", accessorType: "int" },
         { class: "IntervalYear", javaType: "int", friendlyType: "Period" }
-        { class: "Decimal9", maxPrecisionDigits: 9, friendlyType: "BigDecimal", fields: [{name:"value", type:"int"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] },
+        { class: "Decimal9", maxPrecisionDigits: 9, friendlyType: "BigDecimal",
+          fields: [{name:"value", type:"int"}, {name: "scale", type: "int", include: false},
+                   {name: "precision", type: "int", include: false}] },
       ]
     },
     {
@@ -76,9 +85,11 @@
         { class: "BigInt"},
         { class: "UInt8" },
         { class: "Float8", javaType: "double" , boxedType: "Double", fields: [{name: "value", type: "double"}], },
-        { class: "Date", javaType: "long", friendlyType: "DateTime" },
-        { class: "TimeStamp", javaType: "long", friendlyType: "DateTime" }
-        { class: "Decimal18", maxPrecisionDigits: 18, friendlyType: "BigDecimal", fields: [{name:"value", type:"long"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] },
+        { class: "Date", javaType: "long", friendlyType: "DateTime", accessorType: "long" },
+        { class: "TimeStamp", javaType: "long", friendlyType: "DateTime", accessorType: "long" }
+        { class: "Decimal18", maxPrecisionDigits: 18, friendlyType: "BigDecimal",
+          fields: [{name:"value", type:"long"}, {name: "scale", type: "int", include: false},
+                   {name: "precision", type: "int", include: false}] },
         <#--
         { class: "Money", maxPrecisionDigits: 2, scale: 1, },
         -->
@@ -90,16 +101,18 @@
       javaType: "DrillBuf",
       boxedType: "DrillBuf",
       minor: [
-        { class: "IntervalDay", millisecondsOffset: 4, friendlyType: "Period", fields: [ {name: "days", type:"int"}, {name: "milliseconds", type:"int"}] }
+        { class: "IntervalDay", millisecondsOffset: 4, friendlyType: "Period",
+          fields: [ {name: "days", type:"int"}, {name: "milliseconds", type:"int"}] }
       ]
     },
     {
       major: "Fixed",
       width: 16,
       javaType: "DrillBuf"
-      boxedType: "DrillBuf",      
+      boxedType: "DrillBuf",
       minor: [
-        { class: "Interval", daysOffset: 4, millisecondsOffset: 8, friendlyType: "Period", fields: [ {name: "months", type: "int"}, {name: "days", type:"int"}, {name: "milliseconds", type:"int"}] }
+        { class: "Interval", daysOffset: 4, millisecondsOffset: 8, friendlyType: "Period",
+          fields: [ {name: "months", type: "int"}, {name: "days", type:"int"}, {name: "milliseconds", type:"int"}] }
       ]
     },
     {
@@ -107,12 +120,15 @@
       width: 12,
       javaType: "DrillBuf",
       boxedType: "DrillBuf",
+      accessorDisabled: true,
       minor: [
         <#--
         { class: "TimeTZ" },
         { class: "Interval" }
         -->
-        { class: "Decimal28Dense", maxPrecisionDigits: 28, nDecimalDigits: 3, friendlyType: "BigDecimal", fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
+        { class: "Decimal28Dense", maxPrecisionDigits: 28, nDecimalDigits: 3, friendlyType: "BigDecimal",
+          fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"},
+                   {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
       ]
     },
     {
@@ -120,9 +136,11 @@
       width: 16,
       javaType: "DrillBuf",
       boxedType: "DrillBuf",
-      
+      accessorDisabled: true,
       minor: [
-        { class: "Decimal38Dense", maxPrecisionDigits: 38, nDecimalDigits: 4, friendlyType: "BigDecimal", fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
+        { class: "Decimal38Dense", maxPrecisionDigits: 38, nDecimalDigits: 4,friendlyType: "BigDecimal",
+          fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"},
+                   {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
       ]
     },
     {
@@ -131,7 +149,9 @@
       javaType: "DrillBuf",
       boxedType: "DrillBuf",
       minor: [
-        { class: "Decimal38Sparse", maxPrecisionDigits: 38, nDecimalDigits: 6, friendlyType: "BigDecimal", fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
+        { class: "Decimal38Sparse", maxPrecisionDigits: 38, nDecimalDigits: 6, friendlyType: "BigDecimal",
+          fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"},
+                   {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
       ]
     },
     {
@@ -140,7 +160,9 @@
       javaType: "DrillBuf",
       boxedType: "DrillBuf",
       minor: [
-        { class: "Decimal28Sparse", maxPrecisionDigits: 28, nDecimalDigits: 5, friendlyType: "BigDecimal", fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"}, {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
+        { class: "Decimal28Sparse", maxPrecisionDigits: 28, nDecimalDigits: 5, friendlyType: "BigDecimal",
+          fields: [{name: "start", type: "int"}, {name: "buffer", type: "DrillBuf"},
+                   {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
       ]
     },
     {
@@ -150,8 +172,8 @@
       boxedType: "DrillBuf",
       fields: [{name: "start", type: "int"}, {name: "end", type: "int"}, {name: "buffer", type: "DrillBuf"}],
       minor: [
-        { class: "VarBinary" , friendlyType: "byte[]" },
-        { class: "VarChar" , friendlyType: "Text" },
+        { class: "VarBinary" , friendlyType: "byte[]", accessorType: "byte[]", accessorLabel: "Bytes" },
+        { class: "VarChar" , friendlyType: "Text", accessorType: "String" },
         { class: "Var16Char" , friendlyType: "String" }
       ]
     },
@@ -161,7 +183,8 @@
       javaType: "int",
       boxedType: "Integer",
       minor: [
-        { class: "Bit" , friendlyType: "Boolean", fields: [{name: "value", type: "int"}] }
+        { class: "Bit" , friendlyType: "Boolean", accessorType: "int"
+          fields: [{name: "value", type: "int"}] }
       ]
     }
   ]

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/includes/license.ftl
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/includes/license.ftl b/exec/vector/src/main/codegen/includes/license.ftl
index 0455fd8..586b456 100644
--- a/exec/vector/src/main/codegen/includes/license.ftl
+++ b/exec/vector/src/main/codegen/includes/license.ftl
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/*
  * 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
@@ -15,4 +14,4 @@
  * 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.
- ******************************************************************************/
\ No newline at end of file
+ ******************************************************************************/

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/includes/vv_imports.ftl
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/includes/vv_imports.ftl b/exec/vector/src/main/codegen/includes/vv_imports.ftl
index 11e0f5d..87a2106 100644
--- a/exec/vector/src/main/codegen/includes/vv_imports.ftl
+++ b/exec/vector/src/main/codegen/includes/vv_imports.ftl
@@ -1,12 +1,12 @@
-<#-- 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 
+<#-- 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. -->
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -65,9 +65,3 @@ import org.joda.time.DateTime;
 import org.joda.time.Period;
 
 import org.apache.drill.exec.util.Text;
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
index 289aa60..0e48853 100644
--- a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
+++ b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -31,7 +31,7 @@ package org.apache.drill.exec.vector.complex.impl;
  */
 @SuppressWarnings("unused")
 abstract class AbstractFieldReader extends AbstractBaseReader implements FieldReader{
-  
+
   AbstractFieldReader(){
     super();
   }
@@ -44,24 +44,24 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
     return true;
   }
 
-  <#list ["Object", "BigDecimal", "Integer", "Long", "Boolean", 
+  <#list ["Object", "BigDecimal", "Integer", "Long", "Boolean",
           "Character", "DateTime", "Period", "Double", "Float",
           "Text", "String", "Byte", "Short", "byte[]"] as friendlyType>
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
-  
+
   public ${friendlyType} read${safeType}(int arrayIndex){
     fail("read${safeType}(int arrayIndex)");
     return null;
   }
-  
+
   public ${friendlyType} read${safeType}(){
     fail("read${safeType}()");
     return null;
   }
-  
+
   </#list>
-  
+
   public void copyAsValue(MapWriter writer){
     fail("CopyAsValue MapWriter");
   }
@@ -72,7 +72,7 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
   public void copyAsField(String name, ListWriter writer){
     fail("CopyAsFieldList");
   }
-  
+
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign boxedType = (minor.boxedType!type.boxedType) />
 
@@ -83,15 +83,15 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
   public void read(Nullable${name}Holder holder){
     fail("${name}");
   }
-  
+
   public void read(int arrayIndex, ${name}Holder holder){
     fail("Repeated${name}");
   }
-  
+
   public void read(int arrayIndex, Nullable${name}Holder holder){
     fail("Repeated${name}");
   }
-  
+
   public void copyAsValue(${name}Writer writer){
     fail("CopyAsValue${name}");
   }
@@ -99,7 +99,7 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
     fail("CopyAsField${name}");
   }
   </#list></#list>
-  
+
   public FieldReader reader(String name){
     fail("reader(String name)");
     return null;
@@ -108,19 +108,19 @@ abstract class AbstractFieldReader extends AbstractBaseReader implements FieldRe
   public FieldReader reader(){
     fail("reader()");
     return null;
-    
+
   }
-  
+
   public int size(){
     fail("size()");
     return -1;
   }
-  
+
   private void fail(String name){
     throw new IllegalArgumentException(String.format("You tried to read a [%s] type when you are using a field reader of type [%s].", name, this.getClass().getSimpleName()));
   }
-  
-  
+
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/ColumnAccessors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/ColumnAccessors.java b/exec/vector/src/main/codegen/templates/ColumnAccessors.java
new file mode 100644
index 0000000..f1fbf2f
--- /dev/null
+++ b/exec/vector/src/main/codegen/templates/ColumnAccessors.java
@@ -0,0 +1,331 @@
+/*
+ * 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.
+ */
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="/org/apache/drill/exec/vector/accessor/ColumnAccessors.java" />
+<#include "/@includes/license.ftl" />
+<#macro getType label>
+    @Override
+    public ValueType valueType() {
+  <#if label == "Int">
+      return ValueType.INTEGER;
+  <#else>
+      return ValueType.${label?upper_case};
+  </#if>
+    }
+</#macro>
+<#macro bindReader prefix drillType>
+  <#if drillType = "Decimal9" || drillType == "Decimal18">
+    private MaterializedField field;
+  </#if>
+    private ${prefix}${drillType}Vector.Accessor accessor;
+
+    @Override
+    public void bind(RowIndex vectorIndex, ValueVector vector) {
+      bind(vectorIndex);
+  <#if drillType = "Decimal9" || drillType == "Decimal18">
+      field = vector.getField();
+  </#if>
+      accessor = ((${prefix}${drillType}Vector) vector).getAccessor();
+    }
+
+  <#if drillType = "Decimal9" || drillType == "Decimal18">
+    @Override
+    public void bind(RowIndex vectorIndex, MaterializedField field, VectorAccessor va) {
+      bind(vectorIndex, field, va);
+      this.field = field;
+    }
+
+ </#if>
+   private ${prefix}${drillType}Vector.Accessor accessor() {
+      if (vectorAccessor == null) {
+        return accessor;
+      } else {
+        return ((${prefix}${drillType}Vector) vectorAccessor.vector()).getAccessor();
+      }
+    }
+</#macro>
+<#macro get drillType accessorType label isArray>
+    @Override
+    public ${accessorType} get${label}(<#if isArray>int index</#if>) {
+  <#if isArray>
+    <#assign index=", index"/>
+    <#assign getObject="getSingleObject">
+  <#else>
+    <#assign index=""/>
+    <#assign getObject="getObject">
+  </#if>
+  <#if drillType == "VarChar">
+      return new String(accessor().get(vectorIndex.index()${index}), Charsets.UTF_8);
+  <#elseif drillType == "Var16Char">
+      return new String(accessor().get(vectorIndex.index()${index}), Charsets.UTF_16);
+  <#elseif drillType == "VarBinary">
+      return accessor().get(vectorIndex.index()${index});
+  <#elseif drillType == "Decimal9" || drillType == "Decimal18">
+      return DecimalUtility.getBigDecimalFromPrimitiveTypes(
+                accessor().get(vectorIndex.index()${index}),
+                field.getScale(),
+                field.getPrecision());
+  <#elseif accessorType == "BigDecimal" || accessorType == "Period">
+      return accessor().${getObject}(vectorIndex.index()${index});
+  <#else>
+      return accessor().get(vectorIndex.index()${index});
+  </#if>
+    }
+</#macro>
+<#macro bindWriter prefix drillType>
+  <#if drillType = "Decimal9" || drillType == "Decimal18">
+    private MaterializedField field;
+  </#if>
+    private ${prefix}${drillType}Vector.Mutator mutator;
+
+    @Override
+    public void bind(RowIndex vectorIndex, ValueVector vector) {
+      bind(vectorIndex);
+  <#if drillType = "Decimal9" || drillType == "Decimal18">
+      field = vector.getField();
+  </#if>
+      this.mutator = ((${prefix}${drillType}Vector) vector).getMutator();
+    }
+</#macro>
+<#macro set drillType accessorType label nullable verb>
+    @Override
+    public void set${label}(${accessorType} value) {
+  <#if drillType == "VarChar">
+      byte bytes[] = value.getBytes(Charsets.UTF_8);
+      mutator.${verb}Safe(vectorIndex.index(), bytes, 0, bytes.length);
+  <#elseif drillType == "Var16Char">
+      byte bytes[] = value.getBytes(Charsets.UTF_16);
+      mutator.${verb}Safe(vectorIndex.index(), bytes, 0, bytes.length);
+  <#elseif drillType == "VarBinary">
+      mutator.${verb}Safe(vectorIndex.index(), value, 0, value.length);
+  <#elseif drillType == "Decimal9">
+      mutator.${verb}Safe(vectorIndex.index(),
+          DecimalUtility.getDecimal9FromBigDecimal(value,
+              field.getScale(), field.getPrecision()));
+  <#elseif drillType == "Decimal18">
+      mutator.${verb}Safe(vectorIndex.index(),
+          DecimalUtility.getDecimal18FromBigDecimal(value,
+              field.getScale(), field.getPrecision()));
+  <#elseif drillType == "IntervalYear">
+      mutator.${verb}Safe(vectorIndex.index(), value.getYears() * 12 + value.getMonths());
+  <#elseif drillType == "IntervalDay">
+      mutator.${verb}Safe(vectorIndex.index(),<#if nullable> 1,</#if>
+                      value.getDays(),
+                      ((value.getHours() * 60 + value.getMinutes()) * 60 +
+                       value.getSeconds()) * 1000 + value.getMillis());
+  <#elseif drillType == "Interval">
+      mutator.${verb}Safe(vectorIndex.index(),<#if nullable> 1,</#if>
+                      value.getYears() * 12 + value.getMonths(),
+                      value.getDays(),
+                      ((value.getHours() * 60 + value.getMinutes()) * 60 +
+                       value.getSeconds()) * 1000 + value.getMillis());
+  <#else>
+      mutator.${verb}Safe(vectorIndex.index(), <#if cast=="set">(${javaType}) </#if>value);
+  </#if>
+    }
+</#macro>
+
+package org.apache.drill.exec.vector.accessor;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.vector.*;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.util.DecimalUtility;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
+import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
+import org.apache.drill.exec.vector.accessor.impl.AbstractArrayReader;
+import org.apache.drill.exec.vector.accessor.impl.AbstractArrayWriter;
+import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader.VectorAccessor;
+
+import com.google.common.base.Charsets;
+import org.joda.time.Period;
+
+/**
+ * Basic accessors for most Drill vector types and modes. These are bare-bones
+ * accessors: they do only the most rudimentary type conversions. For all,
+ * there is only one way to get/set values; they don't convert from, say,
+ * a double to an int or visa-versa.
+ * <p>
+ * Writers work only with single vectors. Readers work with either single
+ * vectors or a "hyper vector": a collection of vectors indexed together.
+ * The details are hidden behind the {@link RowIndex} interface. If the reader
+ * accesses a single vector, then the mutator is cached at bind time. However,
+ * if the reader works with a hyper vector, then the vector is null at bind
+ * time and must be retrieved for each row (since the vector differs row-by-
+ * row.)
+ */
+
+// This class is generated using freemarker and the ${.template_name} template.
+
+public class ColumnAccessors {
+
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#assign drillType=minor.class>
+    <#assign javaType=minor.javaType!type.javaType>
+    <#assign accessorType=minor.accessorType!type.accessorType!minor.friendlyType!javaType>
+    <#assign label=minor.accessorLabel!type.accessorLabel!accessorType?capitalize>
+    <#assign notyet=minor.accessorDisabled!type.accessorDisabled!false>
+    <#assign cast=minor.accessorCast!minor.accessorCast!type.accessorCast!"none">
+    <#assign friendlyType=minor.friendlyType!"">
+    <#if accessorType=="BigDecimal">
+      <#assign label="Decimal">
+    </#if>
+    <#if ! notyet>
+  //------------------------------------------------------------------------
+  // ${drillType} readers and writers
+
+  public static class ${drillType}ColumnReader extends AbstractColumnReader {
+
+    <@bindReader "" drillType />
+
+    <@getType label />
+
+    <@get drillType accessorType label false/>
+  }
+
+  public static class Nullable${drillType}ColumnReader extends AbstractColumnReader {
+
+    <@bindReader "Nullable" drillType />
+
+    <@getType label />
+
+    @Override
+    public boolean isNull() {
+      return accessor().isNull(vectorIndex.index());
+    }
+
+    <@get drillType accessorType label false/>
+  }
+
+  public static class Repeated${drillType}ColumnReader extends AbstractArrayReader {
+
+    <@bindReader "Repeated" drillType />
+
+    <@getType label />
+
+    @Override
+    public int size() {
+      return accessor().getInnerValueCountAt(vectorIndex.index());
+    }
+
+    <@get drillType accessorType label true/>
+  }
+
+  public static class ${drillType}ColumnWriter extends AbstractColumnWriter {
+
+    <@bindWriter "" drillType />
+
+    <@getType label />
+
+    <@set drillType accessorType label false "set" />
+  }
+
+  public static class Nullable${drillType}ColumnWriter extends AbstractColumnWriter {
+
+    <@bindWriter "Nullable" drillType />
+
+    <@getType label />
+
+    @Override
+    public void setNull() {
+      mutator.setNull(vectorIndex.index());
+    }
+
+    <@set drillType accessorType label true "set" />
+  }
+
+  public static class Repeated${drillType}ColumnWriter extends AbstractArrayWriter {
+
+    <@bindWriter "Repeated" drillType />
+
+    <@getType label />
+
+    protected BaseRepeatedValueVector.BaseRepeatedMutator mutator() {
+      return mutator;
+    }
+
+    <@set drillType accessorType label false "add" />
+  }
+
+    </#if>
+  </#list>
+</#list>
+  public static void defineReaders(
+      Class<? extends AbstractColumnReader> readers[][]) {
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#assign drillType=minor.class>
+    <#assign notyet=minor.accessorDisabled!type.accessorDisabled!false>
+    <#if ! notyet>
+    <#assign typeEnum=drillType?upper_case>
+    readers[MinorType.${typeEnum}.ordinal()][DataMode.REQUIRED.ordinal()] = ${drillType}ColumnReader.class;
+    readers[MinorType.${typeEnum}.ordinal()][DataMode.OPTIONAL.ordinal()] = Nullable${drillType}ColumnReader.class;
+    </#if>
+  </#list>
+</#list>
+  }
+
+  public static void defineWriters(
+      Class<? extends AbstractColumnWriter> writers[][]) {
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#assign drillType=minor.class>
+    <#assign notyet=minor.accessorDisabled!type.accessorDisabled!false>
+    <#if ! notyet>
+    <#assign typeEnum=drillType?upper_case>
+    writers[MinorType.${typeEnum}.ordinal()][DataMode.REQUIRED.ordinal()] = ${drillType}ColumnWriter.class;
+    writers[MinorType.${typeEnum}.ordinal()][DataMode.OPTIONAL.ordinal()] = Nullable${drillType}ColumnWriter.class;
+    </#if>
+  </#list>
+</#list>
+  }
+
+  public static void defineArrayReaders(
+      Class<? extends AbstractArrayReader> readers[]) {
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#assign drillType=minor.class>
+    <#assign notyet=minor.accessorDisabled!type.accessorDisabled!false>
+    <#if ! notyet>
+    <#assign typeEnum=drillType?upper_case>
+    readers[MinorType.${typeEnum}.ordinal()] = Repeated${drillType}ColumnReader.class;
+    </#if>
+  </#list>
+</#list>
+  }
+
+  public static void defineArrayWriters(
+      Class<? extends AbstractArrayWriter> writers[]) {
+<#list vv.types as type>
+  <#list type.minor as minor>
+    <#assign drillType=minor.class>
+    <#assign notyet=minor.accessorDisabled!type.accessorDisabled!false>
+    <#if ! notyet>
+    <#assign typeEnum=drillType?upper_case>
+    writers[MinorType.${typeEnum}.ordinal()] = Repeated${drillType}ColumnWriter.class;
+    </#if>
+  </#list>
+</#list>
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/FixedValueVectors.java b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
index b2a5dc3..23188ce 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -16,8 +16,6 @@
  * limitations under the License.
  */
 
-import java.lang.Override;
-
 <@pp.dropOutputFile />
 <#list vv.types as type>
 <#list type.minor as minor>
@@ -30,6 +28,7 @@ import java.lang.Override;
 package org.apache.drill.exec.vector;
 
 <#include "/@includes/vv_imports.ftl" />
+import org.apache.drill.exec.util.DecimalUtility;
 
 /**
  * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
@@ -137,7 +136,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     allocationMonitor = 0;
     zeroVector();
     super.reset();
-    }
+  }
 
   private void allocateBytes(final long size) {
     if (size > MAX_ALLOCATION_SIZE) {
@@ -198,6 +197,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     data.writerIndex(actualLength);
   }
 
+  @Override
   public TransferPair getTransferPair(BufferAllocator allocator){
     return new TransferImpl(getField(), allocator);
   }
@@ -304,7 +304,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
     <#if (type.width > 8)>
-
     public ${minor.javaType!type.javaType} get(int index) {
       return data.slice(index * ${type.width}, ${type.width});
     }
@@ -423,27 +422,27 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     <#elseif (minor.class == "Decimal28Sparse") || (minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")>
 
     public void get(int index, ${minor.class}Holder holder) {
-        holder.start = index * ${type.width};
-        holder.buffer = data;
-        holder.scale = getField().getScale();
-        holder.precision = getField().getPrecision();
+      holder.start = index * ${type.width};
+      holder.buffer = data;
+      holder.scale = getField().getScale();
+      holder.precision = getField().getPrecision();
     }
 
     public void get(int index, Nullable${minor.class}Holder holder) {
-        holder.isSet = 1;
-        holder.start = index * ${type.width};
-        holder.buffer = data;
-        holder.scale = getField().getScale();
-        holder.precision = getField().getPrecision();
+      holder.isSet = 1;
+      holder.start = index * ${type.width};
+      holder.buffer = data;
+      holder.scale = getField().getScale();
+      holder.precision = getField().getPrecision();
     }
 
-      @Override
-      public ${friendlyType} getObject(int index) {
+    @Override
+    public ${friendlyType} getObject(int index) {
       <#if (minor.class == "Decimal28Sparse") || (minor.class == "Decimal38Sparse")>
       // Get the BigDecimal object
-      return org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale());
+      return DecimalUtility.getBigDecimalFromSparse(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale());
       <#else>
-      return org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDense(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale(), ${minor.maxPrecisionDigits}, ${type.width});
+      return DecimalUtility.getBigDecimalFromDense(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale(), ${minor.maxPrecisionDigits}, ${type.width});
       </#if>
     }
 
@@ -585,150 +584,165 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     * @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, 0, ${type.width});
-   }
+    public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      data.setBytes(index * ${type.width}, value, 0, ${type.width});
+    }
 
-   public void setSafe(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     data.setBytes(index * ${type.width}, value, 0, ${type.width});
-   }
+    public void setSafe(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      data.setBytes(index * ${type.width}, value, 0, ${type.width});
+    }
 
   <#if (minor.class == "Interval")>
-   public void set(int index, int months, int days, int milliseconds){
-     final int offsetIndex = index * ${type.width};
-     data.setInt(offsetIndex, months);
-     data.setInt((offsetIndex + ${minor.daysOffset}), days);
-     data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
-   }
-
-   protected void set(int index, ${minor.class}Holder holder){
-     set(index, holder.months, holder.days, holder.milliseconds);
-   }
-
-   protected void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.months, holder.days, holder.milliseconds);
-   }
-
-   public void setSafe(int index, int months, int days, int milliseconds){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, months, days, milliseconds);
-   }
-
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.months, holder.days, holder.milliseconds);
-   }
-
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.months, holder.days, holder.milliseconds);
-   }
-
-   <#elseif (minor.class == "IntervalDay")>
-   public void set(int index, int days, int milliseconds){
-     final int offsetIndex = index * ${type.width};
-     data.setInt(offsetIndex, days);
-     data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
-   }
-
-   protected void set(int index, ${minor.class}Holder holder){
-     set(index, holder.days, holder.milliseconds);
-   }
-   protected void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.days, holder.milliseconds);
-   }
-
-   public void setSafe(int index, int days, int milliseconds){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, days, milliseconds);
-   }
+    public void set(int index, int months, int days, int milliseconds){
+      final int offsetIndex = index * ${type.width};
+      data.setInt(offsetIndex, months);
+      data.setInt((offsetIndex + ${minor.daysOffset}), days);
+      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
+    }
 
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.days, holder.milliseconds);
-   }
+    protected void set(int index, ${minor.class}Holder holder){
+      set(index, holder.months, holder.days, holder.milliseconds);
+    }
 
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.days, holder.milliseconds);
-   }
+    protected void set(int index, Nullable${minor.class}Holder holder){
+      set(index, holder.months, holder.days, holder.milliseconds);
+    }
+
+    public void setSafe(int index, int months, int days, int milliseconds){
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      set(index, months, days, milliseconds);
+    }
+
+    public void setSafe(int index, Nullable${minor.class}Holder holder){
+      setSafe(index, holder.months, holder.days, holder.milliseconds);
+    }
 
-   <#elseif (minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")>
+    public void setSafe(int index, ${minor.class}Holder holder){
+      setSafe(index, holder.months, holder.days, holder.milliseconds);
+    }
 
-   public void set(int index, ${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
+  <#elseif (minor.class == "IntervalDay")>
+    public void set(int index, int days, int milliseconds){
+      final int offsetIndex = index * ${type.width};
+      data.setInt(offsetIndex, days);
+      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
+    }
 
-   void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
+    protected void set(int index, ${minor.class}Holder holder){
+      set(index, holder.days, holder.milliseconds);
+    }
 
-   public void setSafe(int index,  Nullable${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-   public void setSafe(int index,  ${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
+    protected void set(int index, Nullable${minor.class}Holder holder){
+      set(index, holder.days, holder.milliseconds);
+    }
 
-   public void setSafe(int index, int start, DrillBuf buffer){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, start, buffer);
-   }
+    public void setSafe(int index, int days, int milliseconds){
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      set(index, days, milliseconds);
+    }
 
-   public void set(int index, int start, DrillBuf buffer){
-     data.setBytes(index * ${type.width}, buffer, start, ${type.width});
-   }
+    public void setSafe(int index, ${minor.class}Holder holder){
+      setSafe(index, holder.days, holder.milliseconds);
+    }
 
-   <#else>
+    public void setSafe(int index, Nullable${minor.class}Holder holder){
+      setSafe(index, holder.days, holder.milliseconds);
+    }
 
-   protected void set(int index, ${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
+  <#elseif (minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")>
+    public void set(int index, ${minor.class}Holder holder){
+      set(index, holder.start, holder.buffer);
+    }
 
-   public void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
+    void set(int index, Nullable${minor.class}Holder holder){
+      set(index, holder.start, holder.buffer);
+    }
 
-   public void set(int index, int start, DrillBuf buffer){
-     data.setBytes(index * ${type.width}, buffer, start, ${type.width});
-   }
+    public void setSafe(int index,  Nullable${minor.class}Holder holder){
+      setSafe(index, holder.start, holder.buffer);
+    }
 
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
+    public void setSafe(int index,  ${minor.class}Holder holder){
+      setSafe(index, holder.start, holder.buffer);
+    }
 
-   public void setSafe(int index, int start, DrillBuf buffer){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, holder);
-   }
-
-   public void set(int index, Nullable${minor.class}Holder holder){
-     data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
-   }
+    public void setSafe(int index, int start, DrillBuf buffer){
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      set(index, start, buffer);
+    }
+
+  <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
+    public void set(int index, BigDecimal value) {
+      DecimalUtility.getSparseFromBigDecimal(value, data, index * ${type.width},
+           field.getScale(), field.getPrecision(), ${minor.nDecimalDigits});
+    }
+
+    public void setSafe(int index, BigDecimal value) {
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      set(index, value);
+    }
+
+  </#if>
+    public void set(int index, int start, DrillBuf buffer){
+      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
+    }
+
+  <#else>
+    protected void set(int index, ${minor.class}Holder holder){
+      set(index, holder.start, holder.buffer);
+    }
+
+    public void set(int index, Nullable${minor.class}Holder holder){
+      set(index, holder.start, holder.buffer);
+    }
+
+    public void set(int index, int start, DrillBuf buffer){
+      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
+    }
+
+    public void setSafe(int index, ${minor.class}Holder holder){
+      setSafe(index, holder.start, holder.buffer);
+    }
+
+    public void setSafe(int index, Nullable${minor.class}Holder holder){
+      setSafe(index, holder.start, holder.buffer);
+    }
+
+    public void setSafe(int index, int start, DrillBuf buffer){
+      while(index >= getValueCapacity()) {
+        reAlloc();
+      }
+      set(index, holder);
+    }
+
+    public void set(int index, Nullable${minor.class}Holder holder){
+      data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
+    }
    </#if>
 
-   @Override
-   public void generateTestData(int count) {
-     setValueCount(count);
-     boolean even = true;
-     final int valueCount = getAccessor().getValueCount();
-     for(int i = 0; i < valueCount; i++, even = !even) {
-       final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
-       for(int w = 0; w < ${type.width}; w++){
-         data.setByte(i + w, b);
-       }
-     }
-   }
+    @Override
+    public void generateTestData(int count) {
+      setValueCount(count);
+      boolean even = true;
+      final int valueCount = getAccessor().getValueCount();
+      for(int i = 0; i < valueCount; i++, even = !even) {
+        final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
+        for(int w = 0; w < ${type.width}; w++){
+          data.setByte(i + w, b);
+        }
+      }
+    }
 
    <#else> <#-- type.width <= 8 -->
     public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
@@ -792,7 +806,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
   </#if> <#-- type.width -->
-
     @Override
     public void setValueCount(int valueCount) {
       final int currentValueCapacity = getValueCapacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index fcbe79a..170c606 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
 
@@ -252,7 +253,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
   </#if>
 
-
   @Override
   public void load(SerializedField metadata, DrillBuf buffer) {
     clear();
@@ -486,7 +486,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     }
 
     <#if type.major == "VarLen">
-
     private void fillEmpties(int index){
       final ${valuesName}.Mutator valuesMutator = values.getMutator();
       for (int i = lastSet; i < index; i++) {
@@ -547,7 +546,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       values.getMutator().set(index, holder);
     }
 
-
     public void set(int index, Nullable${minor.class}Holder holder){
       final ${valuesName}.Mutator valuesMutator = values.getMutator();
       <#if type.major == "VarLen">
@@ -602,9 +600,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
-
     public void setSafe(int index, Nullable${minor.class}Holder value) {
-
       <#if type.major == "VarLen">
       if (index > lastSet + 1) {
         fillEmpties(index);
@@ -617,7 +613,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     }
 
     public void setSafe(int index, ${minor.class}Holder value) {
-
       <#if type.major == "VarLen">
       if (index > lastSet + 1) {
         fillEmpties(index);
@@ -630,19 +625,32 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     }
 
     <#if !(type.major == "VarLen" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense" || minor.class == "Interval" || minor.class == "IntervalDay")>
-      public void setSafe(int index, ${minor.javaType!type.javaType} value) {
-        <#if type.major == "VarLen">
-        if (index > lastSet + 1) {
-          fillEmpties(index);
-        }
-        </#if>
-        bits.getMutator().setSafe(index, 1);
-        values.getMutator().setSafe(index, value);
-        setCount++;
+    public void setSafe(int index, ${minor.javaType!type.javaType} value) {
+      <#if type.major == "VarLen">
+      if (index > lastSet + 1) {
+        fillEmpties(index);
       }
+      </#if>
+      bits.getMutator().setSafe(index, 1);
+      values.getMutator().setSafe(index, value);
+      setCount++;
+    }
 
     </#if>
+    <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
+    public void set(int index, BigDecimal value) {
+      bits.getMutator().set(index, 1);
+      values.getMutator().set(index, value);
+      setCount++;
+    }
 
+    public void setSafe(int index, BigDecimal value) {
+      bits.getMutator().setSafe(index, 1);
+      values.getMutator().setSafe(index, value);
+      setCount++;
+    }
+
+    </#if>
     @Override
     public void setValueCount(int valueCount) {
       assert valueCount >= 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
index 2a758eb..159a8e7 100644
--- a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
@@ -160,23 +160,23 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
     }
   }
 
-    public void copyFrom(int inIndex, int outIndex, Repeated${minor.class}Vector v) {
-      final Accessor vAccessor = v.getAccessor();
-      final int count = vAccessor.getInnerValueCountAt(inIndex);
-      mutator.startNewValue(outIndex);
-      for (int i = 0; i < count; i++) {
-        mutator.add(outIndex, vAccessor.get(inIndex, i));
-      }
+  public void copyFrom(int inIndex, int outIndex, Repeated${minor.class}Vector v) {
+    final Accessor vAccessor = v.getAccessor();
+    final int count = vAccessor.getInnerValueCountAt(inIndex);
+    mutator.startNewValue(outIndex);
+    for (int i = 0; i < count; i++) {
+      mutator.add(outIndex, vAccessor.get(inIndex, i));
     }
+  }
 
-    public void copyFromSafe(int inIndex, int outIndex, Repeated${minor.class}Vector v) {
-      final Accessor vAccessor = v.getAccessor();
-      final int count = vAccessor.getInnerValueCountAt(inIndex);
-      mutator.startNewValue(outIndex);
-      for (int i = 0; i < count; i++) {
-        mutator.addSafe(outIndex, vAccessor.get(inIndex, i));
-      }
+  public void copyFromSafe(int inIndex, int outIndex, Repeated${minor.class}Vector v) {
+    final Accessor vAccessor = v.getAccessor();
+    final int count = vAccessor.getInnerValueCountAt(inIndex);
+    mutator.startNewValue(outIndex);
+    for (int i = 0; i < count; i++) {
+      mutator.addSafe(outIndex, vAccessor.get(inIndex, i));
     }
+  }
 
   public boolean allocateNewSafe() {
     /* boolean to keep track if all the memory allocation were successful
@@ -236,7 +236,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
   }
 
   <#else>
-
   @Override
   public void allocateNew(int valueCount, int innerValueCount) {
     clear();
@@ -258,7 +257,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
   }
 
   </#if>
-
   // This is declared a subclass of the accessor declared inside of FixedWidthVector, this is also used for
   // variable length vectors, as they should ahve consistent interface as much as possible, if they need to diverge
   // in the future, the interface shold be declared in the respective value vector superclasses for fixed and variable
@@ -348,7 +346,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
     }
 
     <#else>
-
     public void addSafe(int index, ${minor.javaType!type.javaType} srcValue) {
       final int nextOffset = offsets.getAccessor().get(index+1);
       values.getMutator().setSafe(nextOffset, srcValue);
@@ -356,7 +353,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
     }
 
     </#if>
-
     public void setSafe(int index, Repeated${minor.class}Holder h) {
       final ${minor.class}Holder ih = new ${minor.class}Holder();
       final ${minor.class}Vector.Accessor hVectorAccessor = h.vector.getAccessor();
@@ -385,8 +381,16 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       values.getMutator().setSafe(nextOffset, <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
       offsets.getMutator().setSafe(arrayIndex+1, nextOffset+1);
     }
+
     </#if>
+    <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
+    public void addSafe(int index, BigDecimal value) {
+      int nextOffset = offsets.getAccessor().get(index+1);
+      values.getMutator().setSafe(nextOffset, value);
+      offsets.getMutator().setSafe(index+1, nextOffset+1);
+    }
 
+    </#if>
     protected void add(int index, ${minor.class}Holder holder) {
       int nextOffset = offsets.getAccessor().get(index+1);
       values.getMutator().set(nextOffset, holder);

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index e970271..105ea47 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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

http://git-wip-us.apache.org/repos/asf/drill/blob/381eab66/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 92019ec..e192107 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -29,6 +29,11 @@ import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.proto.UserBitShared.NamePart;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 
+/**
+ * Meta-data description of a column characterized by a name and a type
+ * (including both data type and cardinality AKA mode). For map types,
+ * the description includes the nested columns.)
+ */
 
 public class MaterializedField {
   private final String name;
@@ -44,7 +49,7 @@ public class MaterializedField {
 
   public static MaterializedField create(SerializedField serField){
     LinkedHashSet<MaterializedField> children = new LinkedHashSet<>();
-    for (SerializedField sf:serField.getChildList()) {
+    for (SerializedField sf : serField.getChildList()) {
       children.add(MaterializedField.create(sf));
     }
     return new MaterializedField(serField.getNamePart().getName(), serField.getMajorType(), children);
@@ -61,7 +66,6 @@ public class MaterializedField {
     return serializedFieldBuilder.build();
   }
 
-
   public SerializedField.Builder getAsBuilder(){
     return SerializedField.newBuilder()
         .setMajorType(type)
@@ -110,7 +114,6 @@ public class MaterializedField {
 //    return seg.getNameSegment().getPath();
 //  }
 
-
   // TODO: rewrite without as direct match rather than conversion then match.
   public boolean matches(SerializedField field){
     MaterializedField f = create(field);
@@ -142,41 +145,17 @@ public class MaterializedField {
 //    return sb.toString();
 //  }
 
-  public String getPath() {
-    return getName();
-  }
-
-  public String getLastName() {
-    return getName();
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public int getWidth() {
-    return type.getWidth();
-  }
-
-  public MajorType getType() {
-    return type;
-  }
-
-  public int getScale() {
-      return type.getScale();
-  }
-  public int getPrecision() {
-      return type.getPrecision();
-  }
-  public boolean isNullable() {
-    return type.getMode() == DataMode.OPTIONAL;
-  }
-
-  public DataMode getDataMode() {
-    return type.getMode();
-  }
-
-  public MaterializedField getOtherNullableVersion(){
+  public String getPath() { return getName(); }
+  public String getLastName() { return getName(); }
+  public String getName() { return name; }
+  public int getWidth() { return type.getWidth(); }
+  public MajorType getType() { return type; }
+  public int getScale() { return type.getScale(); }
+  public int getPrecision() { return type.getPrecision(); }
+  public boolean isNullable() { return type.getMode() == DataMode.OPTIONAL; }
+  public DataMode getDataMode() { return type.getMode(); }
+
+  public MaterializedField getOtherNullableVersion() {
     MajorType mt = type;
     DataMode newDataMode = null;
     switch (mt.getMode()){
@@ -220,7 +199,6 @@ public class MaterializedField {
             Objects.equals(this.type, other.type);
   }
 
-
   @Override
   public String toString() {
     final int maxLen = 10;
@@ -228,7 +206,6 @@ public class MaterializedField {
     return name + "(" + type.getMinorType().name() + ":" + type.getMode().name() + ")" + childStr;
   }
 
-
   private String toString(Collection<?> collection, int maxLen) {
     StringBuilder builder = new StringBuilder();
     builder.append("[");