You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2017/12/21 05:19:28 UTC

[01/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Repository: drill
Updated Branches:
  refs/heads/master eb0c40306 -> 40de8ca4f


http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/package-info.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/package-info.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/package-info.java
new file mode 100644
index 0000000..9bc654b
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/package-info.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package provides a "dummy" set of writers. The dummy writers provide
+ * the same API as the "real" writers, but the dummy writers simply discard
+ * their data. The dummy writers are used when implementing projection:
+ * non-projected columns may still have to be processed (as in a CSV file,
+ * say), but their values are not needed. One way to do this is to do an
+ * if-statement for each value:<pre><code>
+ * if (column-a-is-projected) {
+ *   aWriter.setSomething(value);
+ * }</code></pre>
+ * The dummy writers convert the if-statement into a virtual function call,
+ * same as is done to handle the type-specific nature of vectors:
+ * <pre><code>
+ * aWriter.setSomething(value);
+ * </code></pre>
+ * <p>
+ * The theory is that the virtual function dispatch is simpler, and faster,
+ * than doing continual if-checks everywhere in the code.
+ * <p>
+ * The dummy writers reside in this package so that the various factory
+ * methods can automatically build the dummy versions when given a null
+ * value vector (which we then interpret to mean that there is no physical
+ * backing to the column.)
+ * <p>
+ * At present, most methods that return a value simply return zero or
+ * null.
+ * Experience will show whether it is worthwhile implementing some
+ * basics, such as a value type or index. For now, these return null,
+ * assuming that the caller won't do anything with the column other
+ * than set a value.
+ * <p>
+ * Some simpler dummy writers appear as nested classes inside the
+ * "real" writers.
+ */
+
+package org.apache.drill.exec.vector.accessor.writer.dummy;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/package-info.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/package-info.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/package-info.java
new file mode 100644
index 0000000..f536c09
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/package-info.java
@@ -0,0 +1,151 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation of the vector writers. The code will make much more sense if
+ * we start with a review of Drill’s complex vector data model. Drill has 38+
+ * data (“minor”) types. Drill also has three cardinalities (“modes”). The
+ * result is over 120+ different vector types. Then, when you add maps, repeated
+ * maps, lists and repeated lists, you rapidly get an explosion of types that
+ * the writer code must handle.
+ *
+ * <h4>Understanding the Vector Model</h4>
+ *
+ * Vectors can be categorized along multiple dimensions:
+ * <ul>
+ * <li>By data (minor) type</li>
+ * <li>By cardinality (mode)</li>
+ * <li>By fixed or variable width</li>
+ * <li>By repeat levels</li>
+ * </ul>
+ * <p>
+ * A repeated map, a list, a repeated list and any array (repeated) scalar all
+ * are array-like. Nullable and required modes are identical (single values),
+ * but a nullable has an additional is-set (“bit”) vector.
+ * <p>
+ * The writers (and readers) borrow concepts from JSON and relational theory
+ * to simplify the problem:
+ * <p>
+ * <ul>
+ * <li>Both the top-level row, and a Drill map are “tuples” and are treated
+ * similarly in the model.</li>
+ * <li>All non-map, non-list (that is, scalar) data types are treated
+ * uniformly.</li>
+ * <li>All arrays (whether a list, a repeated list, a repeated map, or a
+ * repeated scalar) are treated uniformly.</li>
+ * </ul>
+ *
+ * <h4>Repeat Levels</h4>
+ *
+ * JSON and Parquet can be understood as a series of one or more "repeat
+ * levels." First, let's identify the repeat levels above the batch
+ * level:
+ * <ul>
+ * <li>The top-most level is the "result set": the entire collection of
+ * rows that come from a file (or other data source.)</li>
+ * <li>Result sets are divided into batches: collections of up to 64K
+ * rows.</li>
+ * </ul>
+ *
+ * Then, within a batch:
+ * <ul>
+ * <li>Each batch is a collection or rows. A batch-level index points
+ * to the current row.</li>
+ * </ul>Scalar arrays introduce a repeat level: each row has 0, 1 or
+ * many elements in the array-valued column. An offset vector indexes
+ * to the first value for each row. Each scalar array has its own
+ * per-array index to point to the next write position.</li>
+ * <li>Map arrays introduce a repeat level for a group of columns
+ * (those that make up the map.) A single offset vector points to
+ * the common start position for the columns. A common index points
+ * to the common next write position.<li>
+ * <li>Lists also introduce a repeat level. (Details to be worked
+ * out.</li>
+ * </ul>
+ *
+ * For repeated vectors, one can think of the structure either top-down
+ * or bottom-up:
+ * <ul>
+ * <li>Top down: the row position points into an offset vector. The
+ * offset vector value points to either the data value, or into another
+ * offset vector.</li>
+ * <li>Bottom-up: values are appended to the end of the vector. Values
+ * are "pinched off" to form an array (for repeated maps) or for a row.
+ * In this view, indexes bubble upward. The inner-most last write position
+ * is written as the array end position in the enclosing offset vector.
+ * This may occur up several levels.</li>
+ * </ul>
+ *
+ * <h4>Writer Data Model</h4>
+ *
+ * The above leads to a very simple, JSON-like data model:
+ * <ul>
+ * <li>A tuple reader or writer models a row. (Usually via a subclass.) Column
+ * are accessible by name or position.</li>
+ * <li>Every column is modeled as an object.</li>
+ * <li>The object can have an object type: scalar, tuple or array.</li>
+ * <li>An array has a single element type (but many run-time elements)</li>
+ * <li>A scalar can be nullable or not, and provides a uniform get/set
+ * interface.</li>
+ * </ul>
+ * <p>
+ * This data model is similar to; but has important differences from, the prior,
+ * generated, readers and writers.
+ * <p>
+ * The object layer is new: it is the simplest way to model the three “object
+ * types.” An app using this code would use just the leaf scalar readers and
+ * writers.
+ *
+ * <h4>Writer Performance</h4>
+ *
+ * To maximize performance, have a single version for all "data modes":
+ * (nullable, required, repeated). Some items of note:
+ * <ul>
+ * <li>The writers bypass DrillBuf and the UDLE to needed writes to direct
+ * memory.</li>
+ * <li>The writers buffer the buffer address and implement a number of methods
+ * to synchronize that address when the buffer changes (on a new batch or during
+ * vector resize).</li>
+ * <li>Writing require a single bounds check. In most cases, the write is within
+ * bounds so the single check is all that is needed.</li>
+ * <li>If the write is out of bounds, then the writer determines the new vector
+ * size and performs the needed reallocation. To avoid multiple doublings, the
+ * writer computes the needed new size and allocates that size directly.</li>
+ * <li>Vector reallocation is improved to eliminate zeroing the new half of the
+ * buffer, data is left “garbage-filled.”</li>
+ * <li>If the vector would grow beyond 16 MB, then overflow is triggered, via a
+ * listener, which causes the buffer to be replaced. The write then
+ * continues.</li>
+ * <li>Offset vector updates are integrated into the writers using an
+ * `OffsetVectorWriter`. This writer caches the last write position so that each
+ * array write needs a single offset update, rather than the read and write as
+ * in previous code.</li>
+ * <li>The writers keep track of the “last write position” and perform
+ * “fill-empties” work if the new write position is more than one position
+ * behind the last write. All types now correctly support “fill-empties”
+ * (before, only nullable types did so reliably.)</li>
+ * <li>Null handling is done by an additional writer layer that wraps the
+ * underlying data writer. This avoids the need for a special nullable writer:
+ * the same nullable layer works for all data types.</li>
+ * <li>Array handling is done similarly: an array writer manages the offset
+ * vector and works the same for repeated scalars, repeated maps and
+ * (eventually) lists and repeated lists.</li>
+ * </ul>
+ */
+
+package org.apache.drill.exec.vector.accessor.writer;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
index 6b60471..5ac28c5 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
@@ -64,7 +64,6 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
       valueVector.close();
     }
     vectors.clear();
-
     super.close();
   }
 
@@ -178,7 +177,7 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
    *
    * Note that this method does not enforce any vector type check nor throws a schema change exception.
    */
-  protected void putChild(String name, ValueVector vector) {
+  public void putChild(String name, ValueVector vector) {
     putVector(name, vector);
     field.addChild(vector.getField());
   }
@@ -280,6 +279,16 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
   }
 
   @Override
+  public int getAllocatedSize() {
+    int size = 0;
+
+    for (final ValueVector v : vectors.values()) {
+      size += v.getAllocatedSize();
+    }
+    return size;
+  }
+
+  @Override
   public void collectLedgers(Set<BufferLedger> ledgers) {
     for (final ValueVector v : vectors.values()) {
       v.collectLedgers(ledgers);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 2b41b8b..8472f80 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
@@ -87,14 +87,10 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
 
 
   @Override
-  public UInt4Vector getOffsetVector() {
-    return offsets;
-  }
+  public UInt4Vector getOffsetVector() { return offsets; }
 
   @Override
-  public ValueVector getDataVector() {
-    return vector;
-  }
+  public ValueVector getDataVector() { return vector; }
 
   @Override
   public void setInitialCapacity(int numRecords) {
@@ -127,6 +123,11 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
   }
 
   @Override
+  public int getAllocatedSize() {
+    return offsets.getAllocatedSize() + vector.getAllocatedSize();
+  }
+
+  @Override
   public int getBufferSizeFor(int valueCount) {
     if (valueCount == 0) {
       return 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
index 9569946..7de5ce6 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/ListVector.java
@@ -52,7 +52,6 @@ public class ListVector extends BaseRepeatedValueVector {
   private Accessor accessor = new Accessor();
   private UnionListWriter writer;
   private UnionListReader reader;
-  private CallBack callBack;
 
   public ListVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
     super(field, allocator);
@@ -61,7 +60,6 @@ public class ListVector extends BaseRepeatedValueVector {
     this.field.addChild(getDataVector().getField());
     this.writer = new UnionListWriter(this);
     this.reader = new UnionListReader(this);
-    this.callBack = callBack;
   }
 
   public UnionListWriter getWriter() {
@@ -203,6 +201,8 @@ public class ListVector extends BaseRepeatedValueVector {
             .addChild(bits.getMetadata())
             .addChild(vector.getMetadata());
   }
+
+  @Override
   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(VectorDescriptor descriptor) {
     AddOrGetResult<T> result = super.addOrGetVector(descriptor);
     reader = new UnionListReader(this);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
index 19c910b..4a501b8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
@@ -67,10 +67,7 @@ public class MapVector extends AbstractMapVector {
   }
 
   @Override
-  public FieldReader getReader() {
-    //return new SingleMapReaderImpl(MapVector.this);
-    return reader;
-  }
+  public FieldReader getReader() { return reader; }
 
   transient private MapTransferPair ephPair;
   transient private MapSingleCopier ephPair2;
@@ -95,9 +92,7 @@ public class MapVector extends AbstractMapVector {
   }
 
   @Override
-  protected boolean supportsDirectRead() {
-    return true;
-  }
+  protected boolean supportsDirectRead() { return true; }
 
   public Iterator<String> fieldNameIterator() {
     return getChildFieldNames().iterator();
@@ -124,6 +119,15 @@ public class MapVector extends AbstractMapVector {
   }
 
   @Override
+  public int getAllocatedSize() {
+    int size = 0;
+    for (final ValueVector v : this) {
+      size += v.getAllocatedSize();
+    }
+    return size;
+  }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     if (valueCount == 0) {
       return 0;
@@ -353,6 +357,10 @@ public class MapVector extends AbstractMapVector {
     return getChildByOrdinal(id);
   }
 
+  public void setMapValueCount(int valueCount) {
+    this.valueCount = valueCount;
+  }
+
   public class Mutator extends BaseValueVector.BaseMutator {
 
     @Override
@@ -360,7 +368,7 @@ public class MapVector extends AbstractMapVector {
       for (final ValueVector v : getChildren()) {
         v.getMutator().setValueCount(valueCount);
       }
-      MapVector.this.valueCount = valueCount;
+      setMapValueCount(valueCount);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
index be9ebee..6442417 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
@@ -198,14 +198,10 @@ public class RepeatedListVector extends AbstractContainerVector
     }
 
     @Override
-    public RepeatedListAccessor getAccessor() {
-      return accessor;
-    }
+    public RepeatedListAccessor getAccessor() { return accessor; }
 
     @Override
-    public RepeatedListMutator getMutator() {
-      return mutator;
-    }
+    public RepeatedListMutator getMutator() { return mutator; }
 
     @Override
     public FieldReader getReader() {
@@ -277,11 +273,8 @@ public class RepeatedListVector extends AbstractContainerVector
     }
   }
 
-
-    @Override
-  public RepeatedListReaderImpl getReader() {
-    return reader;
-  }
+  @Override
+  public RepeatedListReaderImpl getReader() { return reader; }
 
   @Override
   public DelegateRepeatedVector.RepeatedListAccessor getAccessor() {
@@ -334,6 +327,11 @@ public class RepeatedListVector extends AbstractContainerVector
   }
 
   @Override
+  public int getAllocatedSize() {
+    return delegate.getAllocatedSize();
+  }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     return delegate.getBufferSizeFor(valueCount);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 6b29258..57f1a67 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
@@ -63,18 +63,22 @@ public class RepeatedMapVector extends AbstractMapVector
   private final Mutator mutator = new Mutator();
   private final EmptyValuePopulator emptyPopulator;
 
-  public RepeatedMapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack){
+  public RepeatedMapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
     super(field, allocator, callBack);
     this.offsets = new UInt4Vector(BaseRepeatedValueVector.OFFSETS_FIELD, allocator);
     this.emptyPopulator = new EmptyValuePopulator(offsets);
   }
 
-  @Override
-  public UInt4Vector getOffsetVector() {
-    return offsets;
+  public RepeatedMapVector(MaterializedField field, UInt4Vector offsets, CallBack callBack) {
+    super(field, offsets.getAllocator(), callBack);
+    this.offsets = offsets;
+    this.emptyPopulator = new EmptyValuePopulator(offsets);
   }
 
   @Override
+  public UInt4Vector getOffsetVector() { return offsets; }
+
+  @Override
   public ValueVector getDataVector() {
     throw new UnsupportedOperationException();
   }
@@ -93,9 +97,7 @@ public class RepeatedMapVector extends AbstractMapVector
   }
 
   @Override
-  public RepeatedMapReaderImpl getReader() {
-    return reader;
-  }
+  public RepeatedMapReaderImpl getReader() { return reader; }
 
   @Override
   public void allocateNew(int groupCount, int innerValueCount) {
@@ -137,6 +139,11 @@ public class RepeatedMapVector extends AbstractMapVector
   }
 
   @Override
+  public int getAllocatedSize() {
+    return offsets.getAllocatedSize() + super.getAllocatedSize();
+  }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     if (valueCount == 0) {
       return 0;


[05/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 4da3d9e..c20ee89 100644
--- a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
@@ -18,6 +18,9 @@
 
 import java.lang.Override;
 
+import org.apache.drill.common.types.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
 import org.mortbay.jetty.servlet.Holder;
@@ -55,7 +58,10 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
 
   public Repeated${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    addOrGetVector(VectorDescriptor.create(Types.required(field.getType().getMinorType())));
+    MajorType majorType = field.getType();
+    addOrGetVector(VectorDescriptor.create(Types.withScaleAndPrecision(
+        majorType.getMinorType(), DataMode.REQUIRED,
+        majorType.getScale(), majorType.getPrecision())));
   }
 
   @Override
@@ -341,25 +347,12 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       addSafe(index, bytes, 0, bytes.length);
     }
 
-    public void addEntry(int index, byte[] bytes) throws VectorOverflowException {
-      addEntry(index, bytes, 0, bytes.length);
-    }
-
     public void addSafe(int index, byte[] bytes, int start, int length) {
       final int nextOffset = offsets.getAccessor().get(index+1);
       values.getMutator().setSafe(nextOffset, bytes, start, length);
       offsets.getMutator().setSafe(index+1, nextOffset+1);
     }
 
-    public void addEntry(int index, byte[] bytes, int start, int length) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(index+1);
-      values.getMutator().setArrayItem(nextOffset, bytes, start, length);
-      offsets.getMutator().setSafe(index+1, nextOffset+1);
-    }
-
     <#else>
     public void addSafe(int index, ${minor.javaType!type.javaType} srcValue) {
       final int nextOffset = offsets.getAccessor().get(index+1);
@@ -367,15 +360,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       offsets.getMutator().setSafe(index+1, nextOffset+1);
     }
 
-    public void addEntry(int index, ${minor.javaType!type.javaType} srcValue) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(index+1);
-      values.getMutator().setArrayItem(nextOffset, srcValue);
-      offsets.getMutator().setSafe(index+1, nextOffset+1);
-    }
-
     </#if>
     public void setSafe(int index, Repeated${minor.class}Holder h) {
       final ${minor.class}Holder ih = new ${minor.class}Holder();
@@ -393,14 +377,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       offsets.getMutator().setSafe(index+1, nextOffset+1);
     }
 
-    public void addEntry(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(index+1);
-      values.getMutator().setArrayItem(nextOffset, holder);
-      offsets.getMutator().setSafe(index+1, nextOffset+1);
-    }
 
     public void addSafe(int index, Nullable${minor.class}Holder holder) {
       final int nextOffset = offsets.getAccessor().get(index+1);
@@ -408,15 +384,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       offsets.getMutator().setSafe(index+1, nextOffset+1);
     }
 
-    public void addEntry(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(index+1);
-      values.getMutator().setArrayItem(nextOffset, holder);
-      offsets.getMutator().setSafe(index+1, nextOffset+1);
-    }
-
     /**
      * Backfill missing offsets from the given last written position to the
      * given current write position. Used by the "new" size-safe column
@@ -427,11 +394,7 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
      * @param index the current write position to be initialized
      */
 
-    public void fillEmptiesBounded(int lastWrite, int index)
-            throws VectorOverflowException {
-      if (index >= UInt4Vector.MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
+    public void fillEmpties(int lastWrite, int index) {
       // If last write was 2, offsets are [0, 3, 6]
       // If next write is 4, offsets must be: [0, 3, 6, 6, 6]
       // Remember the offsets are one more than row count.
@@ -449,15 +412,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       offsets.getMutator().setSafe(rowIndex+1, nextOffset+1);
     }
 
-    public void addEntry(int rowIndex, <#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) throws VectorOverflowException {
-      if (rowIndex >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(rowIndex+1);
-      values.getMutator().setArrayItem(nextOffset, <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      offsets.getMutator().setSafe(rowIndex+1, nextOffset+1);
-    }
-
     </#if>
     <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
     public void addSafe(int index, BigDecimal value) {
@@ -466,15 +420,6 @@ public final class Repeated${minor.class}Vector extends BaseRepeatedValueVector
       offsets.getMutator().setSafe(index+1, nextOffset+1);
     }
 
-    public void addEntry(int index, BigDecimal value) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      final int nextOffset = offsets.getAccessor().get(index+1);
-      values.getMutator().setArrayItem(nextOffset, value);
-      offsets.getMutator().setSafe(index+1, nextOffset+1);
-    }
-
     </#if>
     protected void add(int index, ${minor.class}Holder holder) {
       final int nextOffset = offsets.getAccessor().get(index+1);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/codegen/templates/UnionVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionVector.java b/exec/vector/src/main/codegen/templates/UnionVector.java
index c198544..a46779d 100644
--- a/exec/vector/src/main/codegen/templates/UnionVector.java
+++ b/exec/vector/src/main/codegen/templates/UnionVector.java
@@ -116,7 +116,6 @@ public class UnionVector implements ValueVector {
     }
     return mapVector;
   }
-
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign fields = minor.fields!type.fields />
   <#assign uncappedName = name?uncap_first/>
@@ -136,9 +135,7 @@ public class UnionVector implements ValueVector {
     }
     return ${uncappedName}Vector;
   }
-
   </#if>
-
   </#list></#list>
 
   private static final MajorType LIST_TYPE = Types.optional(MinorType.LIST);
@@ -183,8 +180,7 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
-  public void setInitialCapacity(int numRecords) {
-  }
+  public void setInitialCapacity(int numRecords) { }
 
   @Override
   public int getValueCapacity() {
@@ -192,8 +188,7 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
-  public void close() {
-  }
+  public void close() { }
 
   @Override
   public void clear() {
@@ -201,9 +196,7 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
-  public MaterializedField getField() {
-    return field;
-  }
+  public MaterializedField getField() { return field; }
 
   @Override
   public void collectLedgers(Set<BufferLedger> ledgers) {
@@ -290,9 +283,7 @@ public class UnionVector implements ValueVector {
     }
 
     @Override
-    public void splitAndTransfer(int startIndex, int length) {
-
-    }
+    public void splitAndTransfer(int startIndex, int length) { }
 
     @Override
     public ValueVector getTo() {
@@ -306,14 +297,10 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
-  public Accessor getAccessor() {
-    return accessor;
-  }
+  public Accessor getAccessor() { return accessor; }
 
   @Override
-  public Mutator getMutator() {
-    return mutator;
-  }
+  public Mutator getMutator() { return mutator; }
 
   @Override
   public FieldReader getReader() {
@@ -347,6 +334,11 @@ public class UnionVector implements ValueVector {
   }
 
   @Override
+  public int getAllocatedSize() {
+    return internalMap.getAllocatedSize();
+  }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     if (valueCount == 0) {
       return 0;
@@ -381,7 +373,6 @@ public class UnionVector implements ValueVector {
 
   public class Accessor extends BaseValueVector.BaseAccessor {
 
-
     @Override
     public Object getObject(int index) {
       int type = typeVector.getAccessor().get(index);
@@ -406,12 +397,9 @@ public class UnionVector implements ValueVector {
       }
     }
 
-    public byte[] get(int index) {
-      return null;
-    }
+    public byte[] get(int index) { return null; }
 
-    public void get(int index, ComplexHolder holder) {
-    }
+    public void get(int index, ComplexHolder holder) { }
 
     public void get(int index, UnionHolder holder) {
       FieldReader reader = new UnionReader(UnionVector.this);
@@ -420,9 +408,7 @@ public class UnionVector implements ValueVector {
     }
 
     @Override
-    public int getValueCount() {
-      return valueCount;
-    }
+    public int getValueCount() { return valueCount; }
 
     @Override
     public boolean isNull(int index) {
@@ -436,7 +422,7 @@ public class UnionVector implements ValueVector {
 
   public class Mutator extends BaseValueVector.BaseMutator {
 
-    UnionWriter writer;
+    protected UnionWriter writer;
 
     @Override
     public void setValueCount(int valueCount) {

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 e5432da..a29194a 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -54,7 +54,6 @@ package org.apache.drill.exec.vector;
  */
 
 public final class ${minor.class}Vector extends BaseDataValueVector implements VariableWidthVector {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
 
   private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
   private static final int INITIAL_BYTE_COUNT = Math.min(INITIAL_VALUE_ALLOCATION * DEFAULT_RECORD_BYTE_COUNT, MAX_BUFFER_SIZE);
@@ -68,20 +67,17 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   private final Accessor accessor;
   private final Mutator mutator;
 
-  private final UInt${type.width}Vector.Accessor oAccessor;
-
   private int allocationSizeInBytes = INITIAL_BYTE_COUNT;
   private int allocationMonitor = 0;
 
   public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
-    this.oAccessor = offsetVector.getAccessor();
     this.accessor = new Accessor();
     this.mutator = new Mutator();
   }
 
   @Override
-  public FieldReader getReader(){
+  public FieldReader getReader() {
     return reader;
   }
 
@@ -94,6 +90,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   @Override
+  public int getAllocatedSize() {
+    return offsetVector.getAllocatedSize() + data.capacity();
+  }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     if (valueCount == 0) {
       return 0;
@@ -104,12 +105,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   @Override
-  public int getValueCapacity(){
+  public int getValueCapacity() {
     return Math.max(offsetVector.getValueCapacity() - 1, 0);
   }
 
   @Override
-  public int getByteCapacity(){
+  public int getByteCapacity() {
     return data.capacity();
   }
 
@@ -124,7 +125,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
    */
   public int getVarByteLength(){
     final int valueCount = getAccessor().getValueCount();
-    if(valueCount == 0) {
+    if (valueCount == 0) {
       return 0;
     }
     return offsetVector.getAccessor().get(valueCount);
@@ -132,10 +133,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
   @Override
   public SerializedField getMetadata() {
-    return getMetadataBuilder() //
+    return getMetadataBuilder()
              .addChild(offsetVector.getMetadata())
-             .setValueCount(getAccessor().getValueCount()) //
-             .setBufferLength(getBufferSize()) //
+             .setValueCount(getAccessor().getValueCount())
+             .setBufferLength(getBufferSize())
              .build();
   }
 
@@ -170,21 +171,21 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return buffers;
   }
 
-  public long getOffsetAddr(){
+  public long getOffsetAddr() {
     return offsetVector.getBuffer().memoryAddress();
   }
 
-  public UInt${type.width}Vector getOffsetVector(){
+  public UInt${type.width}Vector getOffsetVector() {
     return offsetVector;
   }
 
   @Override
-  public TransferPair getTransferPair(BufferAllocator allocator){
+  public TransferPair getTransferPair(BufferAllocator allocator) {
     return new TransferImpl(getField(), allocator);
   }
 
   @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
     return new TransferImpl(getField().withPath(ref), allocator);
   }
 
@@ -193,7 +194,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return new TransferImpl((${minor.class}Vector) to);
   }
 
-  public void transferTo(${minor.class}Vector target){
+  public void transferTo(${minor.class}Vector target) {
     target.clear();
     this.offsetVector.transferTo(target.offsetVector);
     target.data = data.transferOwnership(target.allocator).buffer;
@@ -272,23 +273,23 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
   }
 
   private class TransferImpl implements TransferPair{
-    ${minor.class}Vector to;
+    private final ${minor.class}Vector to;
 
     public TransferImpl(MaterializedField field, BufferAllocator allocator){
       to = new ${minor.class}Vector(field, allocator);
     }
 
-    public TransferImpl(${minor.class}Vector to){
+    public TransferImpl(${minor.class}Vector to) {
       this.to = to;
     }
 
     @Override
-    public ${minor.class}Vector getTo(){
+    public ${minor.class}Vector getTo() {
       return to;
     }
 
     @Override
-    public void transfer(){
+    public void transfer() {
       transferTo(to);
     }
 
@@ -309,7 +310,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     if (size > MAX_ALLOCATION_SIZE) {
       throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
     }
-    allocationSizeInBytes = (int) size;
+    allocationSizeInBytes = (int)size;
     offsetVector.setInitialCapacity(valueCount + 1);
   }
 
@@ -386,12 +387,17 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       throw new OversizedAllocationException("Unable to expand the buffer. Max allowed buffer size is reached.");
     }
 
-    logger.trace("Reallocating VarChar, new size {}", newAllocationSize);
-    final DrillBuf newBuf = allocator.buffer((int)newAllocationSize);
+    reallocRaw((int) newAllocationSize);
+  }
+
+  @Override
+  public DrillBuf reallocRaw(int newAllocationSize) {
+    final DrillBuf newBuf = allocator.buffer(newAllocationSize);
     newBuf.setBytes(0, data, 0, data.capacity());
     data.release();
     data = newBuf;
-    allocationSizeInBytes = (int)newAllocationSize;
+    allocationSizeInBytes = newAllocationSize;
+    return data;
   }
 
   public void decrementAllocationMonitor() {
@@ -430,6 +436,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
 
   public final class Accessor extends BaseValueVector.BaseAccessor implements VariableWidthAccessor {
     final UInt${type.width}Vector.Accessor oAccessor = offsetVector.getAccessor();
+    
     public long getStartEnd(int index){
       return oAccessor.getTwoAsLong(index);
     }
@@ -463,7 +470,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       holder.buffer = data;
     }
 
-
     <#switch minor.class>
     <#case "VarChar">
     @Override
@@ -541,10 +547,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       }
     }
 
-    public void setScalar(int index, byte[] bytes) throws VectorOverflowException {
-      setScalar(index, bytes, 0, bytes.length);
-    }
-
     /**
      * Set the variable length element at the specified index to the supplied byte array.
      *
@@ -575,23 +577,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       }
     }
 
-    public void setScalar(int index, DrillBuf bytes, int start, int length) throws VectorOverflowException {
-      assert index >= 0;
-
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      int currentOffset = offsetVector.getAccessor().get(index);
-      final int newSize = currentOffset + length;
-      if (newSize > MAX_BUFFER_SIZE) {
-        throw new VectorOverflowException();
-      }
-      while (! data.setBytesBounded(currentOffset, bytes, start, length)) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, newSize);
-    }
-
     public void setSafe(int index, byte[] bytes, int start, int length) {
       assert index >= 0;
 
@@ -608,28 +593,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       }
     }
 
-    public void setScalar(int index, byte[] bytes, int start, int length) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setArrayItem(index, bytes, start, length);
-    }
-
-    public void setArrayItem(int index, byte[] bytes, int start, int length) throws VectorOverflowException {
-      assert index >= 0;
-
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      final int newSize = currentOffset + length;
-      if (newSize > MAX_BUFFER_SIZE) {
-        throw new VectorOverflowException();
-      }
-
-      while (! data.setBytesBounded(currentOffset, bytes, start, length)) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, newSize);
-    }
-
     @Override
     public void setValueLengthSafe(int index, int length) {
       final int offset = offsetVector.getAccessor().get(index);
@@ -654,32 +617,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       }
     }
 
-    public void setScalar(int index, int start, int end, DrillBuf buffer) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setArrayItem(index, start, end, buffer);
-    }
-
-    public void setArrayItem(int index, int start, int end, DrillBuf buffer) throws VectorOverflowException {
-      final int len = end - start;
-      final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-      final int newSize = outputStart + len;
-      if (newSize > MAX_BUFFER_SIZE) {
-        throw new VectorOverflowException();
-      }
-
-      offsetVector.getMutator().setSafe(index+1, newSize);
-      try{
-        buffer.getBytes(start, data, outputStart, len);
-      } catch (IndexOutOfBoundsException e) {
-        while (data.capacity() < newSize) {
-          reAlloc();
-        }
-        buffer.getBytes(start, data, outputStart, len);
-      }
-    }
-
     public void setSafe(int index, Nullable${minor.class}Holder holder) {
       assert holder.isSet == 1;
 
@@ -700,37 +637,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       offsetVector.getMutator().setSafe(index+1,  outputStart + len);
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setArrayItem(index, holder);
-    }
-
-    public void setArrayItem(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      assert holder.isSet == 1;
-
-      final int start = holder.start;
-      final int end =   holder.end;
-      final int len = end - start;
-
-      final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-      final int newSize = outputStart + len;
-      if (newSize > MAX_BUFFER_SIZE) {
-        throw new VectorOverflowException();
-      }
-
-      try {
-        holder.buffer.getBytes(start, data, outputStart, len);
-      } catch (IndexOutOfBoundsException e) {
-        while (data.capacity() < newSize) {
-          reAlloc();
-        }
-        holder.buffer.getBytes(start, data, outputStart, len);
-      }
-      offsetVector.getMutator().setSafe(index+1, newSize);
-    }
-
     public void setSafe(int index, ${minor.class}Holder holder) {
       final int start = holder.start;
       final int end =   holder.end;
@@ -748,34 +654,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       offsetVector.getMutator().setSafe( index+1,  outputStart + len);
     }
 
-    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setArrayItem(index, holder);
-   }
-
-    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      final int start = holder.start;
-      final int end =   holder.end;
-      final int len = end - start;
-      final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-      final int newSize = outputStart + len;
-      if (newSize > MAX_BUFFER_SIZE) {
-        throw new VectorOverflowException();
-      }
-
-      try {
-        holder.buffer.getBytes(start, data, outputStart, len);
-      } catch (IndexOutOfBoundsException e) {
-        while(data.capacity() < newSize) {
-          reAlloc();
-        }
-        holder.buffer.getBytes(start, data, outputStart, len);
-      }
-      offsetVector.getMutator().setSafe( index+1, newSize);
-    }
-
     /**
      * Backfill missing offsets from the given last written position to the
      * given current write position. Used by the "new" size-safe column
@@ -785,20 +663,9 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
      * to be copied forward
      * @param index the current write position filling occurs up to,
      * but not including, this position
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
      */
 
-    public void fillEmptiesBounded(int lastWrite, int index)
-            throws VectorOverflowException {
-
-      // Index is the next write index, which might be "virtual",
-      // that is, past the last row at EOF. This check only protects
-      // the actual data written here, which is up to index-1.
-
-      if (index > UInt4Vector.MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
+    public void fillEmpties(int lastWrite, int index) {
       // If last write was 2, offsets are [0, 3, 6]
       // If next write is 4, offsets must be: [0, 3, 6, 6, 6]
       // Remember the offsets are one more than row count.
@@ -810,7 +677,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       }
     }
 
-    protected void set(int index, int start, int length, DrillBuf buffer){
+    protected void set(int index, int start, int length, DrillBuf buffer) {
       assert index >= 0;
       final int currentOffset = offsetVector.getAccessor().get(index);
       offsetVector.getMutator().set(index + 1, currentOffset + length);
@@ -818,33 +685,20 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
       data.setBytes(currentOffset, bb);
     }
 
-    protected void set(int index, Nullable${minor.class}Holder holder){
+    protected void set(int index, Nullable${minor.class}Holder holder) {
       final int length = holder.end - holder.start;
       final int currentOffset = offsetVector.getAccessor().get(index);
       offsetVector.getMutator().set(index + 1, currentOffset + length);
       data.setBytes(currentOffset, holder.buffer, holder.start, length);
     }
 
-    protected void set(int index, ${minor.class}Holder holder){
+    protected void set(int index, ${minor.class}Holder holder) {
       final int length = holder.end - holder.start;
       final int currentOffset = offsetVector.getAccessor().get(index);
       offsetVector.getMutator().set(index + 1, currentOffset + length);
       data.setBytes(currentOffset, holder.buffer, holder.start, length);
     }
 
-  <#if (minor.class == "VarChar")>
-    public void setScalar(int index, String value) throws VectorOverflowException {
-      if (index >= MAX_ROW_COUNT) {
-        throw new VectorOverflowException();
-      }
-      // Treat a null string as an empty string.
-      if (value != null) {
-        byte encoded[] = value.getBytes(Charsets.UTF_8);
-        setScalar(index, encoded, 0, encoded.length);
-      }
-    }
-
-  </#if>
     @Override
     public void setValueCount(int valueCount) {
       final int currentByteCapacity = getByteCapacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
new file mode 100644
index 0000000..558aab8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/ColumnMetadata.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Metadata description of a column including names, types and structure
+ * information.
+ */
+
+public interface ColumnMetadata {
+  enum StructureType {
+    PRIMITIVE, LIST, TUPLE
+  }
+
+  public static final int DEFAULT_ARRAY_SIZE = 10;
+
+  ColumnMetadata.StructureType structureType();
+  TupleMetadata mapSchema();
+  MaterializedField schema();
+  String name();
+  MajorType majorType();
+  MinorType type();
+  DataMode mode();
+  boolean isNullable();
+  boolean isArray();
+  boolean isVariableWidth();
+  boolean isMap();
+  boolean isList();
+
+  /**
+   * Report whether one column is equivalent to another. Columns are equivalent
+   * if they have the same name, type and structure (ignoring internal structure
+   * such as offset vectors.)
+   */
+
+  boolean isEquivalent(ColumnMetadata other);
+
+  /**
+   * For variable-width columns, specify the expected column width to be used
+   * when allocating a new vector. Does nothing for fixed-width columns.
+   *
+   * @param width the expected column width
+   */
+
+  void setExpectedWidth(int width);
+
+  /**
+   * Get the expected width for a column. This is the actual width for fixed-
+   * width columns, the specified width (defaulting to 50) for variable-width
+   * columns.
+   * @return the expected column width of the each data value. Does not include
+   * "overhead" space such as for the null-value vector or offset vector
+   */
+
+  int expectedWidth();
+
+  /**
+   * For an array column, specify the expected average array cardinality.
+   * Ignored for non-array columns. Used when allocating new vectors.
+   *
+   * @param childCount the expected average array cardinality. Defaults to
+   * 1 for non-array columns, 10 for array columns
+   */
+
+  void setExpectedElementCount(int childCount);
+
+  /**
+   * Returns the expected array cardinality for array columns, or 1 for
+   * non-array columns.
+   *
+   * @return the expected value cardinality per value (per-row for top-level
+   * columns, per array element for arrays within lists)
+   */
+
+  int expectedElementCount();
+
+  /**
+   * Create an empty version of this column. If the column is a scalar,
+   * produces a simple copy. If a map, produces a clone without child
+   * columns.
+   *
+   * @return empty clone of this column
+   */
+
+  ColumnMetadata cloneEmpty();
+
+  /**
+   * Reports whether, in this context, the column is projected outside
+   * of the context. (That is, whether the column is backed by an actual
+   * value vector.)
+   */
+
+  boolean isProjected();
+  void setProjected(boolean projected);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 4d29d55..b4b23c7 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
@@ -92,6 +92,10 @@ public class MaterializedField {
     return withPathAndType(name, getType());
   }
 
+  public MaterializedField cloneEmpty() {
+    return create(name, type);
+  }
+
   public MaterializedField withType(MajorType type) {
     return withPathAndType(name, type);
   }
@@ -170,7 +174,37 @@ public class MaterializedField {
             Objects.equals(this.type, other.type);
   }
 
+  /**
+   * Determine if one column is logically equivalent to another. This is
+   * a tricky issue. The rules here:
+   * <ul>
+   * <li>The other schema is assumed to be non-null (unlike
+   * <tt>equals()</tt>).</li>
+   * <li>Names must be identical, ignoring case. (Drill, like SQL, is
+   * case insensitive.)
+   * <li>Type, mode, precision and scale must be identical.</li>
+   * <li>Child columns are ignored unless the type is a map. That is, the
+   * hidden "$bits" and "$offsets" vector columns are not compared, as
+   * one schema may be an "original" (without these hidden columns) while
+   * the other may come from a vector (which has the hidden columns added.
+   * The standard <tt>equals()</tt> comparison does consider hidden
+   * columns.</li>
+   * <li>For maps, the child columns are compared recursively. This version
+   * requires that the two sets of columns appear in the same order. (It
+   * assumes it is being used in a context where column indexes make
+   * sense.) Operators that want to reconcile two maps that differ only in
+   * column order need a different comparison.</li>
+   * </ul>
+   *
+   * @param other another field
+   * @return <tt>true</tt> if the columns are identical according to the
+   * above rules, <tt>false</tt> if they differ
+   */
+
   public boolean isEquivalent(MaterializedField other) {
+    if (this == other) {
+      return true;
+    }
     if (! name.equalsIgnoreCase(other.name)) {
       return false;
     }
@@ -199,7 +233,7 @@ public class MaterializedField {
       return true;
     }
 
-    if (children == null  ||  other.children == null) {
+    if (children == null || other.children == null) {
       return children == other.children;
     }
     if (children.size() != other.children.size()) {
@@ -226,11 +260,12 @@ public class MaterializedField {
    * Includes field name, its type with precision and scale if any and data mode.
    * Nested fields if any are included. Number of nested fields to include is limited to 10.</p>
    *
-   * <b>FIELD_NAME(TYPE(PRECISION,SCALE):DATA_MODE)[NESTED_FIELD_1, NESTED_FIELD_2]</b>
+   * <b>FIELD_NAME(TYPE(PRECISION,SCALE):DATA_MODE)[NESTED_FIELD_1, NESTED_FIELD_2]</b><br>
    * <p>Example: ok(BIT:REQUIRED), col(VARCHAR(3):OPTIONAL), emp_id(DECIMAL28SPARSE(6,0):REQUIRED)</p>
    *
    * @return materialized field string representation
    */
+
   @Override
   public String toString() {
     final int maxLen = 10;
@@ -258,7 +293,7 @@ public class MaterializedField {
         .append(childString);
 
     return builder.toString();
-}
+  }
 
   /**
    * Return true if two fields have identical MinorType and Mode.

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
new file mode 100644
index 0000000..8f597be
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleMetadata.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record;
+
+import java.util.List;
+
+/**
+ * Metadata description of the schema of a row or a map.
+ * 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.
+ * <p>
+ * Models the physical schema of a row set showing the logical hierarchy of fields
+ * with map fields as first-class fields. Map members appear as children
+ * under the map, much as they appear in the physical value-vector
+ * implementation.
+ * <ul>
+ * <li>Provides fast lookup by name or index.</li>
+ * <li>Provides a nested schema, in this same form, for maps.</li>
+ * </ul>
+ * This form is useful when performing semantic analysis and when
+ * working with vectors.
+ * <p>
+ * In the future, this structure will also gather metadata useful
+ * for vector processing such as expected widths and so on.
+ */
+
+public interface TupleMetadata extends Iterable<ColumnMetadata> {
+
+  /**
+   * Add a new column to the schema.
+   *
+   * @param columnSchema
+   * @return the index of the new column
+   */
+  ColumnMetadata add(MaterializedField field);
+  int addColumn(ColumnMetadata column);
+
+  int size();
+  boolean isEmpty();
+  int index(String name);
+  ColumnMetadata metadata(int index);
+  ColumnMetadata metadata(String name);
+  MaterializedField column(int index);
+  MaterializedField column(String name);
+  boolean isEquivalent(TupleMetadata other);
+  ColumnMetadata parent();
+
+  /**
+   * Return the schema as a list of <tt>MaterializedField</tt> objects
+   * which can be used to create other schemas. Not valid for a
+   * flattened schema.
+   *
+   * @return a list of the top-level fields. Maps contain their child
+   * fields
+   */
+
+  List<MaterializedField> toFieldList();
+
+  /**
+   * Full name of the column. Note: this name cannot be used to look up
+   * the column because of ambiguity. The name "a.b.c" may mean a single
+   * column with that name, or may mean maps "a", and "b" with column "c",
+   * etc.
+   *
+   * @return full, dotted, column name
+   */
+
+  String fullName(ColumnMetadata column);
+  String fullName(int index);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
new file mode 100644
index 0000000..5853c93
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/TupleNameSpace.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Implementation of a tuple name space. Tuples allow both indexed and
+ * named access to their members.
+ *
+ * @param <T> the type of object representing each column
+ */
+
+public class TupleNameSpace<T> implements Iterable<T> {
+  private final Map<String,Integer> nameSpace = CaseInsensitiveMap.newHashMap();
+  private final List<T> entries = new ArrayList<>();
+
+  public int add(String key, T value) {
+    if (indexOf(key) != -1) {
+      throw new IllegalArgumentException("Duplicate entry: " + key);
+    }
+    int index = entries.size();
+    nameSpace.put(key, index);
+    entries.add(value);
+    return index;
+  }
+
+  public T get(int index) {
+    return entries.get(index);
+  }
+
+  public T get(String key) {
+    int index = indexOf(key);
+    if (index == -1) {
+      return null;
+    }
+    return get(index);
+  }
+
+  public int indexOf(String key) {
+    Integer index = nameSpace.get(key);
+    if (index == null) {
+      return -1;
+    }
+    return index;
+  }
+
+  public int count() { return entries.size(); }
+
+  @Override
+  public Iterator<T> iterator() {
+    return entries.iterator();
+  }
+
+  public boolean isEmpty() {
+    return entries.isEmpty();
+  }
+
+  public List<T> entries() {
+    return ImmutableList.copyOf(entries);
+  }
+
+  @Override
+  public String toString() {
+    return entries.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
index 100997e..4fd0cbd 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/AllocationHelper.java
@@ -39,7 +39,7 @@ public class AllocationHelper {
       ((VariableWidthVector) vector).allocateNew(valueCount * bytesPerValue, valueCount);
     } else if (vector instanceof RepeatedFixedWidthVectorLike) {
       ((RepeatedFixedWidthVectorLike) vector).allocateNew(valueCount, childValCount);
-    } else if (vector instanceof RepeatedVariableWidthVectorLike && childValCount > 0 && bytesPerValue > 0) {
+    } else if (vector instanceof RepeatedVariableWidthVectorLike) {
       // Assertion thrown if byte count is zero in the full allocateNew,
       // so use default version instead.
       ((RepeatedVariableWidthVectorLike) vector).allocateNew(childValCount * bytesPerValue, valueCount, childValCount);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index e98a417..4391e8c 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -37,6 +37,16 @@ public abstract class BaseDataValueVector extends BaseValueVector {
     data = allocator.getEmpty();
   }
 
+  /**
+   * Core of vector allocation. Given a new size (which must be a power of two), allocate
+   * the new buffer, copy the current values, and leave the unused parts garbage-filled.
+   *
+   * @param newAllocationSize new buffer size as a power of two
+   * @return the new buffer
+   */
+
+  public abstract DrillBuf reallocRaw(int newAllocationSize);
+
   @Override
   public void clear() {
     if (data != null) {
@@ -82,6 +92,11 @@ public abstract class BaseDataValueVector extends BaseValueVector {
     return data.writerIndex();
   }
 
+  @Override
+  public int getAllocatedSize() {
+    return data.capacity();
+  }
+
   public DrillBuf getBuffer() { return data; }
 
   /**
@@ -101,6 +116,7 @@ public abstract class BaseDataValueVector extends BaseValueVector {
     // No state in an Accessor to reset
   }
 
+  @Override
   public void collectLedgers(Set<BufferLedger> ledgers) {
     BufferLedger ledger = data.getLedger();
     if (ledger != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
index f879fc4..219db12 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -193,6 +193,16 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
     allocationSizeInBytes = curSize;
   }
 
+  // This version uses the base version because this vector appears to not be
+  // used, so not worth the effort to avoid zero-fill.
+
+  public DrillBuf reallocRaw(int newAllocationSize) {
+    while (allocationSizeInBytes < newAllocationSize) {
+      reAlloc();
+    }
+    return data;
+  }
+
   /**
    * {@inheritDoc}
    */
@@ -438,20 +448,6 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       set(index, value);
     }
 
-    public void setScalar(int index, int value) throws VectorOverflowException {
-      if (index >= MAX_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
-    public void setArrayItem(int index, int value) throws VectorOverflowException {
-      if (index >= MAX_CAPACITY) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
     public void setSafe(int index, BitHolder holder) {
       while(index >= getValueCapacity()) {
         reAlloc();
@@ -459,20 +455,6 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       set(index, holder.value);
     }
 
-    public void setScalar(int index, BitHolder holder) throws VectorOverflowException {
-      if (index >= MAX_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
-    public void setArrayItem(int index, BitHolder holder) throws VectorOverflowException {
-      if (index >= MAX_CAPACITY) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
     public void setSafe(int index, NullableBitHolder holder) {
       while(index >= getValueCapacity()) {
         reAlloc();
@@ -480,20 +462,6 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       set(index, holder.value);
     }
 
-    public void setScalar(int index, NullableBitHolder holder) throws VectorOverflowException {
-      if (index >= MAX_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
-    public void setArrayItem(int index, NullableBitHolder holder) throws VectorOverflowException {
-      if (index >= MAX_CAPACITY) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
     @Override
     public final void setValueCount(int valueCount) {
       int currentValueCapacity = getValueCapacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
index a9a1631..09bcdd8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/FixedWidthVector.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.vector;
 
-
 public interface FixedWidthVector extends ValueVector {
 
   /**
@@ -27,8 +26,8 @@ public interface FixedWidthVector extends ValueVector {
    */
   void allocateNew(int valueCount);
 
-/**
- * Zero out the underlying buffer backing this vector.
- */
+  /**
+   * Zero out the underlying buffer backing this vector.
+   */
   void zeroVector();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
index c9edeb0..8d515d8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ObjectVector.java
@@ -95,8 +95,7 @@ public class ObjectVector extends BaseValueVector {
     }
 
     @Override
-    public void generateTestData(int values) {
-    }
+    public void generateTestData(int values) { }
 
     @Override
     public void exchange(ValueVector.Mutator other) { }
@@ -130,6 +129,9 @@ public class ObjectVector extends BaseValueVector {
   }
 
   @Override
+  public int getAllocatedSize() { return 0; }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) {
     throw new UnsupportedOperationException("ObjectVector does not support this");
   }
@@ -147,9 +149,7 @@ public class ObjectVector extends BaseValueVector {
   }
 
   @Override
-  public MaterializedField getField() {
-    return field;
-  }
+  public MaterializedField getField() { return field; }
 
   @Override
   public TransferPair getTransferPair(BufferAllocator allocator) {
@@ -172,14 +172,10 @@ public class ObjectVector extends BaseValueVector {
   }
 
   @Override
-  public int getValueCapacity() {
-    return maxCount;
-  }
+  public int getValueCapacity() { return maxCount; }
 
   @Override
-  public Accessor getAccessor() {
-    return accessor;
-  }
+  public Accessor getAccessor() { return accessor; }
 
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
@@ -197,9 +193,7 @@ public class ObjectVector extends BaseValueVector {
   }
 
   @Override
-  public Mutator getMutator() {
-    return mutator;
-  }
+  public Mutator getMutator() { return mutator; }
 
   @Override
   public Iterator<ValueVector> iterator() {
@@ -222,9 +216,7 @@ public class ObjectVector extends BaseValueVector {
     }
 
     @Override
-    public int getValueCount() {
-      return count;
-    }
+    public int getValueCount() { return count; }
 
     public Object get(int index) {
       return getObject(index);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
index 8288fe2..5565fa4 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/UntypedNullVector.java
@@ -27,8 +27,6 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
-import static org.apache.calcite.sql.parser.impl.SqlParserImplConstants.C;
-
 /** UntypedNullVector is to represent a value vector with {@link org.apache.drill.common.types.MinorType#NULL}
  *  All values in the vector represent two semantic implications: 1) the value is unknown, 2) the type is unknown.
  *  Because of this, we only have to keep track of the number of values in value vector,
@@ -37,7 +35,6 @@ import static org.apache.calcite.sql.parser.impl.SqlParserImplConstants.C;
  *
  */
 public final class UntypedNullVector extends BaseDataValueVector implements FixedWidthVector {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UntypedNullVector.class);
 
   /**
    * Width of each fixed-width value.
@@ -57,14 +54,10 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
   public FieldReader getReader() { throw new UnsupportedOperationException(); }
 
   @Override
-  public int getBufferSizeFor(final int valueCount) {
-    return 0;
-  }
+  public int getBufferSizeFor(final int valueCount) { return 0; }
 
   @Override
-  public int getValueCapacity(){
-    return ValueVector.MAX_ROW_COUNT;
-  }
+  public int getValueCapacity() { return ValueVector.MAX_ROW_COUNT; }
 
   @Override
   public Accessor getAccessor() { return accessor; }
@@ -73,31 +66,29 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
   public Mutator getMutator() { return mutator; }
 
   @Override
-  public void setInitialCapacity(final int valueCount) {
-  }
+  public void setInitialCapacity(final int valueCount) { }
 
   @Override
-  public void allocateNew() {
-  }
+  public void allocateNew() { }
 
   @Override
-  public boolean allocateNewSafe() {
-    return true;
-  }
+  public boolean allocateNewSafe() { return true; }
 
   @Override
-  public void allocateNew(final int valueCount) {
-  }
+  public void allocateNew(final int valueCount) { }
 
   @Override
-  public void reset() {
-  }
+  public void reset() { }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public void zeroVector() {
+  public void zeroVector() { }
+
+  @Override
+  public DrillBuf reallocRaw(int newAllocationSize) {
+    throw new UnsupportedOperationException();
   }
 
   @Override
@@ -127,19 +118,15 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
     return new TransferImpl((UntypedNullVector) to);
   }
 
-  public void transferTo(UntypedNullVector target){
-  }
+  public void transferTo(UntypedNullVector target) { }
 
-  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) {
-  }
+  public void splitAndTransferTo(int startIndex, int length, UntypedNullVector target) { }
 
   @Override
-  public int getPayloadByteCount(int valueCount) {
-    return 0;
-  }
+  public int getPayloadByteCount(int valueCount) { return 0; }
 
   private class TransferImpl implements TransferPair{
-    private UntypedNullVector to;
+    private final UntypedNullVector to;
 
     public TransferImpl(MaterializedField field, BufferAllocator allocator){
       to = new UntypedNullVector(field, allocator);
@@ -150,9 +137,7 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
     }
 
     @Override
-    public UntypedNullVector getTo(){
-      return to;
-    }
+    public UntypedNullVector getTo() { return to; }
 
     @Override
     public void transfer(){
@@ -173,11 +158,9 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from){
-  }
+  public void copyFrom(int fromIndex, int thisIndex, UntypedNullVector from) { }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from){
-  }
+  public void copyFromSafe(int fromIndex, int thisIndex, UntypedNullVector from) { }
 
   private void checkBounds(int index) {
     if (index < 0 || index >= valueCount) {
@@ -216,7 +199,6 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
     public void get(int index, UntypedNullHolder holder) {
       checkBounds(index);
     }
-
   }
 
   /**
@@ -224,7 +206,7 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
    * value counts.
    *
    */
-   public final class Mutator extends BaseMutator {
+  public final class Mutator extends BaseMutator {
 
     private Mutator() {}
 
@@ -266,5 +248,4 @@ public final class UntypedNullVector extends BaseDataValueVector implements Fixe
       UntypedNullVector.this.valueCount = valueCount;
     }
   }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index a090cad..bc06803 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -25,6 +25,7 @@ import io.netty.buffer.DrillBuf;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
+import org.apache.drill.exec.memory.AllocationManager;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
@@ -32,12 +33,15 @@ import org.apache.drill.exec.record.TransferPair;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
 /**
- * An abstraction that is used to store a sequence of values in an individual column.
+ * An abstraction that is used to store a sequence of values in an individual
+ * column.
  *
- * A {@link ValueVector value vector} stores underlying data in-memory in a columnar fashion that is compact and
- * efficient. The column whose data is stored, is referred by {@link #getField()}.
+ * A {@link ValueVector value vector} stores underlying data in-memory in a
+ * columnar fashion that is compact and efficient. The column whose data is
+ * stored, is referred by {@link #getField()}.
  *
- * A vector when instantiated, relies on a {@link org.apache.drill.exec.record.DeadBuf dead buffer}. It is important
+ * A vector when instantiated, relies on a
+ * {@link org.apache.drill.exec.record.DeadBuf dead buffer}. It is important
  * that vector is allocated before attempting to read or write.
  *
  * There are a few "rules" around vectors:
@@ -45,37 +49,34 @@ import org.apache.drill.exec.vector.complex.reader.FieldReader;
  * <ul>
  *   <li>Values need to be written in order (e.g. index 0, 1, 2, 5).</li>
  *   <li>Null vectors start with all values as null before writing anything.</li>
- *   <li>For variable width types, the offset vector should be all zeros before writing.</li>
+ *   <li>For variable width types, the offset vector should be all zeros before
+ *   writing.</li>
  *   <li>You must call setValueCount before a vector can be read.</li>
  *   <li>You should never write to a vector once it has been read.</li>
- *   <li>Vectors may not grow larger than the number of bytes specified
- *   in {@link #MAX_BUFFER_SIZE} to prevent memory fragmentation. Use the
+ *   <li>Vectors may not grow larger than the number of bytes specified in
+ *   {@link #MAX_BUFFER_SIZE} to prevent memory fragmentation. Use the
  *   <tt>setBounded()</tt> methods in the mutator to enforce this rule.</li>
  * </ul>
  *
- * Please note that the current implementation doesn't enforce those rules, hence we may find few places that
- * deviate from these rules (e.g. offset vectors in Variable Length and Repeated vector)
+ * Please note that the current implementation doesn't enforce those rules,
+ * hence we may find few places that deviate from these rules (e.g. offset
+ * vectors in Variable Length and Repeated vector)
  *
  * This interface "should" strive to guarantee this order of operation:
  * <blockquote>
- * allocate > mutate > setvaluecount > access > clear (or allocate to start the process over).
+ * allocate > mutate > setvaluecount > access > clear (or allocate
+ * to start the process over).
  * </blockquote>
  */
+
 public interface ValueVector extends Closeable, Iterable<ValueVector> {
 
   /**
    * Maximum allowed size of the buffer backing a value vector.
+   * Set to the Netty chunk size to prevent memory fragmentation.
    */
 
-  int MAX_BUFFER_SIZE = VectorUtils.maxSize();
-
-  /**
-   * Debug-time system option that artificially limits vector lengths
-   * for testing. Must be set prior to the first reference to this
-   * class. (Made deliberately difficult to prevent misuse...)
-   */
-
-  String MAX_BUFFER_SIZE_KEY = "drill.max_vector";
+  int MAX_BUFFER_SIZE = AllocationManager.chunkSize();
 
   /**
    * Maximum allowed row count in a vector. Repeated vectors
@@ -167,10 +168,24 @@ public interface ValueVector extends Closeable, Iterable<ValueVector> {
 
   /**
    * Returns the number of bytes that is used by this vector instance.
+   * This is a bit of a misnomer. Returns the number of bytes used by
+   * data in this instance.
    */
   int getBufferSize();
 
   /**
+   * Returns the total size of buffers allocated by this vector. Has
+   * meaning only when vectors are directly allocated and each vector
+   * has its own buffer. Does not have meaning for vectors deserialized
+   * from the network or disk in which multiple vectors share the
+   * same vector.
+   *
+   * @return allocated buffer size, in bytes
+   */
+
+  int getAllocatedSize();
+
+  /**
    * Returns the number of bytes that is used by this vector if it holds the given number
    * of values. The result will be the same as if Mutator.setValueCount() were called, followed
    * by calling getBufferSize(), but without any of the closing side-effects that setValueCount()

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
index d04234c..f5373d0 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.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
@@ -33,8 +33,10 @@ public interface VariableWidthVector extends ValueVector {
    */
   int getByteCapacity();
 
+  @Override
   VariableWidthMutator getMutator();
 
+  @Override
   VariableWidthAccessor getAccessor();
 
   interface VariableWidthAccessor extends Accessor {

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/VectorUtils.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/VectorUtils.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/VectorUtils.java
deleted file mode 100644
index 6b29eb2..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/VectorUtils.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.vector;
-
-public class VectorUtils {
-
-  /**
-   * Vectors cannot be any larger than the Netty memory allocation
-   * block size.
-   */
-
-  private static final int ABSOLUTE_MAX_SIZE = 16 * 1024 * 1024;
-
-  /**
-   * Minimum size selected to prevent pathological performance if vectors
-   * are limited to an unusably small size. This limit is a judgment call,
-   * not based on any known limits.
-   */
-
-  private static final int ABSOLUTE_MIN_SIZE = 16 * 1024;
-
-  private VectorUtils() { }
-
-  /**
-   * Static function called once per run to compute the maximum
-   * vector size, in bytes. Normally uses the hard-coded limit,
-   * but allows setting a system property to override the limit
-   * for testing. The configured value must be within reasonable
-   * bounds.
-   * @return the maximum vector size, in bytes
-   */
-
-  static int maxSize() {
-    String prop = System.getProperty( ValueVector.MAX_BUFFER_SIZE_KEY );
-    int value = ABSOLUTE_MAX_SIZE;
-    if (prop != null) {
-      try {
-        value = Integer.parseInt(prop);
-        value = Math.max(value, ABSOLUTE_MIN_SIZE);
-        value = Math.min(value, ABSOLUTE_MAX_SIZE);
-      } catch (NumberFormatException e) {
-        // Ignore
-      }
-    }
-    return value;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
index e6f0544..fc89d71 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ZeroVector.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.vector;
 
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Set;
 
@@ -105,13 +106,16 @@ public class ZeroVector implements ValueVector {
 
   @Override
   public Iterator<ValueVector> iterator() {
-    return Iterators.emptyIterator();
+    return Collections.emptyIterator();
   }
 
   @Override
   public int getBufferSize() { return 0; }
 
   @Override
+  public int getAllocatedSize() { return 0; }
+
+  @Override
   public int getBufferSizeFor(final int valueCount) { return 0; }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 708d0db..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/AccessorUtilities.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
index 040dcda..8f33f0e 100644
--- 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
@@ -17,36 +17,90 @@
  */
 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.
+ * Generic array reader. An array is one of the following:
+ * <ul>
+ * <li>Array of scalars. Read the values using {@link #elements()}, which provides
+ * an array-like access to the scalars.</li>
+ * <li>A repeated map. Use {@link #tuple(int)} to get a tuple reader for a
+ * specific array element. Use {@link #size()} to learn the number of maps in
+ * the array.</li>
+ * <li>List of lists. Use the {@link #array(int)} method to get the nested list
+ * at a given index. Use {@link #size()} to learn the number of maps in
+ * the array.</li>
+ * </ul>
+ * {@see ArrayWriter}
  */
 
-public interface ArrayReader extends ColumnAccessor {
+public interface ArrayReader {
+
+  /**
+   * Number of elements in the array.
+   * @return the number of elements
+   */
+
   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);
+
+  /**
+   * The object type of the list entry. All entries have the same
+   * type.
+   * @return the object type of each entry
+   */
+
+  ObjectType entryType();
+
+  /**
+   * Return a reader for the elements of a scalar array.
+   * @return reader for scalar elements
+   */
+
+  ScalarElementReader elements();
+
+  /**
+   * Return a generic object reader for the array entry. Not available
+   * for scalar elements. Positions the reader to read the selected
+   * element.
+   *
+   * @param index array index
+   * @return generic object reader
+   */
+
+  ObjectReader entry(int index);
+  TupleReader tuple(int index);
   ArrayReader array(int index);
+
+  /**
+   * Return the generic object reader for the array element. This
+   * version <i>does not</i> position the reader, the client must
+   * call {@link setPosn()} to set the position. This form allows
+   * up-front setup of the readers when convenient for the caller.
+   */
+
+  ObjectReader entry();
+  TupleReader tuple();
+  ArrayReader array();
+
+  /**
+   * Set the array reader to read a given array entry. Not used for
+   * scalars, only for maps and arrays when using the non-indexed
+   * methods {@link #entry()}, {@link #tuple()} and {@link #array()}.
+   */
+
+  void setPosn(int index);
+
+  /**
+   * Return the entire array as an <tt>List</tt> of objects.
+   * Note, even if the array is scalar, the elements are still returned
+   * as a list. This method is primarily for testing.
+   * @return array as a <tt>List</tt> of objects
+   */
+
+  Object getObject();
+
+  /**
+   * Return the entire array as a string. Primarily for debugging.
+   * @return string representation of the array
+   */
+
+  String getAsString();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index 16ff89e..49a1e77 100644
--- 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
@@ -18,25 +18,65 @@
 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.
+ * 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 {
+public interface ArrayWriter {
+
+  /**
+   * Number of elements written thus far to the array.
+   * @return the number of elements
+   */
 
   int size();
 
   /**
-   * Determine if the next position is valid for writing. Will be invalid
-   * if the writer hits a size or other limit.
+   * The object type of the list entry. All entries have the same
+   * type.
+   * @return the object type of each entry
+   */
+
+  ObjectWriter entry();
+
+  /**
+   * Return a generic object writer for the array entry.
+   *
+   * @return generic object reader
+   */
+
+  ObjectType entryType();
+  ScalarWriter scalar();
+  TupleWriter tuple();
+  ArrayWriter array();
+
+  /**
+   * When the array contains a tuple or an array, call <tt>save()</tt>
+   * after each array value. Not necessary when writing scalars; each
+   * set operation calls save automatically.
+   */
+
+  void save();
+
+  /**
+   * Write the values of an array from a list of arguments.
+   * @param values values for each array element
+   * @throws VectorOverflowException
+   */
+  void set(Object ...values);
+
+  /**
+   * Write the array given an array of values. The type of array must match
+   * the type of element in the array. That is, if the value is an <tt>int</tt>,
+   * provide an <tt>int[]</tt> array.
    *
-   * @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
+   * @param array array of values to write
+   * @throws VectorOverflowException
    */
 
-  boolean valid();
+  void setObject(Object array);
+//  void setList(List<? extends Object> list);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 44cd48a..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnAccessor.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
deleted file mode 100644
index 4932567..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReader.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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();
-  Object getObject();
-  TupleReader map();
-  ArrayReader array();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReaderIndex.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReaderIndex.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReaderIndex.java
new file mode 100644
index 0000000..b40b705
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnReaderIndex.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Index into a vector batch, or an array, at read time.
+ * Supports direct, indirect and hyper-batches.
+ */
+
+public interface ColumnReaderIndex {
+  int batchIndex();
+  int vectorIndex();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 0cc691c..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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();
-}


[13/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/package-info.java
new file mode 100644
index 0000000..4c11499
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/package-info.java
@@ -0,0 +1,304 @@
+/*
+ * 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.
+ */
+
+/**
+ * Handles the details of the result set loader implementation.
+ * <p>
+ * The primary purpose of this loader, and the most complex to understand and
+ * maintain, is overflow handling.
+ *
+ * <h4>Detailed Use Cases</h4>
+ *
+ * Let's examine it by considering a number of
+ * use cases.
+ * <table style="border: 1px solid; border-collapse: collapse;">
+ * <tr><th>Row</th><th>a</th><th>b</th><th>c</th><th>d</th><th>e</th><th>f</th><th>g</th><th>h</th></tr>
+ * <tr><td>n-2</td><td>X</td><td>X</td><td>X</td><td>X</td><td>X</td><td>X</td><td>-</td><td>-</td></tr>
+ * <tr><td>n-1</td><td>X</td><td>X</td><td>X</td><td>X</td><td> </td><td> </td><td>-</td><td>-</td></tr>
+ * <tr><td>n  </td><td>X</td><td>!</td><td>O</td><td> </td><td>O</td><td> </td><td>O</td><td> </td></tr>
+ * </table>
+ * Here:
+ * <ul>
+ * <li>n-2, n-1, and n are rows. n is the overflow row.</li>
+ * <li>X indicates a value was written before overflow.</li>
+ * <li>Blank indicates no value was written in that row.</li>
+ * <li>! indicates the value that triggered overflow.</li>
+ * <li>- indicates a column that did not exist prior to overflow.</li>
+ * <li>O indicates a value written after overflow.</li>
+ * </ul>
+ * Column a is written before overflow occurs, b causes overflow, and all other
+ * columns either are not written, or written after overflow.
+ * <p>
+ * The scenarios, identified by column names above, are:
+ * <dl>
+ * <dt>a</dt>
+ * <dd>a contains values for all three rows.
+ * <ul>
+ * <li>Two values were written in the "main" batch, while a third was written to
+ * what becomes the overflow row.</li>
+ * <li>When overflow occurs, the last write position is at n. It must be moved
+ * back to n-1.</li>
+ * <li>Since data was written to the overflow row, it is copied to the look-
+ * ahead batch.</li>
+ * <li>The last write position in the lookahead batch is 0 (since data was
+ * copied into the 0th row.</li>
+ * <li>When harvesting, no empty-filling is needed. Values in the main
+ * batch are zero-filled when the batch is finished, values in the look-ahead
+ * batch are back-filled when the first value is written.</li>
+ * <li>When starting the next batch, the last write position must be set to 0 to
+ * reflect the presence of the value for row n.</li>
+ * </ul>
+ * </dd>
+ * <dt>b</dt>
+ * <dd>b contains values for all three rows. The value for row n triggers
+ * overflow.
+ * <ul>
+ * <li>The last write position is at n-1, which is kept for the "main"
+ * vector.</li>
+ * <li>A new overflow vector is created and starts empty, with the last write
+ * position at -1.</li>
+ * <li>Once created, b is immediately written to the overflow vector, advancing
+ * the last write position to 0.</li>
+ * <li>Harvesting, and starting the next for column b works the same as column
+ * a.</li>
+ * </ul>
+ * </dd>
+ * <dt>c</dt>
+ * <dd>Column c has values for all rows.
+ * <ul>
+ * <li>The value for row n is written after overflow.</li>
+ * <li>At overflow, the last write position is at n-1.</li>
+ * <li>At overflow, a new lookahead vector is created with the last write
+ * position at -1.</li>
+ * <li>The value of c is written to the lookahead vector, advancing the last
+ * write position to -1.</li>
+ * <li>Harvesting, and starting the next for column c works the same as column
+ * a.</li>
+ * </ul>
+ * </dd>
+ * <dt>d</dt>
+ * <dd>Column d writes values to the last two rows before overflow, but not to
+ * the overflow row.
+ * <ul>
+ * <li>The last write position for the main batch is at n-1.</li>
+ * <li>The last write position in the lookahead batch remains at -1.</li>
+ * <li>Harvesting for column d requires filling an empty value for row n-1.</li>
+ * <li>When starting the next batch, the last write position must be set to -1,
+ * indicating no data yet written.</li>
+ * </ul>
+ * </dd>
+ * <dt>f</dt>
+ * <dd>Column f has no data in the last position of the main batch, and no data
+ * in the overflow row.
+ * <ul>
+ * <li>The last write position is at n-2.</li>
+ * <li>An empty value must be written into position n-1 during harvest.</li>
+ * <li>On start of the next batch, the last write position starts at -1.</li>
+ * </ul>
+ * </dd>
+ * <dt>g</dt>
+ * <dd>Column g is added after overflow, and has a value written to the overflow
+ * row.
+ * <ul>
+ * <li>On harvest, column g is simply skipped.</li>
+ * <li>On start of the next row, the last write position can be left unchanged
+ * since no "exchange" was done.</li>
+ * </ul>
+ * </dd>
+ * <dt>h</dt>
+ * <dd>Column h is added after overflow, but does not have data written to it
+ * during the overflow row. Similar to column g, but the last write position
+ * starts at -1 for the next batch.</dd>
+ * </dl>
+ *
+ * <h4>General Rules</h4>
+ *
+ * The above can be summarized into a smaller set of rules:
+ * <p>
+ * At the time of overflow on row n:
+ * <ul>
+ * <li>Create or clear the lookahead vector.</li>
+ * <li>Copy (last write position - n  + 1) values from row n in the old vector to 0
+ * in the new one. If the copy count is negative, copy nothing. (A negative
+ * copy count means that the last write position is behind the current
+ * row position. Should not occur after back-filling.)</li>
+ * <li>Save the last write position from the old vector, clamped at n.
+ * (That is, if the last write position is before n, keep it. If at
+ * n+1, set it back to n.)</li>
+ * <li>Set the last write position of the overflow vector to (original last
+ * write position - n), clamped at -1. That is, if the original last write
+ * position was before n, the new one is -1. If the original last write
+ * position is after n, shift it down by n places.</li>
+ * <li>Swap buffers from the main vectors and the overflow vectors. This sets
+ * aside the main values, and allows writing to continue using the overflow
+ * buffers.</li>
+ * </ul>
+ * <p>
+ * As the overflow write proceeds:
+ * <ul>
+ * <li>For existing columns, write as normal. The last write position moves from
+ * -1 to 0.</li>
+ * <li>Columns not written leave the last write position at -1.</li>
+ * <li>If a new column appears, set its last write position to -1. If it is then
+ * written, proceed as in the first point above.</li>
+ * </ul>
+ * <p>
+ * At harvest time:
+ * <ul>
+ * <li>For every writer, save the last write position.</li>
+ * <li>Swap the overflow and main buffers to put the main batch back into the
+ * main vectors.</li>
+ * <li>Reset the last write position for all writers to the values saved at
+ * overflow time above.</li>
+ * <li>Finish the batch for the main vectors as normal. No special handling
+ * needed.</li>
+ * </ul>
+ * <p>
+ * When starting the next batch:
+ * <ul>
+ * <li>Swap buffers again, putting the overflow row back into the main vectors.
+ * (At this point, the harvested vectors should all have zero buffers.)</li>
+ * <li>Restore the last write position saved during harvest.</li>
+ * </ul>
+ * <h4>Constraints</h4>
+ * A number of constraints are worth pointing out:
+ * <ul>
+ * <li>Writers are bound to vectors, so we can't easily swap vectors during
+ * overflow.</li>
+ * <li>The project operator to which this operator feeds data also binds to
+ * vectors, so the same set of vectors must be presented on every batch.</li>
+ * <li>The client binds to writers, so we cannot swap writers between main and
+ * overflow batches.</li>
+ * <li>Therefore, the unit of swapping is the buffer that backs the vectors.
+ * </li>
+ * <li>Swapping is not copying; it is only exchanging pointers.</li>
+ * <li>The only copying in this entire process occurs when moving previously-
+ * written values in the overflow row to the new vector at the time of
+ * overflow.</li>
+ * </ul>
+ *
+ * <h4>Arrays</h4>
+ *
+ * The above covers the case of scalar, top-level columns. The extension to
+ * scalar maps is straightforward: at run time, the members of maps are just
+ * simple scalar vectors that reside in a map name space, but the structure
+ * of map fields is the same as for top-level fields. (Think of map fields
+ * as being "flattened" into the top-level tuple.)
+ * <p>
+ * Arrays are a different matter: each row can have many values associated
+ * with it. Consider an array of scalars. We have:
+ * <pre><code>
+ *    Row 0   Row 1     Row 2
+ *    0 1 2   3 4 5     6 7 8
+ * [ [a b c] [d e f] | [g h i] ]
+ * </code></pre>
+ * Here, the letters indicate values. The brackets show the overall vector
+ * (outer brackets) and individual rows (inner brackets). The vertical line
+ * shows where overflow occurred. The same rules as discussed earier still
+ * apply, but we must consider both the row indexes and the array indexes.
+ * <ul>
+ * <li>Overflow occurs at the row level. Here row 2 overflowed and must
+ * be moved to the look-ahead vector.</li>
+ * <li>Value movement occurs at the value level. Here, values 6, 7 and 8
+ * must be move to the look-ahead vector.</li>
+ * </ul>
+ * The result, after overflow, is:
+ * <pre><code>
+ *    Row 0   Row 1       Row 0
+ *    0 1 2   3 4 5       0 1 2
+ * [ [a b c] [d e f] ] [ [g h i] ]
+ * </code></pre>
+ * Further, we must consider lists: a column may consist of a list of
+ * arrays. Or, a column may consist of an array of maps, one of which is
+ * a list of arrays. So, the above reasoning must apply recursively down
+ * the value tree.
+ * <p>
+ * As it turns out, there is a simple recursive algorithm, which is a
+ * simple extension of the reasoning for the top-level scalar case, that can
+ * handle arrays:
+ * <ul>
+ * <li>Start with the row index of the overflow row.</li>
+ * <li>If column c, say, is an array, obtain the index of the first value for
+ * the overflow row.</li>
+ * <li>If c is a list, or a repeated map, then repeat the above, for each
+ * member of c (a single column for a list, a set of columns for a map), but
+ * replace the row index with the index of the first element.</li>
+ * </ul>
+ * The result will be a walk of the value tree in which the overflow index
+ * starts as an index relative to the result set (a row index), and is
+ * recursively replaced with an array offset for each level of the array.
+ *
+ * <h4>Resynching Writers after Overflow</h4>
+ *
+ * When an overflow occurs, our focus is starts with the single top-level row
+ * that will not fit into the current batch. We move this row to the look-ahead
+ * vectors. Doing so is quite simple when each row is a simple tuple. As
+ * described above, the work is quite a bit more complex when the structure
+ * is a JSON-like tree flattened into vectors.
+ * <p>
+ * Consider the writers. Each writer corresponds to a single vector. Writers
+ * are grouped into logical tree nodes. Those in the root node write to
+ * (single, scalar) columns that are either top-level columns, or nested
+ * some level down in single-value (not array) tuples. Another tree level
+ * occurs in an array: the elements of the array use a different
+ * (faster-changing) index than the top (row-level) writers. Different arrays
+ * have different indexes: a row may have, say, four elements in array A,
+ * but 20 elements in array B.
+ * <p>
+ * Further, arrays can be singular (a repeated int, say) or for an entire
+ * tuple (a repeated map.) And, since Drill supports the full JSON model, in
+ * the most general case, there is a tree of array indexes that can be nested
+ * to an arbitrary level. (A row can have an array of maps which contains a
+ * column that is, itself, a list of repeated maps, a field of which is an
+ * array of ints.)
+ * <p>
+ * Writers handle this index tree via a tree of {@link ColumnWriterIndex}
+ * objects, often specialized for various tasks.
+ * <p>
+ * Now we can get to the key concept in this section: how we update those indexes
+ * after an overflow. The top-level index reverts to zero. (We start writing
+ * the 0th row in the new look-ahead batch.) But, nested indexes (those for arrays)
+ * will start at some other position depending on the number elements already
+ * written in an overflow row. The number of such elements is determined by a
+ * top-down traversal of the tree (to determine the start offset of each array
+ * for the row.) Resetting the writer indexes is a bottom-up process: based on
+ * the number of elements in that array, the writer index is reset to match.
+ * <p>
+ * This flow is the opposite of the "normal" case in which a new batch is started
+ * top-down, with each index being reset to zero.
+ *
+ * <h4>The Need for a Uniform Structure</h4>
+ *
+ * Drill has vastly different implementations and interfaces for:
+ * <ul>
+ * <li>Result sets (as a {@link VectorContainer}),</li>
+ * <li>Arrays (as a generated repeated vector),</li>
+ * <li>Lists (as a {@link ListVector}),</li>
+ * <li>Repeated lists (as a {@link RepeatedList vector}, and</li>
+ * <li>Repeated maps ({@link RepeatedMapVector}.</li>
+ * </ul>
+ * If we were to work directly with the above abstractions the code would be
+ * vastly complex. Instead, we abstract out the common structure into the
+ * {@link TupleMode} abstraction. In particular, we use the
+ * single tuple model which works with a single batch. This model provides a
+ * simple, uniform interface to work with columns and tuples (rows, maps),
+ * and a simple way to work with arrays. This interface reduces the above
+ * array algorithm to a simple set of recursive method calls.
+ */
+
+package org.apache.drill.exec.physical.rowSet.impl;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
new file mode 100644
index 0000000..40da4ec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
@@ -0,0 +1,117 @@
+/*
+ * 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.physical.rowSet.model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.TupleSchema.AbstractColumnMetadata;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Base implementation for a tuple model which is common to the "single"
+ * and "hyper" cases. Deals primarily with the structure of the model,
+ * which is common between the two physical implementations.
+ */
+
+public abstract class BaseTupleModel implements TupleModel {
+
+  public static abstract class BaseColumnModel implements ColumnModel {
+
+    /**
+     * Extended schema associated with a column.
+     */
+
+    protected final ColumnMetadata schema;
+
+    public BaseColumnModel(ColumnMetadata schema) {
+      this.schema = schema;
+    }
+
+    @Override
+    public ColumnMetadata schema() { return schema; }
+
+    @Override
+    public TupleModel mapModel() { return null; }
+  }
+
+  /**
+   * Columns within the tuple. Columns may, themselves, be represented
+   * as tuples.
+   */
+
+  protected final List<ColumnModel> columns;
+
+  /**
+   * Descriptive schema associated with the columns above. Unlike a
+   * {@link VectorContainer}, this abstraction keeps the schema in sync
+   * with vectors as columns are added.
+   */
+
+  protected final TupleSchema schema;
+
+  public BaseTupleModel() {
+
+    // Schema starts empty and is built as columns are added.
+    // This ensures that the schema stays in sync with the
+    // backing vectors.
+
+    schema = new TupleSchema();
+    columns = new ArrayList<>();
+  }
+
+  public BaseTupleModel(TupleSchema schema, List<ColumnModel> columns) {
+    this.schema = schema;
+    this.columns = columns;
+    assert schema.size() == columns.size();
+  }
+
+  @Override
+  public TupleMetadata schema() { return schema; }
+
+  @Override
+  public int size() { return schema.size(); }
+
+  @Override
+  public ColumnModel column(int index) {
+    return columns.get(index);
+  }
+
+  @Override
+  public ColumnModel column(String name) {
+    return column(schema.index(name));
+  }
+
+  /**
+   * Perform the work of keeping the list of columns and schema in-sync
+   * as columns are added. This is protected because derived classes
+   * must add logic to keep the new column in sync with the underlying
+   * container or map vector.
+   *
+   * @param column column implementation to add
+   */
+
+  protected void addBaseColumn(BaseColumnModel column) {
+    schema.add((AbstractColumnMetadata) column.schema());
+    columns.add(column);
+    assert columns.size() == schema.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ContainerVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ContainerVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ContainerVisitor.java
new file mode 100644
index 0000000..28c8c59
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ContainerVisitor.java
@@ -0,0 +1,115 @@
+/*
+ * 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.physical.rowSet.model;
+
+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.record.MaterializedField;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
+import org.apache.drill.exec.vector.complex.ListVector;
+import org.apache.drill.exec.vector.complex.RepeatedListVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+
+public class ContainerVisitor<R, A> {
+
+  public R apply(VectorContainer container, A arg) {
+    return visitContainer(container, arg);
+  }
+
+  private R visitContainer(VectorContainer container, A arg) {
+    return visitChildren(container, arg);
+  }
+
+  public R visitChildren(VectorContainer container, A arg) {
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      @SuppressWarnings("resource")
+      ValueVector vector = container.getValueVector(i).getValueVector();
+      apply(vector, arg);
+    }
+    return null;
+  }
+
+  protected R apply(ValueVector vector, A arg) {
+    MaterializedField schema = vector.getField();
+    MajorType majorType = schema.getType();
+    MinorType type = majorType.getMinorType();
+    DataMode mode = majorType.getMode();
+    switch (type) {
+    case MAP:
+      if (mode == DataMode.REPEATED) {
+        return visitRepeatedMap((RepeatedMapVector) vector, arg);
+      } else {
+        return visitMap((AbstractMapVector) vector, arg);
+      }
+    case LIST:
+      if (mode == DataMode.REPEATED) {
+        return visitRepeatedList((RepeatedListVector) vector, arg);
+      } else {
+        return visitList((ListVector) vector, arg);
+      }
+    default:
+      if (mode == DataMode.REPEATED) {
+        return visitRepeatedPrimitive((BaseRepeatedValueVector) vector, arg);
+      } else {
+        return visitPrimitive(vector, arg);
+      }
+    }
+  }
+
+  protected R visitRepeatedMap(RepeatedMapVector vector, A arg) {
+    visitChildren(vector, arg);
+    return visitVector(vector, arg);
+  }
+
+  protected R visitMap(AbstractMapVector vector, A arg) {
+    visitChildren(vector, arg);
+    return visitVector(vector, arg);
+  }
+
+  private R visitChildren(AbstractMapVector vector, A arg) {
+    for (int i = 0; i < vector.size(); i++) {
+      apply(vector.getChildByOrdinal(i), arg);
+    }
+    return null;
+  }
+
+  protected R visitRepeatedList(RepeatedListVector vector, A arg) {
+    return visitVector(vector, arg);
+  }
+
+  protected R visitList(ListVector vector, A arg) {
+    return visitVector(vector, arg);
+  }
+
+  protected R visitRepeatedPrimitive(BaseRepeatedValueVector vector, A arg) {
+    return visitVector(vector, arg);
+  }
+
+  protected R visitPrimitive(ValueVector vector, A arg) {
+    return visitVector(vector, arg);
+  }
+
+  protected R visitVector(ValueVector vector, A arg) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
new file mode 100644
index 0000000..bb5e18e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/MetadataProvider.java
@@ -0,0 +1,93 @@
+/*
+ * 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.physical.rowSet.model;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+
+/**
+ * Interface for retrieving and/or creating metadata given
+ * a vector.
+ */
+
+public interface MetadataProvider {
+  ColumnMetadata metadata(int index, MaterializedField field);
+  MetadataProvider childProvider(ColumnMetadata colMetadata);
+  TupleMetadata tuple();
+
+  public static class VectorDescrip {
+    public final MetadataProvider parent;
+    public final ColumnMetadata metadata;
+
+    public VectorDescrip(MetadataProvider provider, int index,
+        MaterializedField field) {
+      parent = provider;
+      metadata = provider.metadata(index, field);
+    }
+  }
+
+  public static class MetadataCreator implements MetadataProvider {
+
+    private final TupleSchema tuple;
+
+    public MetadataCreator() {
+      tuple = new TupleSchema();
+    }
+
+    public MetadataCreator(TupleSchema tuple) {
+      this.tuple = tuple;
+    }
+
+    @Override
+    public ColumnMetadata metadata(int index, MaterializedField field) {
+      return tuple.addView(field);
+    }
+
+    @Override
+    public MetadataProvider childProvider(ColumnMetadata colMetadata) {
+      return new MetadataCreator((TupleSchema) colMetadata.mapSchema());
+    }
+
+    @Override
+    public TupleMetadata tuple() { return tuple; }
+  }
+
+  public static class MetadataRetrieval implements MetadataProvider {
+
+    private final TupleMetadata tuple;
+
+    public MetadataRetrieval(TupleMetadata schema) {
+      tuple = schema;
+    }
+
+    @Override
+    public ColumnMetadata metadata(int index, MaterializedField field) {
+      return tuple.metadata(index);
+    }
+
+    @Override
+    public MetadataProvider childProvider(ColumnMetadata colMetadata) {
+      return new MetadataRetrieval((TupleSchema) colMetadata.mapSchema());
+    }
+
+    @Override
+    public TupleMetadata tuple() { return tuple; }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ReaderIndex.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ReaderIndex.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ReaderIndex.java
new file mode 100644
index 0000000..c4b0415
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/ReaderIndex.java
@@ -0,0 +1,53 @@
+/*
+ * 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.physical.rowSet.model;
+
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+
+/**
+ * Row set index base class used when indexing rows within a row
+ * set for a row set reader. Keeps track of the current position,
+ * which starts before the first row, meaning that the client
+ * must call <tt>next()</tt> to advance to the first row.
+ */
+
+public abstract class ReaderIndex implements ColumnReaderIndex {
+
+  protected int rowIndex = -1;
+  protected final int rowCount;
+
+  public ReaderIndex(int rowCount) {
+    this.rowCount = rowCount;
+  }
+
+  public int position() { return rowIndex; }
+  public void set(int index) { rowIndex = index; }
+
+  public boolean next() {
+    if (++rowIndex < rowCount ) {
+      return true;
+    } else {
+      rowIndex--;
+      return false;
+    }
+  }
+
+  public int size() { return rowCount; }
+
+  public boolean valid() { return rowIndex < rowCount; }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
new file mode 100644
index 0000000..3db01dd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/SchemaInference.java
@@ -0,0 +1,61 @@
+/*
+ * 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.physical.rowSet.model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.VectorContainer;
+
+/**
+ * Produce a metadata schema from a vector container. Used when given a
+ * record batch without metadata.
+ */
+
+public class SchemaInference {
+
+  public TupleMetadata infer(VectorContainer container) {
+    List<ColumnMetadata> columns = new ArrayList<>();
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      MaterializedField field = container.getValueVector(i).getField();
+      columns.add(inferVector(field));
+    }
+    return TupleSchema.fromColumns(columns);
+  }
+
+  private ColumnMetadata inferVector(MaterializedField field) {
+    if (field.getType().getMinorType() == MinorType.MAP) {
+      return TupleSchema.newMap(field, inferMapSchema(field));
+    } else {
+      return TupleSchema.fromField(field);
+    }
+  }
+
+  private TupleSchema inferMapSchema(MaterializedField field) {
+    List<ColumnMetadata> columns = new ArrayList<>();
+    for (MaterializedField child : field.getChildren()) {
+      columns.add(inferVector(child));
+    }
+    return TupleSchema.fromColumns(columns);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
new file mode 100644
index 0000000..5fcba73
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/TupleModel.java
@@ -0,0 +1,117 @@
+/*
+ * 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.physical.rowSet.model;
+
+import javax.sql.RowSet;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+
+/**
+ * Common interface to access a tuple backed by a vector container or a
+ * map vector. Provides a visitor interface to apply tasks such as vector
+ * allocation, reader or writer creation, and so on. Allows either static
+ * or dynamic vector allocation.
+ * <p>
+ * The terminology used here:
+ * <dl>
+ * <dt>Row set</dt>
+ * <dd>A collection of rows stored as value vectors. Elsewhere in
+ * Drill we call this a "record batch", but that term has been overloaded to
+ * mean the runtime implementation of an operator.</dd>
+ * <dt>Tuple</dt>
+ * <dd>The relational-theory term for a row. Drill maps have a fixed schema.
+ * Impala, Hive and other tools use the term "structure" (or "struct") for
+ * what Drill calls a map. A structure is simply a nested tuple, modeled
+ * here by the same tuple abstraction used for rows.</dd>
+ * <dt>Column</dt>
+ * <dd>A column is represented by a vector (which may have internal
+ * null-flag or offset vectors.) Maps are a kind of column that has an
+ * associated tuple. Because this abstraction models structure, array
+ * columns are grouped with single values: the array-ness is just cardinality.</dd>
+ * <dt>Visitor</dt>
+ * <dd>The visitor abstraction (classic Gang-of-Four pattern) allows adding
+ * functionality without complicating the structure classes. Allows the same
+ * abstraction to be used for the testing {@link RowSet} abstractions and
+ * the scan operator "loader" classes.</dd>
+ * <dt>Metadata</dt>
+ * <dd>Metadata is simply data about data. Here, data about tuples and columns.
+ * The column metadata mostly expands on that available in {@link MaterializedField},
+ * but also adds allocation hints.
+ * </dl>
+ * <p>
+ * This abstraction is the physical dual of a {@link VectorContainer}.
+ * The vectors are "owned" by
+ * the associated container. The structure here simply applies additional
+ * metadata and visitor behavior to allow much easier processing that is
+ * possible with the raw container structure.
+ * <p>
+ * A key value of this abstraction is the extended {@link TupleSchema}
+ * associated with the structure.  Unlike a
+ * {@link VectorContainer}, this abstraction keeps the schema in sync
+ * with vectors as columns are added.
+ * <p>
+ * Some future version may wish to merge the two concepts. That way, metadata
+ * discovered by one operator will be available to another. Complex recursive
+ * functions can be replace by a visitor with the recursion handled inside
+ * implementations of this interface.
+ * <p>
+ * Tuples provide access to columns by both index and name. Both the schema and
+ * model classes follow this convention. Compared with the VectorContainer and
+ * {@link AbstractMapVector} classes, the vector index is a first-class concept:
+ * the column model and schema are guaranteed to reside at the same index relative
+ * to the enclosing tuple. In addition, name access is efficient using a hash
+ * index.
+ * <p>
+ * Visitor classes are defined by the "simple" (single batch) and "hyper"
+ * (multi-batch) implementations to allow vector implementations to work
+ * with the specifics of each type of batch.
+ */
+
+public interface TupleModel {
+
+  /**
+   * Common interface to access a column vector, its metadata, and its
+   * tuple definition (for maps.) Provides a visitor interface for common
+   * vector tasks.
+   */
+
+  public interface ColumnModel {
+    ColumnMetadata schema();
+    TupleModel mapModel();
+  }
+
+  /**
+   * Tuple-model interface for the top-level row (tuple) structure.
+   * Provides access to the {@link VectorContainer} representation of the
+   * row set (record batch.)
+   */
+
+  public interface RowSetModel extends TupleModel {
+    VectorContainer container();
+  }
+
+  TupleMetadata schema();
+  int size();
+  ColumnModel column(int index);
+  ColumnModel column(String name);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/BaseReaderBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/BaseReaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/BaseReaderBuilder.java
new file mode 100644
index 0000000..ee856be
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/BaseReaderBuilder.java
@@ -0,0 +1,149 @@
+/*
+ * 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.physical.rowSet.model.hyper;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.VectorDescrip;
+import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
+import org.apache.drill.exec.record.HyperVectorWrapper;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.impl.AccessorUtilities;
+import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
+import org.apache.drill.exec.vector.accessor.reader.ColumnReaderFactory;
+import org.apache.drill.exec.vector.accessor.reader.MapReader;
+import org.apache.drill.exec.vector.accessor.reader.ObjectArrayReader;
+import org.apache.drill.exec.vector.accessor.reader.VectorAccessor;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+
+public abstract class BaseReaderBuilder {
+
+  /**
+   * Read-only row index into the hyper row set with batch and index
+   * values mapping via an SV4.
+   */
+
+  public static class HyperRowIndex extends ReaderIndex {
+
+    private final SelectionVector4 sv4;
+
+    public HyperRowIndex(SelectionVector4 sv4) {
+      super(sv4.getCount());
+      this.sv4 = sv4;
+    }
+
+    @Override
+    public int vectorIndex() {
+      return AccessorUtilities.sv4Index(sv4.get(rowIndex));
+    }
+
+    @Override
+    public int batchIndex( ) {
+      return AccessorUtilities.sv4Batch(sv4.get(rowIndex));
+    }
+  }
+
+  /**
+   * Vector accessor used by the column accessors to obtain the vector for
+   * each column value. That is, position 0 might be batch 4, index 3,
+   * while position 1 might be batch 1, index 7, and so on.
+   */
+
+  public static class HyperVectorAccessor implements VectorAccessor {
+
+    private final ValueVector[] vectors;
+    private ColumnReaderIndex rowIndex;
+
+    public HyperVectorAccessor(VectorWrapper<?> vw) {
+      vectors = vw.getValueVectors();
+    }
+
+    @Override
+    public void bind(ColumnReaderIndex index) {
+      rowIndex = index;
+    }
+
+    @Override
+    public ValueVector vector() {
+      return vectors[rowIndex.batchIndex()];
+    }
+  }
+
+
+  protected AbstractObjectReader[] buildContainerChildren(
+      VectorContainer container, MetadataProvider mdProvider) {
+    List<AbstractObjectReader> readers = new ArrayList<>();
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      VectorWrapper<?> vw = container.getValueVector(i);
+      VectorDescrip descrip = new VectorDescrip(mdProvider, i, vw.getField());
+      readers.add(buildVectorReader(vw, descrip));
+    }
+    return readers.toArray(new AbstractObjectReader[readers.size()]);
+  }
+
+  @SuppressWarnings("unchecked")
+  private AbstractObjectReader buildVectorReader(VectorWrapper<?> vw, VectorDescrip descrip) {
+    MajorType type = vw.getField().getType();
+    if (type.getMinorType() == MinorType.MAP) {
+      if (type.getMode() == DataMode.REPEATED) {
+        return buildMapArrayReader((HyperVectorWrapper<? extends AbstractMapVector>) vw, descrip);
+      } else {
+        return buildMapReader((HyperVectorWrapper<? extends AbstractMapVector>) vw, descrip);
+      }
+    } else {
+      return buildPrimitiveReader(vw, descrip);
+    }
+  }
+
+  private AbstractObjectReader buildMapArrayReader(HyperVectorWrapper<? extends AbstractMapVector> vectors, VectorDescrip descrip) {
+    AbstractObjectReader mapReader = MapReader.build(descrip.metadata, buildMap(vectors, descrip));
+    return ObjectArrayReader.build(new HyperVectorAccessor(vectors), mapReader);
+  }
+
+  private AbstractObjectReader buildMapReader(HyperVectorWrapper<? extends AbstractMapVector> vectors, VectorDescrip descrip) {
+    return MapReader.build(descrip.metadata, buildMap(vectors, descrip));
+  }
+
+  private AbstractObjectReader buildPrimitiveReader(VectorWrapper<?> vw, VectorDescrip descrip) {
+    return ColumnReaderFactory.buildColumnReader(
+        vw.getField().getType(), new HyperVectorAccessor(vw));
+  }
+
+  private List<AbstractObjectReader> buildMap(HyperVectorWrapper<? extends AbstractMapVector> vectors, VectorDescrip descrip) {
+    List<AbstractObjectReader> readers = new ArrayList<>();
+    MetadataProvider provider = descrip.parent.childProvider(descrip.metadata);
+    MaterializedField mapField = vectors.getField();
+    for (int i = 0; i < mapField.getChildren().size(); i++) {
+      HyperVectorWrapper<? extends ValueVector> child = (HyperVectorWrapper<? extends ValueVector>) vectors.getChildWrapper(new int[] {i});
+      VectorDescrip childDescrip = new VectorDescrip(provider, i, child.getField());
+      readers.add(buildVectorReader(child, childDescrip));
+      i++;
+    }
+    return readers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/package-info.java
new file mode 100644
index 0000000..433231e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementation of a row set model for hyper-batches. A hyper batch is
+ * one that contains a list of batches. The batch is logically comprised
+ * of "hyper-vectors" which are the individual vectors from each batch
+ * stacked "end-to-end."
+ * <p>
+ * Hyper batches allow only reading. So, the only services here are to
+ * parse a hyper-container into a row set model, then use that model to
+ * create a matching set of readers.
+ */
+
+package org.apache.drill.exec.physical.rowSet.model.hyper;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/package-info.java
new file mode 100644
index 0000000..6f24d33
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/package-info.java
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+/**
+ * The "row set model" provides a "dual" of the vector structure used to create,
+ * allocate and work with a collection of vectors. The model provides an enhanced
+ * "metadata" schema, given by {@link TupleMetadata} and {@link ColumnMetadata},
+ * with allocation hints that goes beyond the {@link MaterializedField}
+ * used by value vectors.
+ * <p>
+ * In an ideal world, this structure would not be necessary; the vectors could, by
+ * themselves, provide the needed structure. However, vectors are used in many
+ * places, in many ways, and are hard to evolve. Further, Drill may eventually
+ * choose to move to Arrow, which would not have the structure provided here.
+ * <p>
+ * A set of visitor classes provide the logic to traverse the vector structure,
+ * avoiding the need for multiple implementations of vector traversal. (Traversal
+ * is needed because maps contain vectors, some of which can be maps, resulting
+ * in a tree structure. Further, the API provided by containers (a top-level
+ * tuple) differs from that of a map vector (nested tuple.) This structure provides
+ * a uniform API for both cases.
+ * <p>
+ * Three primary tasks provided by this structure are:
+ * <ol>
+ * <li>Create writers for a set of vectors. Allow incremental write-time
+ * addition of columns, keeping the vectors, columns and metadata all in
+ * sync.</li>
+ * <li>Create readers for a set of vectors. Vectors are immutable once written,
+ * so the reader mechanism does not provide any dynamic schema change
+ * support.</li>
+ * <li>Allocate vectors based on metadata provided. Allocation metadata
+ * includes estimated widths for variable-width columns and estimated
+ * cardinality for array columns.</li>
+ * </ol>
+ * <p>
+ * Drill supports two kinds of batches, reflected by two implementations of
+ * the structure:
+ * <dl>
+ * <dt>Single batch</dt>
+ * <dd>Represents a single batch in which each column is backed by a single
+ * value vector. Single batches support both reading and writing. Writing can
+ * be done only for "new" batches; reading can be done only after writing
+ * is complete. Modeled by the {#link org.apache.drill.exec.physical.rowSet.model.single
+ * single} package.</dd>
+ * <dt>Hyper batch</dt>
+ * <dd>Represents a stacked set of batches in which each column is backed
+ * by a list of columns. A hyper batch is indexed by an "sv4" (four-byte
+ * selection vector.) A hyper batch allows only reading. Modeled by the
+ * {@link org.apache.drill.exec.physical.rowSet.model.hyper hyper} package.</dd>
+ * </dl>
+ */
+
+package org.apache.drill.exec.physical.rowSet.model;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseReaderBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseReaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseReaderBuilder.java
new file mode 100644
index 0000000..80ad19f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseReaderBuilder.java
@@ -0,0 +1,89 @@
+/*
+ * 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.physical.rowSet.model.single;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.VectorDescrip;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
+import org.apache.drill.exec.vector.accessor.reader.ColumnReaderFactory;
+import org.apache.drill.exec.vector.accessor.reader.MapReader;
+import org.apache.drill.exec.vector.accessor.reader.ObjectArrayReader;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+
+public abstract class BaseReaderBuilder {
+
+  protected List<AbstractObjectReader> buildContainerChildren(
+      VectorContainer container, MetadataProvider mdProvider) {
+    List<AbstractObjectReader> writers = new ArrayList<>();
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      @SuppressWarnings("resource")
+      ValueVector vector = container.getValueVector(i).getValueVector();
+      VectorDescrip descrip = new VectorDescrip(mdProvider, i, vector.getField());
+      writers.add(buildVectorReader(vector, descrip));
+    }
+    return writers;
+  }
+
+  private AbstractObjectReader buildVectorReader(ValueVector vector, VectorDescrip descrip) {
+    MajorType type = vector.getField().getType();
+    if (type.getMinorType() == MinorType.MAP) {
+      if (type.getMode() == DataMode.REPEATED) {
+        return buildMapArrayReader((RepeatedMapVector) vector, descrip);
+      } else {
+        return buildMapReader((MapVector) vector, descrip);
+      }
+    } else {
+      return buildPrimitiveReader(vector, descrip);
+    }
+  }
+
+  private AbstractObjectReader buildMapArrayReader(RepeatedMapVector vector, VectorDescrip descrip) {
+    AbstractObjectReader mapReader = MapReader.build(descrip.metadata, buildMap(vector, descrip));
+    return ObjectArrayReader.build(vector, mapReader);
+  }
+
+  private AbstractObjectReader buildMapReader(MapVector vector, VectorDescrip descrip) {
+    return MapReader.build(descrip.metadata, buildMap(vector, descrip));
+  }
+
+  private AbstractObjectReader buildPrimitiveReader(ValueVector vector, VectorDescrip descrip) {
+    return ColumnReaderFactory.buildColumnReader(vector);
+  }
+
+  private List<AbstractObjectReader> buildMap(AbstractMapVector vector, VectorDescrip descrip) {
+    List<AbstractObjectReader> readers = new ArrayList<>();
+    MetadataProvider provider = descrip.parent.childProvider(descrip.metadata);
+    int i = 0;
+    for (ValueVector child : vector) {
+      VectorDescrip childDescrip = new VectorDescrip(provider, i, child.getField());
+      readers.add(buildVectorReader(child, childDescrip));
+      i++;
+    }
+    return readers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseWriterBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseWriterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseWriterBuilder.java
new file mode 100644
index 0000000..bab7b39
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BaseWriterBuilder.java
@@ -0,0 +1,72 @@
+/*
+ * 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.physical.rowSet.model.single;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.VectorDescrip;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+
+/**
+ * Build a set of writers for a single (non-hyper) vector container.
+ */
+
+public abstract class BaseWriterBuilder {
+
+  protected List<AbstractObjectWriter> buildContainerChildren(VectorContainer container, MetadataProvider mdProvider) {
+    List<AbstractObjectWriter> writers = new ArrayList<>();
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      @SuppressWarnings("resource")
+      ValueVector vector = container.getValueVector(i).getValueVector();
+      VectorDescrip descrip = new VectorDescrip(mdProvider, i, vector.getField());
+      writers.add(buildVectorWriter(vector, descrip));
+    }
+    return writers;
+  }
+
+  private AbstractObjectWriter buildVectorWriter(ValueVector vector, VectorDescrip descrip) {
+    MajorType type = vector.getField().getType();
+    if (type.getMinorType() == MinorType.MAP) {
+      return ColumnWriterFactory.buildMapWriter(descrip.metadata,
+          (AbstractMapVector) vector,
+          buildMap((AbstractMapVector) vector, descrip));
+    } else {
+      return ColumnWriterFactory.buildColumnWriter(descrip.metadata, vector);
+    }
+  }
+
+  private List<AbstractObjectWriter> buildMap(AbstractMapVector vector, VectorDescrip descrip) {
+    List<AbstractObjectWriter> writers = new ArrayList<>();
+    MetadataProvider provider = descrip.parent.childProvider(descrip.metadata);
+    int i = 0;
+    for (ValueVector child : vector) {
+      VectorDescrip childDescrip = new VectorDescrip(provider, i, child.getField());
+      writers.add(buildVectorWriter(child, childDescrip));
+      i++;
+    }
+    return writers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
new file mode 100644
index 0000000..30f60b3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/BuildVectorsFromMetadata.java
@@ -0,0 +1,97 @@
+/*
+ * 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.physical.rowSet.model.single;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+
+/**
+ * Build (materialize) as set of vectors based on a provided
+ * metadata schema.
+ */
+
+public class BuildVectorsFromMetadata {
+
+  private final BufferAllocator allocator;
+
+  public BuildVectorsFromMetadata(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public VectorContainer build(TupleMetadata schema) {
+    VectorContainer container = new VectorContainer(allocator);
+    for (int i = 0; i < schema.size(); i++) {
+      container.add(buildVector(schema.metadata(i)));
+    }
+
+    // Build the row set from a matching triple of schema, container and
+    // column models.
+
+    container.buildSchema(SelectionVectorMode.NONE);
+    return container;
+  }
+
+  private ValueVector buildVector(ColumnMetadata metadata) {
+    if (metadata.isMap()) {
+      return buildMap(metadata);
+    } else {
+      return TypeHelper.getNewVector(metadata.schema(), allocator, null);
+    }
+  }
+
+  /**
+   * Build a map column including the members of the map given a map
+   * column schema.
+   *
+   * @param schema the schema of the map column
+   * @return the completed map vector column model
+   */
+
+  private AbstractMapVector buildMap(ColumnMetadata schema) {
+
+    // Creating the map vector will create its contained vectors if we
+    // give it a materialized field with children. So, instead pass a clone
+    // without children so we can add them.
+
+    MaterializedField mapField = schema.schema();
+    MaterializedField emptyClone = MaterializedField.create(mapField.getName(), mapField.getType());
+
+    // Don't get the map vector from the vector cache. Map vectors may
+    // have content that varies from batch to batch. Only the leaf
+    // vectors can be cached.
+
+    AbstractMapVector mapVector = (AbstractMapVector) TypeHelper.getNewVector(emptyClone, allocator, null);
+
+    // Create the contents building the model as we go.
+
+    TupleMetadata mapSchema = schema.mapSchema();
+    for (int i = 0; i < mapSchema.size(); i++) {
+      ColumnMetadata childSchema = mapSchema.metadata(i);
+      mapVector.putChild(childSchema.name(), buildVector(childSchema));
+    }
+
+    return mapVector;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
new file mode 100644
index 0000000..34a6960
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/VectorAllocator.java
@@ -0,0 +1,112 @@
+/*
+ * 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.physical.rowSet.model.single;
+
+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.physical.impl.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataCreator;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+
+/**
+ * Given a vector container, and a metadata schema that matches the container,
+ * walk the schema tree to allocate new vectors according to a given
+ * row count and the size information provided in column metadata.
+ * <p>
+ * @see {@link AllocationHelper} - the class which this one replaces
+ * @see {@link VectorInitializer} - an earlier cut at implementation
+ * based on data from the {@link RecordBatchSizer}
+ */
+
+// TODO: Does not yet handle lists; lists are a simple extension
+// of the array-handling logic below.
+
+public class VectorAllocator {
+
+  private final VectorContainer container;
+
+  public VectorAllocator(VectorContainer container) {
+    this.container = container;
+  }
+
+  public void allocate(int rowCount) {
+    allocate(rowCount, new MetadataCreator());
+  }
+
+  public void allocate(int rowCount, TupleMetadata schema) {
+    allocate(rowCount, new MetadataRetrieval(schema));
+  }
+
+  public void allocate(int rowCount, MetadataProvider mdProvider) {
+    for (int i = 0; i < container.getNumberOfColumns(); i++) {
+      @SuppressWarnings("resource")
+      ValueVector vector = container.getValueVector(i).getValueVector();
+      allocateVector(vector, mdProvider.metadata(i, vector.getField()), rowCount, mdProvider);
+    }
+  }
+
+  private void allocateVector(ValueVector vector, ColumnMetadata metadata, int valueCount, MetadataProvider mdProvider) {
+    MajorType type = vector.getField().getType();
+    assert vector.getField().getName().equals(metadata.name());
+    assert type.getMinorType() == metadata.type();
+    if (type.getMinorType() == MinorType.MAP) {
+      if (type.getMode() == DataMode.REPEATED) {
+        allocateMapArray((RepeatedMapVector) vector, metadata, valueCount, mdProvider);
+      } else {
+        allocateMap((AbstractMapVector) vector, metadata, valueCount, mdProvider);
+      }
+    } else {
+      allocatePrimitive(vector, metadata, valueCount);
+    }
+  }
+
+  private void allocatePrimitive(ValueVector vector,
+      ColumnMetadata metadata, int valueCount) {
+    AllocationHelper.allocatePrecomputedChildCount(vector,
+        valueCount,
+        metadata.expectedWidth(),
+        metadata.expectedElementCount());
+  }
+
+  private void allocateMapArray(RepeatedMapVector vector,
+      ColumnMetadata metadata, int valueCount, MetadataProvider mdProvider) {
+    ((RepeatedMapVector) vector).getOffsetVector().allocateNew(valueCount);
+    int expectedValueCount = valueCount * metadata.expectedElementCount();
+    allocateMap(vector, metadata, expectedValueCount, mdProvider);
+  }
+
+  private void allocateMap(AbstractMapVector vector, ColumnMetadata metadata, int valueCount, MetadataProvider mdProvider) {
+    MetadataProvider mapProvider = mdProvider.childProvider(metadata);
+    TupleMetadata mapSchema = metadata.mapSchema();
+    assert mapSchema != null;
+    int i = 0;
+    for (ValueVector child : vector) {
+      allocateVector(child, mapProvider.metadata(i, child.getField()), valueCount, mapProvider);
+      i++;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/package-info.java
new file mode 100644
index 0000000..6cb6f27
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/single/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/**
+ * This set of classes models the structure of a batch consisting
+ * of single vectors (as contrasted with a hyper batch.) Provides tools
+ * or metdata-based construction, allocation, reading and writing of
+ * the vectors.
+ * <p>
+ * The classes here walk the container/map/vector tree to apply
+ * operations.
+ */
+
+package org.apache.drill.exec.physical.rowSet.model.single;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/package-info.java
new file mode 100644
index 0000000..d92c6b7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/package-info.java
@@ -0,0 +1,193 @@
+/*
+ * 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 second-generation row set (AKA "record batch") writer used
+ * by client code to<ul>
+ * <li>Define the schema of a result set.</li>
+ * <li>Write data into the vectors backing a row set.</li></ul>
+ * <p>
+ * <h4>Terminology</h4>
+ * The code here follows the "row/column" naming convention rather than
+ * the "record/field" convention.
+ * <dl>
+ * <dt>Result set</dt>
+ * <dd>A set of zero or more row sets that hold rows of data.<dd>
+ * <dt>Row set</dt>
+ * <dd>A collection of rows with a common schema. Also called a "row
+ * batch" or "record batch." (But, in Drill, the term "record batch" also
+ * usually means an operator on that set of records. Here, a row set is
+ * just the rows &nash; separate from operations on that data.</dd>
+ * <dt>Row</dt>
+ * <dd>A single row of data, in the usual database sense. Here, a row is
+ * a kind of tuple (see below) allowing both name and index access to
+ * columns.</dd>
+ * <dt>Tuple</dt>
+ * <dd>In relational theory, a row is a tuple: a collection of values
+ * defined by a schema. Tuple values are indexed by position or name.</dd>
+ * <dt>Column</dt>
+ * <dd>A single value within a row or row set. (Generally, the context
+ * makes clear if the term refers to single value or all values for a
+ * column for a row set. Columns are backed by value vectors.</dd>
+ * <dt>Map</dt>
+ * <dd>In Drill, a map is what other systems call a "structure". It is,
+ * in fact, a nested tuple. In a Java or Python map, each map instance has
+ * a distinct set of name/value pairs. But, in Drill, all map instances have
+ * the same schema; hence the so-called "map" is really a tuple. This
+ * implementation exploits that fact and treats the row, and nested maps,
+ * almost identically: both provide columns indexed by name or position.</dd>
+ * <dt>Row Set Mutator</dt>
+ * <dd>An awkward name, but retains the "mutator" name from the previous
+ * generation. The mechanism to build a result set as series of row sets.</dd>
+ * <dt>Tuple Loader</dt>
+ * <dd>Mechanism to build a single tuple (row or map) by providing name
+ * or index access to columns. A better name would b "tuple writer", but
+ * that name is already used elsewhere.</dd>
+ * <dt>Column Loader</dt>
+ * <dd>Mechanism to write values to a single column.<dd>
+ * </dl>
+ * <h4>Building the Schema</h4>
+ * The row set mutator works for two cases: a known schema or a discovered
+ * schema. A known schema occurs in the case, such as JDBC, where the
+ * underlying data source can describe the schema before reading any rows.
+ * In this case, client code can build the schema and pass that schema to
+ * the mutator directly. Alternatively, the client code can build the
+ * schema column-by-column before the first row is read.
+ * <p>
+ * Readers that discover schema can build the schema incrementally: add
+ * a column, load data for that column for one row, discover the next
+ * column, and so on. Almost any kind of column can be added at any time
+ * within the first batch:<ul>
+ * <li>Required columns are "back-filled" with zeros in the active batch,
+ * if that value
+ * makes sense for the column. (Date and Interval columns will throw an
+ * exception if added after the first row as there is no good "zero"
+ * value for that column. Varchar columns are back-filled with blanks.<li>
+ * <li>Optional (nullable) columns can be added at any time; they are
+ * back-filled with nulls in the active batch. In general, if a column is
+ * added after the first row, it should be nullable, not required, unless
+ * the data source has a "missing = blank or zero" policy.</li>
+ * <li>Repeated (array) columns can be added at any time; they are
+ * back-filled with empty entries in the first batch. Arrays can also be
+ * safely added at any time.</li></ul>
+ * Client code must be aware of the semantics of adding columns at various
+ * times.<ul>
+ * <li>Columns added before or during the first row are the trivial case;
+ * this works for all data types and modes.</li>
+ * <li>Required (non-nullable0 structured columns (Date, Period) cannot be
+ * added after the first row (as there is no good zero-fill value.)</li>
+ * <li>Columns added within the first batch appear to the rest of Drill as
+ * if they were added before the first row: the downstream operators see the
+ * same schema from batch to batch.</li>
+ * <li>Columns added <i>after</i> the first batch will trigger a
+ * schema-change event downstream.</li>
+ * <li>The above is true during an "overflow row" (see below.) Once
+ * overflow occurs, columns added later in that overflow row will actually
+ * appear in the next batch, and will trigger a schema change when that
+ * batch is returned. That is, overflow "time shifts" a row addition from
+ * one batch to the next, and so it also time-shifts the column addition.
+ * </li></ul>
+ * Use the {@link LoaderSchema} class to build the schema. The schema class is
+ * part of the {@link TupleLoader} object available from the
+ * {@link #root()} method.
+ * <h4>Using the Schema</h4>
+ * Presents columns using a physical schema. That is, map columns appear
+ * as columns that provide a nested map schema. Presumes that column
+ * access is primarily structural: first get a map, then process all
+ * columns for the map.
+ * <p>
+ * If the input is a flat structure, then the physical schema has a
+ * flattened schema as the degenerate case.
+ * <p>
+ * In both cases, access to columns is by index or by name. If new columns
+ * are added while loading, their index is always at the end of the existing
+ * columns.
+ * <h4>Writing Data to the Batch</h4>
+ * Each batch is delimited by a call to {@link #startBatch()} and a call to
+ * {@link #harvestWithLookAhead()} to obtain the completed batch. Note that readers do not
+ * call these methods; the scan operator does this work.
+ * <p>
+ * Each row is delimited by a call to {@link #startValue()} and a call to
+ * {@link #saveRow()}. <tt>startRow()</tt> performs initialization necessary
+ * for some vectors such as repeated vectors. <tt>saveRow()</tt> moves the
+ * row pointer ahead.
+ * <p>
+ * A reader can easily reject a row by calling <tt>startRow()</tt>, begin
+ * to load a row, but omitting the call to <tt>saveRow()</tt> In this case,
+ * the next call to <tt>startRow()</tt> repositions the row pointer to the
+ * same row, and new data will overwrite the previous data, effectively erasing
+ * the unwanted row. This also works for the last row; omitting the call to
+ * <tt>saveRow()</tt> causes the batch to hold only the rows actually
+ * saved.
+ * <p>
+ * Readers then write to each column. Columns are accessible via index
+ * ({@link TupleLoader#column(int)} or by name
+ * ({@link TupleLoader#column(String)}. Indexed access is much faster.
+ * Column indexes are defined by the order that columns are added. The first
+ * column is column 0, the second is column 1 and so on.
+ * <p>
+ * Each call to the above methods returns the same column writer, allowing the
+ * reader to cache column writers for additional performance.
+ * <p>
+ * All column writers are of the same class; there is no need to cast to a
+ * type corresponding to the vector. Instead, they provide a variety of
+ * <tt>set<i>Type</i></tt> methods, where the type is one of various Java
+ * primitive or structured types. Most vectors provide just one method, but
+ * others (such as VarChar) provide two. The implementation will throw an
+ * exception if the vector does not support a particular type.
+ * <p>
+ * Note that this class uses the term "loader" for row and column writers
+ * since the term "writer" is already used by the legacy record set mutator
+ * and column writers.
+ * <h4>Handling Batch Limits</h4>
+ * The mutator enforces two sets of batch limits:<ol>
+ * <li>The number of rows per batch. The limit defaults to 64K (the Drill
+ * maximum), but can be set lower by the client.</li>
+ * <li>The size of the largest vector, which is capped at 16 MB. (A future
+ * version may allow adjustable caps, or cap the memory of the entire
+ * batch.</li></ol>
+ * Both limits are presented to the client via the {@link #isFull()}
+ * method. After each call to {@link #saveRow()}, the client should call
+ * <tt>isFull()</tt> to determine if the client can add another row. Note
+ * that failing to do this check will cause the next call to
+ * {@link #startBatch()} to throw an exception.
+ * <p>
+ * The limits have subtle differences, however. Row limits are simple: at
+ * the end of the last row, the mutator notices that no more rows are possible,
+ * and so does not allow starting a new row.
+ * <p>
+ * Vector overflow is more complex. A row may consist of columns (a, b, c).
+ * The client may write column a, but then column b might trigger a vector
+ * overflow. (For example, b is a Varchar, and the value for b is larger than
+ * the space left in the vector.) The client cannot stop and rewrite a. Instead,
+ * the client simply continues writing the row. The mutator, internally, moves
+ * this "overflow" row to a new batch. The overflow row becomes the first row
+ * of the next batch rather than the first row of the current batch.
+ * <p>
+ * For this reason, the client can treat the two overflow cases identically,
+ * as described above.
+ * <p>
+ * There are some subtle differences between the two cases that clients may
+ * occasionally may need to expect:<ul>
+ * <li>When a vector overflow occurs, the returned batch will have one
+ * fewer rows than the client might expect if it is simply counting the rows
+ * written.</li>
+ * <li>A new column added to the batch after overflow occurs will appear in
+ * the <i>next</i> batch, triggering a schema change between the current and
+ * next batches.</li></ul>
+ */
+package org.apache.drill.exec.physical.rowSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index 0497cfd..2d01ef4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -96,6 +96,9 @@ public class BatchSchema implements Iterable<MaterializedField> {
     return result;
   }
 
+  // DRILL-5525: the semantics of this method are badly broken.
+  // Caveat emptor.
+
   @Override
   public boolean equals(Object obj) {
     if (this == obj) {
@@ -108,13 +111,24 @@ public class BatchSchema implements Iterable<MaterializedField> {
       return false;
     }
     BatchSchema other = (BatchSchema) obj;
+    if (selectionVectorMode != other.selectionVectorMode) {
+      return false;
+    }
     if (fields == null) {
-      if (other.fields != null) {
-        return false;
-      }
-    } else if (!fields.equals(other.fields)) {
+      return other.fields == null;
+    }
+
+    // Compare names.
+    // (DRILL-5525: actually compares all fields.)
+
+    if (!fields.equals(other.fields)) {
       return false;
     }
+
+    // Compare types
+    // (DRILL-5525: this code is redundant because any differences
+    // will fail above.)
+
     for (int i = 0; i < fields.size(); i++) {
       MajorType t1 = fields.get(i).getType();
       MajorType t2 = other.fields.get(i).getType();
@@ -128,13 +142,25 @@ public class BatchSchema implements Iterable<MaterializedField> {
         }
       }
     }
-    if (selectionVectorMode != other.selectionVectorMode) {
-      return false;
-    }
     return true;
   }
 
+  /**
+   * Compare that two schemas are identical according to the rules defined
+   * in {@ link MaterializedField#isEquivalent(MaterializedField)}. In particular,
+   * this method requires that the fields have a 1:1 ordered correspondence
+   * in the two schemas.
+   *
+   * @param other another non-null batch schema
+   * @return <tt>true</tt> if the two schemas are equivalent according to
+   * the {@link MaterializedField#isEquivalent(MaterializedField)} rules,
+   * false otherwise
+   */
+
   public boolean isEquivalent(BatchSchema other) {
+    if (this == other) {
+      return true;
+    }
     if (fields == null || other.fields == null) {
       return fields == other.fields;
     }
@@ -172,7 +198,7 @@ public class BatchSchema implements Iterable<MaterializedField> {
   }
 
   /**
-   * Merge two schema to produce a new, merged schema. The caller is responsible
+   * Merge two schemas to produce a new, merged schema. The caller is responsible
    * for ensuring that column names are unique. The order of the fields in the
    * new schema is the same as that of this schema, with the other schema's fields
    * appended in the order defined in the other schema.

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index b4ae2d2..acb7a9b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.record;
 
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.vector.ValueVector;
 
 /**
  * A record batch contains a set of field values for a particular range of
@@ -38,7 +39,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 public interface RecordBatch extends VectorAccessible {
 
   /** max batch size, limited by 2-byte length in SV2: 65536 = 2^16 */
-  public static final int MAX_BATCH_SIZE = 65536;
+  public static final int MAX_BATCH_SIZE = ValueVector.MAX_ROW_COUNT;
 
   /**
    * Describes the outcome of incrementing RecordBatch forward by a call to


[06/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/vector/TestVectorLimits.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/vector/TestVectorLimits.java b/exec/java-exec/src/test/java/org/apache/drill/vector/TestVectorLimits.java
deleted file mode 100644
index 84961b1..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/vector/TestVectorLimits.java
+++ /dev/null
@@ -1,487 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.vector;
-
-import static org.junit.Assert.*;
-
-import org.apache.drill.categories.VectorTest;
-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.record.MaterializedField;
-import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableVarCharVector;
-import org.apache.drill.exec.vector.RepeatedIntVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.drill.exec.vector.VectorOverflowException;
-import org.apache.drill.test.DrillTest;
-import org.apache.drill.test.OperatorFixture;
-import org.bouncycastle.util.Arrays;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import io.netty.buffer.DrillBuf;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test the setScalar() methods in the various generated vector
- * classes. Rather than test all 100+ vectors, we sample a few and
- * rely on the fact that code is generated from a common template.
- */
-
-@Category(VectorTest.class)
-public class TestVectorLimits extends DrillTest {
-
-  public static OperatorFixture fixture;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    fixture = OperatorFixture.builder().build();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    fixture.close();
-  }
-
-  /**
-   * Test a vector directly using the vector mutator to ensure
-   * that the <tt>setScalar</tt> method works for the maximum
-   * row count.
-   * <p>
-   * This test is a proxy for all the other fixed types, since all
-   * share the same code template.
-   */
-
-  @Test
-  public void testFixedVector() {
-
-    // Create a non-nullable int vector: a typical fixed-size vector
-
-    @SuppressWarnings("resource")
-    IntVector vector = new IntVector(makeField(MinorType.INT, DataMode.REQUIRED), fixture.allocator() );
-
-    // Sanity test of generated constants.
-
-    assertTrue( IntVector.MAX_SCALAR_COUNT <= ValueVector.MAX_ROW_COUNT );
-    assertEquals( 4, IntVector.VALUE_WIDTH );
-    assertTrue( IntVector.NET_MAX_SCALAR_SIZE <= ValueVector.MAX_BUFFER_SIZE );
-
-    // Allocate a default size, small vector. Forces test of
-    // the auto-grow (setSafe()) aspect of setScalar().
-
-    vector.allocateNew( );
-
-    // Write to the vector until it complains. At that point,
-    // we should have written up to the static fixed value count
-    // (which is computed to stay below the capacity limit.)
-
-    IntVector.Mutator mutator = vector.getMutator();
-    for (int i = 0; i < 2 * ValueVector.MAX_ROW_COUNT; i++) {
-      try {
-        mutator.setScalar(i, i);
-      } catch (VectorOverflowException e) {
-        assertEquals(IntVector.MAX_SCALAR_COUNT, i);
-        break;
-      }
-    }
-
-    // The vector should be below the allocation limit. Since this
-    // is an int vector, in practice the size will be far below
-    // the overall limit (if the limit stays at 16 MB.) But, it should
-    // be at the type-specific limit since we filled up the vector.
-
-    assertEquals(IntVector.NET_MAX_SCALAR_SIZE, vector.getBuffer().getActualMemoryConsumed());
-    vector.close();
-  }
-
-  @Test
-  public void testNullableFixedVector() {
-
-    @SuppressWarnings("resource")
-    NullableIntVector vector = new NullableIntVector(makeField(MinorType.INT, DataMode.OPTIONAL), fixture.allocator() );
-    vector.allocateNew( );
-
-    NullableIntVector.Mutator mutator = vector.getMutator();
-    for (int i = 0; i < 2 * ValueVector.MAX_ROW_COUNT; i++) {
-      try {
-        mutator.setScalar(i, i);
-      } catch (VectorOverflowException e) {
-        assertEquals(IntVector.MAX_SCALAR_COUNT, i);
-        break;
-      }
-    }
-
-    vector.close();
-  }
-
-  /**
-   * Repeated fixed vector. Using an int vector, each column array can hold
-   * 256 / 4 = 64 values. We write only 10. The vector becomes full when we
-   * exceed 64K items.
-   */
-
-  @Test
-  public void testRepeatedFixedVectorCountLimit() {
-
-    @SuppressWarnings("resource")
-    RepeatedIntVector vector = new RepeatedIntVector(makeField(MinorType.INT, DataMode.REPEATED), fixture.allocator() );
-    vector.allocateNew( );
-
-    RepeatedIntVector.Mutator mutator = vector.getMutator();
-    top:
-    for (int i = 0; i < 2 * ValueVector.MAX_ROW_COUNT; i++) {
-      if (! mutator.startNewValueBounded(i)) {
-        assertEquals(ValueVector.MAX_ROW_COUNT, i);
-        // Continue, let's check the addBounded method also
-      }
-      for (int j = 0; j < 10; j++) {
-        try {
-          mutator.addEntry(i, i * 100 + j);
-        } catch (VectorOverflowException e) {
-          assertEquals(ValueVector.MAX_ROW_COUNT, i);
-          mutator.setValueCount(i);
-          break top;
-        }
-      }
-    }
-
-    vector.close();
-  }
-
-  /**
-   * Repeated fixed vector. Using an int vector, each column array can hold
-   * 256 / 4 = 64 values. We write 100. The vector becomes full when we
-   * exceed the 16 MB size limit.
-   */
-
-  @Test
-  public void testRepeatedFixedVectorBufferLimit() {
-
-    @SuppressWarnings("resource")
-    RepeatedIntVector vector = new RepeatedIntVector(makeField(MinorType.INT, DataMode.REPEATED), fixture.allocator() );
-    vector.allocateNew( );
-
-    RepeatedIntVector.Mutator mutator = vector.getMutator();
-    top:
-    for (int i = 0; i < 2 * ValueVector.MAX_ROW_COUNT; i++) {
-      // We'll never hit the value count limit
-      assertTrue(mutator.startNewValueBounded(i));
-      for (int j = 0; j < 100; j++) {
-        try {
-          mutator.addEntry(i, i * 100 + j);
-        } catch (VectorOverflowException e) {
-          // We should have hit the buffer limit before the value limit.
-          assertTrue(i < ValueVector.MAX_ROW_COUNT);
-          mutator.setValueCount(i);
-          break top;
-        }
-      }
-    }
-
-    vector.close();
-  }
-
-  // To be replaced by a test method in a separate commit.
-
-  public static MaterializedField makeField(MinorType dataType, DataMode mode) {
-    MajorType type = MajorType.newBuilder()
-        .setMinorType(dataType)
-        .setMode(mode)
-        .build();
-
-    return MaterializedField.create("foo", type);
-  }
-
-  /**
-   * Baseline test for a variable-width vector using <tt>setSafe</tt> and
-   * loading the vector up to the maximum size. Doing so will cause the vector
-   * to have a buffer that exceeds the maximum size, demonstrating the
-   * need for <tt>setScalar()</tt>.
-   */
-
-  @Test
-  public void variableVectorBaseline() {
-
-    // Create a non-nullable VarChar vector: a typical variable-size vector
-
-    @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField(MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator() );
-    vector.allocateNew( );
-
-    // A 16 MB value can hold 64K values of up to 256 bytes each.
-    // To force a size overflow, write values much larger.
-    // Write the maximum number of values which will silently
-    // allow the vector to grow beyond the critical size of 16 MB.
-    // Doing this in production would lead to memory fragmentation.
-    // So, this is what the setScalar() method assures we don't do.
-
-    byte dummyValue[] = new byte[512];
-    Arrays.fill(dummyValue, (byte) 'X');
-    VarCharVector.Mutator mutator = vector.getMutator();
-    for (int i = 0; i < 2 * ValueVector.MAX_ROW_COUNT; i++) {
-      mutator.setSafe(i, dummyValue, 0, dummyValue.length);
-    }
-
-    // The vector should be above the allocation limit.
-    // This is why code must migrate to the setScalar() call
-    // away from the setSafe() call.
-
-    assertTrue(ValueVector.MAX_BUFFER_SIZE < vector.getBuffer().getActualMemoryConsumed());
-    vector.close();
-  }
-
-  /**
-   * Test a vector directly using the vector mutator to ensure
-   * that the <tt>setScalar</tt> method works for the maximum
-   * vector size.
-   */
-
-  @Test
-  public void testWideVariableVector() {
-
-    @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField(MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator() );
-    vector.allocateNew( );
-
-    // A 16 MB value can hold 64K values of up to 256 bytes each.
-    // To force a size overflow, write values much larger.
-    // Write to the vector until it complains. At that point,
-    // we should have written up to the maximum buffer size.
-
-    byte dummyValue[] = makeVarCharValue(512);
-    VarCharVector.Mutator mutator = vector.getMutator();
-    int count = 0;
-    for ( ; count < 2 * ValueVector.MAX_ROW_COUNT; count++) {
-      try {
-        mutator.setScalar(count, dummyValue, 0, dummyValue.length);
-      } catch (VectorOverflowException e) {
-        break;
-      }
-    }
-
-    // The vector should be at the allocation limit. If it wasn't, we
-    // should have grown it to hold more data. The value count will
-    // be below the maximum.
-
-    mutator.setValueCount(count);
-    assertEquals(ValueVector.MAX_BUFFER_SIZE, vector.getBuffer().getActualMemoryConsumed());
-    assertTrue(count < ValueVector.MAX_ROW_COUNT);
-    vector.close();
-  }
-
-  private byte[] makeVarCharValue(int n) {
-    byte dummyValue[] = new byte[n];
-    Arrays.fill(dummyValue, (byte) 'X');
-    return dummyValue;
-  }
-
-  @Test
-  public void testNullableWideVariableVector() {
-
-    @SuppressWarnings("resource")
-    NullableVarCharVector vector = new NullableVarCharVector(makeField(MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator() );
-    vector.allocateNew( );
-
-    byte dummyValue[] = makeVarCharValue(512);
-    NullableVarCharVector.Mutator mutator = vector.getMutator();
-    int count = 0;
-    for ( ; count < 2 * ValueVector.MAX_ROW_COUNT; count++) {
-      try {
-        mutator.setScalar(count, dummyValue, 0, dummyValue.length);
-      } catch (VectorOverflowException e) {
-        break;
-      }
-    }
-
-    mutator.setValueCount(count);
-    assertEquals(ValueVector.MAX_BUFFER_SIZE, vector.getValuesVector().getBuffer().getActualMemoryConsumed());
-    assertTrue(count < ValueVector.MAX_ROW_COUNT);
-    vector.close();
-  }
-
-  /**
-   * Test a vector directly using the vector mutator to ensure
-   * that the <tt>setScalar</tt> method works for the maximum
-   * value count.
-   */
-
-  @Test
-  public void testNarrowVariableVector() {
-
-    @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField(MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator() );
-    vector.allocateNew( );
-
-    // Write small values that fit into 16 MB. We should stop writing
-    // when we reach the value count limit.
-
-    byte dummyValue[] = makeVarCharValue(254);
-    VarCharVector.Mutator mutator = vector.getMutator();
-    int count = 0;
-    for (; count < 2 * ValueVector.MAX_ROW_COUNT; count++) {
-      try {
-        mutator.setScalar(count, dummyValue, 0, dummyValue.length);
-      } catch (VectorOverflowException e) {
-        break;
-      }
-    }
-
-    // Buffer size should be at or below the maximum, with count
-    // at the maximum.
-
-    mutator.setValueCount(count);
-    assertTrue(vector.getBuffer().getActualMemoryConsumed() <= ValueVector.MAX_BUFFER_SIZE);
-    assertEquals(ValueVector.MAX_ROW_COUNT, count);
-    vector.close();
-  }
-
-  /**
-   * Test a vector directly using the vector mutator to ensure
-   * that the <tt>setScalar</tt> method works for the maximum
-   * value count. Uses a DrillBuf as input.
-   */
-
-  @Test
-  public void testDirectVariableVector() {
-
-    @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField(MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator() );
-    vector.allocateNew( );
-
-    // Repeat the big-value test, but with data coming from a DrillBuf
-    // (direct memory) rather than a heap array.
-
-    @SuppressWarnings("resource")
-    DrillBuf drillBuf = makeVarCharValueDirect(260);
-    VarCharVector.Mutator mutator = vector.getMutator();
-    int count = 0;
-    for (; count < 2 * ValueVector.MAX_ROW_COUNT; count++) {
-      try {
-        mutator.setScalar(count, drillBuf, 0, 260);
-      } catch (VectorOverflowException e) {
-        break;
-      }
-    }
-    drillBuf.close();
-
-    // Again, vector should be at the size limit, count below the
-    // value limit.
-
-    mutator.setValueCount(count);
-    assertEquals(ValueVector.MAX_BUFFER_SIZE, vector.getBuffer().getActualMemoryConsumed());
-    assertTrue(count < ValueVector.MAX_ROW_COUNT);
-    vector.close();
-  }
-
-  private DrillBuf makeVarCharValueDirect(int n) {
-    byte dummyValue[] = makeVarCharValue(n);
-    DrillBuf drillBuf = fixture.allocator().buffer(dummyValue.length);
-    drillBuf.setBytes(0, dummyValue);
-    return drillBuf;
-  }
-
-  @Test
-  public void testDirectNullableVariableVector() {
-
-    @SuppressWarnings("resource")
-    NullableVarCharVector vector = new NullableVarCharVector(makeField(MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator() );
-    vector.allocateNew( );
-
-    @SuppressWarnings("resource")
-    DrillBuf drillBuf = makeVarCharValueDirect(260);
-    NullableVarCharVector.Mutator mutator = vector.getMutator();
-    int count = 0;
-    for (; count < 2 * ValueVector.MAX_ROW_COUNT; count++) {
-      try {
-        mutator.setScalar(count, drillBuf, 0, 260);
-      } catch (VectorOverflowException e) {
-        break;
-      }
-    }
-    drillBuf.close();
-
-    mutator.setValueCount(count);
-    assertEquals(ValueVector.MAX_BUFFER_SIZE, vector.getValuesVector().getBuffer().getActualMemoryConsumed());
-    assertTrue(count < ValueVector.MAX_ROW_COUNT);
-    vector.close();
-  }
-
-  public static void main(String args[]) {
-    try {
-      setUpBeforeClass();
-      new TestVectorLimits().performanceTest();
-      tearDownAfterClass();
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  private void performanceTest() {
-    @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField(MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator() );
-    byte value[] = makeVarCharValue(1);
-    int warmCount = 100;
-    timeSetSafe(vector, value, warmCount);
-    runSetBounded(vector, value, warmCount);
-    int runCount = 1000;
-    timeSetSafe(vector, value, runCount);
-    runSetBounded(vector, value, runCount);
-    timeSetSafe(vector, value, runCount);
-    vector.close();
-  }
-
-  private void timeSetSafe(VarCharVector vector, byte[] value, int iterCount) {
-    long start = System.currentTimeMillis();
-    for (int i = 0; i < iterCount; i++) {
-      vector.clear();
-      vector.allocateNew( );
-
-      VarCharVector.Mutator mutator = vector.getMutator();
-      for (int j = 0; j < ValueVector.MAX_ROW_COUNT; j++) {
-        mutator.setSafe(j, value, 0, value.length);
-      }
-    }
-    long elapsed = System.currentTimeMillis() - start;
-    System.out.println( iterCount + " runs of setSafe: " + elapsed + " ms." );
-  }
-
-  private void runSetBounded(VarCharVector vector, byte[] value, int iterCount) {
-    long start = System.currentTimeMillis();
-    for (int i = 0; i < iterCount; i++) {
-      vector.clear();
-      vector.allocateNew( );
-
-      VarCharVector.Mutator mutator = vector.getMutator();
-      int posn = 0;
-      for (;;) {
-        try {
-          mutator.setScalar(posn++, value, 0, value.length);
-        } catch (VectorOverflowException e) {
-          break;
-        }
-      }
-    }
-    long elapsed = System.currentTimeMillis() - start;
-    System.out.println( iterCount + " runs of setScalar: " + elapsed + " ms." );
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 82910af..9db9c58 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -509,7 +509,7 @@
                   This is likely due to you adding new dependencies to a java-exec and not updating the excludes in this module. This is important as it minimizes the size of the dependency of Drill application users.
 
                   </message>
-                  <maxsize>30000000</maxsize>
+                  <maxsize>32000000</maxsize>
                   <minsize>15000000</minsize>
                   <files>
                    <file>${project.build.directory}/drill-jdbc-all-${project.version}.jar</file>

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
index 9019507..eda189e 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
@@ -52,7 +52,6 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   private final int offset;
   private final BufferLedger ledger;
   private final BufferManager bufManager;
-//  private final ByteBufAllocator alloc;
   private final boolean isEmpty;
   private volatile int length;
   private final HistoricalLog historicalLog = BaseAllocator.DEBUG ?
@@ -72,7 +71,6 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     this.udle = byteBuf;
     this.isEmpty = isEmpty;
     this.bufManager = manager;
-//    this.alloc = alloc;
     this.addr = byteBuf.memoryAddress() + offset;
     this.ledger = ledger;
     this.length = length;
@@ -106,6 +104,8 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     }
   }
 
+  public long addr() { return addr; }
+
   private long addr(int index) {
     return addr + index;
   }
@@ -882,4 +882,70 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     }
   }
 
+  // The "unsafe" methods are for use ONLY by code that does its own
+  // bounds checking. They are called "unsafe" for a reason: they will crash
+  // the JVM if values are addressed out of bounds.
+
+  /**
+   * Write an integer to the buffer at the given byte index, without
+   * bounds checks.
+   *
+   * @param offset byte (not int) offset of the location to write
+   * @param value the value to write
+   */
+
+  public void unsafePutInt(int offset, int value) {
+    PlatformDependent.putInt(addr + offset, value);
+  }
+
+  /**
+   * Write a long to the buffer at the given byte index, without
+   * bounds checks.
+   *
+   * @param index byte (not long) offset of the location to write
+   * @param value the value to write
+   */
+
+  public void unsafePutLong(int index, long value) {
+    PlatformDependent.putLong(addr + index, value);
+  }
+
+  /**
+   * Write a short to the buffer at the given byte index, without
+   * bounds checks.
+   *
+   * @param offset byte (not short) offset of the location to write
+   * @param value the value to write
+   */
+
+  public void unsafePutShort(int offset, short value) {
+    PlatformDependent.putShort(addr + offset, value);
+  }
+
+  /**
+   * Write a byte to the buffer at the given byte index, without
+   * bounds checks.
+   *
+   * @param offset byte offset of the location to write
+   * @param value the value to write
+   */
+
+  public void unsafePutByte(int offset, byte value) {
+    PlatformDependent.putByte(addr + offset, value);
+  }
+
+  /**
+   * Copy a buffer of heap data to the buffer memory.
+   *
+   * @param srce source byte buffer
+   * @param srcOffset offset within the byte buffer of the start of data
+   * @param destOffset byte offset into this buffer to which to write the
+   * data
+   * @param length length of the data, which must be within the
+   * bounds of this buffer
+   */
+
+  public void unsafeCopyMemory(byte[] srce, int srcOffset, int destOffset, int length) {
+    PlatformDependent.copyMemory(srce, srcOffset, addr + destOffset, length);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java b/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
index 5358ca2..1e70216 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.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,14 +32,16 @@ import com.codahale.metrics.MetricFilter;
 import com.codahale.metrics.MetricRegistry;
 
 /**
- * The base allocator that we use for all of Drill's memory management. Returns UnsafeDirectLittleEndian buffers.
+ * The base allocator that we use for all of Drill's memory management. Returns
+ * UnsafeDirectLittleEndian buffers.
  */
+
 public class PooledByteBufAllocatorL {
-  private static final org.slf4j.Logger memoryLogger = org.slf4j.LoggerFactory.getLogger("drill.allocator");
+  private static final org.slf4j.Logger memoryLogger = org.slf4j.LoggerFactory
+      .getLogger("drill.allocator");
 
   private static final int MEMORY_LOGGER_FREQUENCY_SECONDS = 60;
 
-
   public static final String METRIC_PREFIX = "drill.allocator.";
 
   private final MetricRegistry registry;
@@ -54,7 +56,8 @@ public class PooledByteBufAllocatorL {
   public PooledByteBufAllocatorL(MetricRegistry registry) {
     this.registry = registry;
     allocator = new InnerAllocator();
-    empty = new UnsafeDirectLittleEndian(new DuplicatedByteBuf(Unpooled.EMPTY_BUFFER));
+    empty = new UnsafeDirectLittleEndian(
+        new DuplicatedByteBuf(Unpooled.EMPTY_BUFFER));
   }
 
   public UnsafeDirectLittleEndian allocate(int size) {
@@ -63,7 +66,6 @@ public class PooledByteBufAllocatorL {
     } catch (OutOfMemoryError e) {
       throw new OutOfMemoryException("Failure allocating buffer.", e);
     }
-
   }
 
   public int getChunkSize() {
@@ -72,13 +74,13 @@ public class PooledByteBufAllocatorL {
 
   private class InnerAllocator extends PooledByteBufAllocator {
 
-
     private final PoolArena<ByteBuffer>[] directArenas;
     private final MemoryStatusThread statusThread;
     private final Histogram largeBuffersHist;
     private final Histogram normalBuffersHist;
     private final int chunkSize;
 
+    @SuppressWarnings("unchecked")
     public InnerAllocator() {
       super(true);
 
@@ -87,7 +89,9 @@ public class PooledByteBufAllocatorL {
         f.setAccessible(true);
         this.directArenas = (PoolArena<ByteBuffer>[]) f.get(this);
       } catch (Exception e) {
-        throw new RuntimeException("Failure while initializing allocator.  Unable to retrieve direct arenas field.", e);
+        throw new RuntimeException(
+            "Failure while initializing allocator.  Unable to retrieve direct arenas field.",
+            e);
       }
 
       this.chunkSize = directArenas[0].chunkSize;
@@ -130,17 +134,14 @@ public class PooledByteBufAllocatorL {
 
       largeBuffersHist = registry.histogram(METRIC_PREFIX + "huge.hist");
       normalBuffersHist = registry.histogram(METRIC_PREFIX + "normal.hist");
-
     }
 
-
     private synchronized void removeOldMetrics() {
       registry.removeMatching(new MetricFilter() {
         @Override
         public boolean matches(String name, Metric metric) {
           return name.startsWith("drill.allocator.");
         }
-
       });
     }
 
@@ -152,17 +153,21 @@ public class PooledByteBufAllocatorL {
 
         if (initialCapacity > directArena.chunkSize) {
           // This is beyond chunk size so we'll allocate separately.
-          ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity);
+          ByteBuf buf = UnpooledByteBufAllocator.DEFAULT
+              .directBuffer(initialCapacity, maxCapacity);
 
           hugeBufferCount.incrementAndGet();
           hugeBufferSize.addAndGet(buf.capacity());
           largeBuffersHist.update(buf.capacity());
-          // logger.debug("Allocating huge buffer of size {}", initialCapacity, new Exception());
-          return new UnsafeDirectLittleEndian(new LargeBuffer(buf, hugeBufferSize, hugeBufferCount));
+          // logger.debug("Allocating huge buffer of size {}", initialCapacity,
+          // new Exception());
+          return new UnsafeDirectLittleEndian(
+              new LargeBuffer(buf, hugeBufferSize, hugeBufferCount));
 
         } else {
           // within chunk, use arena.
-          ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity);
+          ByteBuf buf = directArena.allocate(cache, initialCapacity,
+              maxCapacity);
           if (!(buf instanceof PooledUnsafeDirectByteBuf)) {
             fail();
           }
@@ -173,10 +178,9 @@ public class PooledByteBufAllocatorL {
             normalBufferCount.incrementAndGet();
           }
 
-          return new UnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf, normalBufferCount,
-              normalBufferSize);
+          return new UnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf,
+              normalBufferCount, normalBufferSize);
         }
-
       } else {
         throw fail();
       }
@@ -184,10 +188,12 @@ public class PooledByteBufAllocatorL {
 
     private UnsupportedOperationException fail() {
       return new UnsupportedOperationException(
-          "Drill requries that the JVM used supports access sun.misc.Unsafe.  This platform didn't provide that functionality.");
+          "Drill requires that the JVM used supports access sun.misc.Unsafe.  This platform doesn't provide that functionality.");
     }
 
-    public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacity) {
+    @Override
+    public UnsafeDirectLittleEndian directBuffer(int initialCapacity,
+        int maxCapacity) {
       if (initialCapacity == 0 && maxCapacity == 0) {
         newDirectBuffer(initialCapacity, maxCapacity);
       }
@@ -197,13 +203,14 @@ public class PooledByteBufAllocatorL {
 
     @Override
     public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
-      throw new UnsupportedOperationException("Drill doesn't support using heap buffers.");
+      throw new UnsupportedOperationException(
+          "Drill doesn't support using heap buffers.");
     }
 
-
     private void validate(int initialCapacity, int maxCapacity) {
       if (initialCapacity < 0) {
-        throw new IllegalArgumentException("initialCapacity: " + initialCapacity + " (expectd: 0+)");
+        throw new IllegalArgumentException(
+            "initialCapacity: " + initialCapacity + " (expected: 0+)");
       }
       if (initialCapacity > maxCapacity) {
         throw new IllegalArgumentException(String.format(
@@ -223,18 +230,18 @@ public class PooledByteBufAllocatorL {
       @Override
       public void run() {
         while (true) {
-          memoryLogger.trace("Memory Usage: \n{}", PooledByteBufAllocatorL.this.toString());
+          memoryLogger.trace("Memory Usage: \n{}",
+              PooledByteBufAllocatorL.this.toString());
           try {
             Thread.sleep(MEMORY_LOGGER_FREQUENCY_SECONDS * 1000);
           } catch (InterruptedException e) {
             return;
           }
-
         }
       }
-
     }
 
+    @Override
     public String toString() {
       StringBuilder buf = new StringBuilder();
       buf.append(directArenas.length);
@@ -257,8 +264,6 @@ public class PooledByteBufAllocatorL {
       buf.append(" bytes.");
       return buf.toString();
     }
-
-
   }
 
   public static final boolean ASSERT_ENABLED;
@@ -268,5 +273,4 @@ public class PooledByteBufAllocatorL {
     assert isAssertEnabled = true;
     ASSERT_ENABLED = isAssertEnabled;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
index 833a604..3b5967f 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationManager.java
@@ -38,23 +38,25 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
- * Manages the relationship between one or more allocators and a particular UDLE. Ensures that one allocator owns the
- * memory that multiple allocators may be referencing. Manages a BufferLedger between each of its associated allocators.
- * This class is also responsible for managing when memory is allocated and returned to the Netty-based
- * PooledByteBufAllocatorL.
+ * Manages the relationship between one or more allocators and a particular
+ * UDLE. Ensures that one allocator owns the memory that multiple allocators may
+ * be referencing. Manages a BufferLedger between each of its associated
+ * allocators. This class is also responsible for managing when memory is
+ * allocated and returned to the Netty-based PooledByteBufAllocatorL.
  *
- * The only reason that this isn't package private is we're forced to put DrillBuf in Netty's package which need access
- * to these objects or methods.
+ * The only reason that this isn't package private is we're forced to put
+ * DrillBuf in Netty's package which need access to these objects or methods.
  *
- * Threading: AllocationManager manages thread-safety internally. Operations within the context of a single BufferLedger
- * are lockless in nature and can be leveraged by multiple threads. Operations that cross the context of two ledgers
- * will acquire a lock on the AllocationManager instance. Important note, there is one AllocationManager per
- * UnsafeDirectLittleEndian buffer allocation. As such, there will be thousands of these in a typical query. The
+ * Threading: AllocationManager manages thread-safety internally. Operations
+ * within the context of a single BufferLedger are lockless in nature and can be
+ * leveraged by multiple threads. Operations that cross the context of two
+ * ledgers will acquire a lock on the AllocationManager instance. Important
+ * note, there is one AllocationManager per UnsafeDirectLittleEndian buffer
+ * allocation. As such, there will be thousands of these in a typical query. The
  * contention of acquiring a lock on AllocationManager should be very low.
- *
  */
+
 public class AllocationManager {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocationManager.class);
 
   private static final AtomicLong MANAGER_ID_GENERATOR = new AtomicLong(0);
   private static final AtomicLong LEDGER_ID_GENERATOR = new AtomicLong(0);
@@ -87,11 +89,13 @@ public class AllocationManager {
   }
 
   /**
-   * Associate the existing underlying buffer with a new allocator. This will increase the reference count to the
-   * provided ledger by 1.
+   * Associate the existing underlying buffer with a new allocator. This will
+   * increase the reference count to the provided ledger by 1.
+   *
    * @param allocator
    *          The target allocator to associate this buffer with.
-   * @return The Ledger (new or existing) that associates the underlying buffer to this new ledger.
+   * @return The Ledger (new or existing) that associates the underlying buffer
+   *         to this new ledger.
    */
   BufferLedger associate(final BaseAllocator allocator) {
     return associate(allocator, true);
@@ -114,7 +118,6 @@ public class AllocationManager {
         }
         return ledger;
       }
-
     }
     try (AutoCloseableLock write = writeLock.open()) {
       // we have to recheck existing ledger since a second reader => writer could be competing with us.
@@ -138,11 +141,16 @@ public class AllocationManager {
     }
   }
 
+  public static int chunkSize() {
+    return INNER_ALLOCATOR.getChunkSize();
+  }
 
   /**
-   * The way that a particular BufferLedger communicates back to the AllocationManager that it now longer needs to hold
-   * a reference to particular piece of memory.
+   * The way that a particular BufferLedger communicates back to the
+   * AllocationManager that it now longer needs to hold a reference to
+   * particular piece of memory.
    */
+
   private class ReleaseListener {
 
     private final BufferAllocator allocator;
@@ -154,6 +162,7 @@ public class AllocationManager {
     /**
      * Can only be called when you already hold the writeLock.
      */
+
     public void release() {
       allocator.assertOpen();
 
@@ -180,16 +189,17 @@ public class AllocationManager {
           throw new IllegalStateException("The final removal of a ledger should be connected to the owning ledger.");
         }
       }
-
-
     }
   }
 
   /**
-   * The reference manager that binds an allocator manager to a particular BaseAllocator. Also responsible for creating
-   * a set of DrillBufs that share a common fate and set of reference counts.
-   * As with AllocationManager, the only reason this is public is due to DrillBuf being in io.netty.buffer package.
+   * The reference manager that binds an allocator manager to a particular
+   * BaseAllocator. Also responsible for creating a set of DrillBufs that share
+   * a common fate and set of reference counts. As with AllocationManager, the
+   * only reason this is public is due to DrillBuf being in io.netty.buffer
+   * package.
    */
+
   public class BufferLedger {
 
     private final IdentityHashMap<DrillBuf, Object> buffers =
@@ -294,7 +304,6 @@ public class AllocationManager {
           }
         }
       }
-
     }
 
     private void inc() {
@@ -302,9 +311,11 @@ public class AllocationManager {
     }
 
     /**
-     * Decrement the ledger's reference count. If the ledger is decremented to zero, this ledger should release its
-     * ownership back to the AllocationManager
+     * Decrement the ledger's reference count. If the ledger is decremented to
+     * zero, this ledger should release its ownership back to the
+     * AllocationManager
      */
+
     public int decrement(int decrement) {
       allocator.assertOpen();
 
@@ -321,29 +332,36 @@ public class AllocationManager {
     }
 
     /**
-     * Returns the ledger associated with a particular BufferAllocator. If the BufferAllocator doesn't currently have a
-     * ledger associated with this AllocationManager, a new one is created. This is placed on BufferLedger rather than
-     * AllocationManager directly because DrillBufs don't have access to AllocationManager and they are the ones
-     * responsible for exposing the ability to associate multiple allocators with a particular piece of underlying
-     * memory. Note that this will increment the reference count of this ledger by one to ensure the ledger isn't
-     * destroyed before use.
+     * Returns the ledger associated with a particular BufferAllocator. If the
+     * BufferAllocator doesn't currently have a ledger associated with this
+     * AllocationManager, a new one is created. This is placed on BufferLedger
+     * rather than AllocationManager directly because DrillBufs don't have
+     * access to AllocationManager and they are the ones responsible for
+     * exposing the ability to associate multiple allocators with a particular
+     * piece of underlying memory. Note that this will increment the reference
+     * count of this ledger by one to ensure the ledger isn't destroyed before
+     * use.
      *
      * @param allocator
      * @return
      */
+
     public BufferLedger getLedgerForAllocator(BufferAllocator allocator) {
       return associate((BaseAllocator) allocator);
     }
 
     /**
-     * Create a new DrillBuf associated with this AllocationManager and memory. Does not impact reference count.
-     * Typically used for slicing.
+     * Create a new DrillBuf associated with this AllocationManager and memory.
+     * Does not impact reference count. Typically used for slicing.
+     *
      * @param offset
      *          The offset in bytes to start this new DrillBuf.
      * @param length
      *          The length in bytes that this DrillBuf will provide access to.
-     * @return A new DrillBuf that shares references with all DrillBufs associated with this BufferLedger
+     * @return A new DrillBuf that shares references with all DrillBufs
+     *         associated with this BufferLedger
      */
+
     public DrillBuf newDrillBuf(int offset, int length) {
       allocator.assertOpen();
       return newDrillBuf(offset, length, null);
@@ -394,6 +412,7 @@ public class AllocationManager {
      *
      * @return Size in bytes
      */
+
     public int getSize() {
       return size;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index f1fbf2f..33b12be 100644
--- a/exec/vector/src/main/codegen/templates/ColumnAccessors.java
+++ b/exec/vector/src/main/codegen/templates/ColumnAccessors.java
@@ -19,145 +19,131 @@
 <@pp.dropOutputFile />
 <@pp.changeOutputFile name="/org/apache/drill/exec/vector/accessor/ColumnAccessors.java" />
 <#include "/@includes/license.ftl" />
-<#macro getType label>
+<#macro getType drillType label>
     @Override
     public ValueType valueType() {
   <#if label == "Int">
       return ValueType.INTEGER;
+  <#elseif drillType == "VarChar" || drillType == "Var16Char">
+      return ValueType.STRING;
   <#else>
       return ValueType.${label?upper_case};
   </#if>
     }
 </#macro>
-<#macro bindReader prefix drillType>
+<#macro bindReader vectorPrefix drillType isArray >
   <#if drillType = "Decimal9" || drillType == "Decimal18">
-    private MaterializedField field;
+    private MajorType type;
   </#if>
-    private ${prefix}${drillType}Vector.Accessor accessor;
+    private ${vectorPrefix}${drillType}Vector.Accessor accessor;
 
     @Override
-    public void bind(RowIndex vectorIndex, ValueVector vector) {
-      bind(vectorIndex);
+    public void bindVector(ValueVector vector) {
   <#if drillType = "Decimal9" || drillType == "Decimal18">
-      field = vector.getField();
+      type = vector.getField().getType();
   </#if>
-      accessor = ((${prefix}${drillType}Vector) vector).getAccessor();
+      accessor = ((${vectorPrefix}${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;
+    public void bindVector(MajorType type, VectorAccessor va) {
+      super.bindVector(type, va);
+      this.type = type;
     }
 
  </#if>
-   private ${prefix}${drillType}Vector.Accessor accessor() {
+    private ${vectorPrefix}${drillType}Vector.Accessor accessor() {
       if (vectorAccessor == null) {
         return accessor;
       } else {
-        return ((${prefix}${drillType}Vector) vectorAccessor.vector()).getAccessor();
+        return ((${vectorPrefix}${drillType}Vector) vectorAccessor.vector()).getAccessor();
       }
     }
 </#macro>
 <#macro get drillType accessorType label isArray>
     @Override
     public ${accessorType} get${label}(<#if isArray>int index</#if>) {
+    <#assign getObject ="getObject"/>
   <#if isArray>
-    <#assign index=", index"/>
-    <#assign getObject="getSingleObject">
+    <#assign indexVar = "index"/>
   <#else>
-    <#assign index=""/>
-    <#assign getObject="getObject">
+    <#assign indexVar = ""/>
   </#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});
+  <#if drillType == "VarChar" || drillType == "Var16Char" || drillType == "VarBinary">
+      return accessor().get(vectorIndex.vectorIndex(${indexVar}));
   <#elseif drillType == "Decimal9" || drillType == "Decimal18">
       return DecimalUtility.getBigDecimalFromPrimitiveTypes(
-                accessor().get(vectorIndex.index()${index}),
-                field.getScale(),
-                field.getPrecision());
+                accessor().get(vectorIndex.vectorIndex(${indexVar})),
+                type.getScale(),
+                type.getPrecision());
   <#elseif accessorType == "BigDecimal" || accessorType == "Period">
-      return accessor().${getObject}(vectorIndex.index()${index});
+      return accessor().${getObject}(vectorIndex.vectorIndex(${indexVar}));
+  <#elseif drillType == "UInt1">
+      return ((int) accessor().get(vectorIndex.vectorIndex(${indexVar}))) & 0xFF;
   <#else>
-      return accessor().get(vectorIndex.index()${index});
+      return accessor().get(vectorIndex.vectorIndex(${indexVar}));
   </#if>
     }
-</#macro>
-<#macro bindWriter prefix drillType>
-  <#if drillType = "Decimal9" || drillType == "Decimal18">
-    private MaterializedField field;
-  </#if>
-    private ${prefix}${drillType}Vector.Mutator mutator;
+  <#if drillType == "VarChar">
 
     @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();
+    public String getString(<#if isArray>int index</#if>) {
+      return new String(getBytes(${indexVar}), Charsets.UTF_8);
     }
-</#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());
+
+    @Override
+    public String getString(<#if isArray>int index</#if>) {
+      return new String(getBytes(${indexVar}), Charsets.UTF_16);
+    }
+  </#if>
+</#macro>
+<#macro build types vectorType accessorType>
+  <#if vectorType == "Repeated">
+    <#assign fnPrefix = "Array" />
+    <#assign classType = "Element" />
   <#else>
-      mutator.${verb}Safe(vectorIndex.index(), <#if cast=="set">(${javaType}) </#if>value);
+    <#assign fnPrefix = vectorType />
+    <#assign classType = "Scalar" />
   </#if>
-    }
+  <#if vectorType == "Required">
+    <#assign vectorPrefix = "" />
+  <#else>
+    <#assign vectorPrefix = vectorType />
+  </#if>
+  public static void define${fnPrefix}${accessorType}s(
+      Class<? extends Base${classType}${accessorType}> ${accessorType?lower_case}s[]) {
+  <#list 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>
+    ${accessorType?lower_case}s[MinorType.${typeEnum}.ordinal()] = ${vectorPrefix}${drillType}Column${accessorType}.class;
+    </#if>
+  </#list>
+  </#list>
+  }
 </#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.MajorType;
 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 org.apache.drill.exec.vector.accessor.reader.BaseScalarReader;
+import org.apache.drill.exec.vector.accessor.reader.BaseElementReader;
+import org.apache.drill.exec.vector.accessor.reader.VectorAccessor;
+import org.apache.drill.exec.vector.accessor.writer.BaseScalarWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractFixedWidthWriter.BaseFixedWidthWriter;
+import org.apache.drill.exec.vector.accessor.writer.BaseVarWidthWriter;
 
 import com.google.common.base.Charsets;
+
 import org.joda.time.Period;
 
 /**
@@ -191,141 +177,176 @@ public class ColumnAccessors {
     <#if accessorType=="BigDecimal">
       <#assign label="Decimal">
     </#if>
+    <#if drillType == "VarChar" || drillType == "Var16Char">
+      <#assign accessorType = "byte[]">
+      <#assign label = "Bytes">
+    </#if>
     <#if ! notyet>
   //------------------------------------------------------------------------
   // ${drillType} readers and writers
 
-  public static class ${drillType}ColumnReader extends AbstractColumnReader {
+  public static class ${drillType}ColumnReader extends BaseScalarReader {
 
-    <@bindReader "" drillType />
+    <@bindReader "" drillType false />
 
-    <@getType label />
+    <@getType drillType label />
 
     <@get drillType accessorType label false/>
   }
 
-  public static class Nullable${drillType}ColumnReader extends AbstractColumnReader {
+  public static class Nullable${drillType}ColumnReader extends BaseScalarReader {
 
-    <@bindReader "Nullable" drillType />
+    <@bindReader "Nullable" drillType false />
 
-    <@getType label />
+    <@getType drillType 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());
+      return accessor().isNull(vectorIndex.vectorIndex());
     }
 
-    <@get drillType accessorType label true/>
+    <@get drillType accessorType label false />
   }
 
-  public static class ${drillType}ColumnWriter extends AbstractColumnWriter {
+  public static class Repeated${drillType}ColumnReader extends BaseElementReader {
 
-    <@bindWriter "" drillType />
+    <@bindReader "" drillType true />
 
-    <@getType label />
+    <@getType drillType label />
 
-    <@set drillType accessorType label false "set" />
+    <@get drillType accessorType label true />
   }
 
-  public static class Nullable${drillType}ColumnWriter extends AbstractColumnWriter {
-
-    <@bindWriter "Nullable" drillType />
+      <#assign varWidth = drillType == "VarChar" || drillType == "Var16Char" || drillType == "VarBinary" />
+      <#if varWidth>
+  public static class ${drillType}ColumnWriter extends BaseVarWidthWriter {
+      <#else>
+  public static class ${drillType}ColumnWriter extends BaseFixedWidthWriter {
+        <#if drillType = "Decimal9" || drillType == "Decimal18" ||
+             drillType == "Decimal28Sparse" || drillType == "Decimal38Sparse">
+    private MajorType type;
+        </#if>
+    private static final int VALUE_WIDTH = ${drillType}Vector.VALUE_WIDTH;
+      </#if>
+    private final ${drillType}Vector vector;
+
+    public ${drillType}ColumnWriter(final ValueVector vector) {
+      <#if varWidth>
+      super(((${drillType}Vector) vector).getOffsetVector());
+      <#else>
+        <#if drillType = "Decimal9" || drillType == "Decimal18" ||
+             drillType == "Decimal28Sparse" || drillType == "Decimal38Sparse">
+      type = vector.getField().getType();
+        </#if>
+      </#if>
+      this.vector = (${drillType}Vector) vector;
+    }
 
-    <@getType label />
+    @Override public BaseDataValueVector vector() { return vector; }
+
+       <#if ! varWidth>
+    @Override public int width() { return VALUE_WIDTH; }
+
+      </#if>
+      <@getType drillType label />
+      <#if accessorType == "byte[]">
+        <#assign args = ", int len">
+      <#else>
+        <#assign args = "">
+      </#if>
+      <#if javaType == "char">
+        <#assign putType = "short" />
+        <#assign doCast = true />
+      <#else>
+        <#assign putType = javaType />
+        <#assign doCast = (cast == "set") />
+      </#if>
+      <#if ! varWidth>
 
+    </#if>
     @Override
-    public void setNull() {
-      mutator.setNull(vectorIndex.index());
+    public final void set${label}(final ${accessorType} value${args}) {
+      <#-- Must compute the write offset first; can't be inline because the
+           writeOffset() function has a side effect of possibly changing the buffer
+           address (bufAddr). -->
+      <#if varWidth>
+      final int offset = writeIndex(len);
+      <#else>
+      final int writeIndex = writeIndex();
+      <#assign putAddr = "writeIndex * VALUE_WIDTH">
+      </#if>
+      <#if varWidth>
+      drillBuf.unsafeCopyMemory(value, 0, offset, len);
+      offsetsWriter.setNextOffset(offset + len);
+      <#elseif drillType == "Decimal9">
+      drillBuf.unsafePutInt(${putAddr},
+          DecimalUtility.getDecimal9FromBigDecimal(value,
+                type.getScale(), type.getPrecision()));
+      <#elseif drillType == "Decimal18">
+      drillBuf.unsafePutLong(${putAddr},
+          DecimalUtility.getDecimal18FromBigDecimal(value,
+                type.getScale(), type.getPrecision()));
+      <#elseif drillType == "Decimal38Sparse">
+      <#-- Hard to optimize this case. Just use the available tools. -->
+      DecimalUtility.getSparseFromBigDecimal(value, vector.getBuffer(), writeIndex * VALUE_WIDTH,
+               type.getScale(), type.getPrecision(), 6);
+      <#elseif drillType == "Decimal28Sparse">
+      <#-- Hard to optimize this case. Just use the available tools. -->
+      DecimalUtility.getSparseFromBigDecimal(value, vector.getBuffer(), writeIndex * VALUE_WIDTH,
+               type.getScale(), type.getPrecision(), 5);
+      <#elseif drillType == "IntervalYear">
+      drillBuf.unsafePutInt(${putAddr},
+                value.getYears() * 12 + value.getMonths());
+      <#elseif drillType == "IntervalDay">
+      final int offset = ${putAddr};
+      drillBuf.unsafePutInt(offset,     value.getDays());
+      drillBuf.unsafePutInt(offset + 4, periodToMillis(value));
+      <#elseif drillType == "Interval">
+      final int offset = ${putAddr};
+      drillBuf.unsafePutInt(offset,     value.getYears() * 12 + value.getMonths());
+      drillBuf.unsafePutInt(offset + 4, value.getDays());
+      drillBuf.unsafePutInt(offset + 8, periodToMillis(value));
+      <#elseif drillType == "Float4">
+      drillBuf.unsafePutInt(${putAddr}, Float.floatToRawIntBits((float) value));
+      <#elseif drillType == "Float8">
+      drillBuf.unsafePutLong(${putAddr}, Double.doubleToRawLongBits(value));
+      <#else>
+      drillBuf.unsafePut${putType?cap_first}(${putAddr}, <#if doCast>(${putType}) </#if>value);
+      </#if>
+      vectorIndex.nextElement();
     }
-
-    <@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;
+    <#if drillType == "VarChar">
+    
+    @Override
+    public final void setString(String value) {
+      final byte bytes[] = value.getBytes(Charsets.UTF_8);
+      setBytes(bytes, bytes.length);
+    }
+    <#elseif drillType == "Var16Char">
+    
+    @Override
+    public final void setString(String value) {
+      final byte bytes[] = value.getBytes(Charsets.UTF_16);
+      setBytes(bytes, bytes.length);
     }
-
-    <@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 int periodToMillis(Period value) {
+    return ((value.getHours() * 60 +
+             value.getMinutes()) * 60 +
+             value.getSeconds()) * 1000 +
+           value.getMillis();
   }
 
-  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>
-  }
+<@build vv.types "Required" "Reader" />
 
-  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>
-  }
+<@build vv.types "Nullable" "Reader" />
+
+<@build vv.types "Repeated" "Reader" />
+
+<@build vv.types "Required" "Writer" />
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 51938a3..1f6a008 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -22,8 +22,8 @@
 <#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
 
 <#if type.major == "Fixed">
-<@pp.changeOutputFile name="/org/apache/drill/exec/vector/${minor.class}Vector.java" />
-<#include "/@includes/license.ftl" />
+  <@pp.changeOutputFile name="/org/apache/drill/exec/vector/${minor.class}Vector.java" />
+  <#include "/@includes/license.ftl" />
 
 package org.apache.drill.exec.vector;
 
@@ -31,15 +31,18 @@ package org.apache.drill.exec.vector;
 import org.apache.drill.exec.util.DecimalUtility;
 
 /**
- * ${minor.class} implements a vector of fixed width values. Elements in the vector are accessed
- * by position, starting from the logical start of the vector. Values should be pushed onto the
- * vector sequentially, but may be accessed randomly.
+ * ${minor.class} implements a vector of fixed width values. Elements in the
+ * vector are accessed by position, starting from the logical start of the
+ * vector. Values should be pushed onto the vector sequentially, but may be
+ * accessed randomly.
  * <ul>
- * <li>The width of each element is {@link #VALUE_WIDTH} (= ${type.width}) byte<#if type.width != 1>s</#if>.</li>
+ * <li>The width of each element is {@link #VALUE_WIDTH} (= ${type.width})
+ * byte<#if type.width != 1>s</#if>.</li>
  * <li>The equivalent Java primitive is '${minor.javaType!type.javaType}'.</li>
  * </ul>
  *
- * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
+ * NB: this class is automatically generated from ${.template_name} and
+ * ValueVectorTypes.tdd using FreeMarker.
  */
 public final class ${minor.class}Vector extends BaseDataValueVector implements FixedWidthVector {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${minor.class}Vector.class);
@@ -151,13 +154,16 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   /**
-   * Allocate a new buffer that supports setting at least the provided number of values. May actually be sized bigger
-   * depending on underlying buffer rounding size. Must be called prior to using the ValueVector.
+   * Allocate a new buffer that supports setting at least the provided number of
+   * values. May actually be sized bigger depending on underlying buffer
+   * rounding size. Must be called prior to using the ValueVector.
    *
-   * Note that the maximum number of values a vector can allocate is Integer.MAX_VALUE / value width.
+   * Note that the maximum number of values a vector can allocate is
+   * Integer.MAX_VALUE / value width.
    *
    * @param valueCount
-   * @throws OutOfMemoryException if it can't allocate the new buffer
+   * @throws OutOfMemoryException
+   *           if it can't allocate the new buffer
    */
   @Override
   public void allocateNew(final int valueCount) {
@@ -211,18 +217,24 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       throw new OversizedAllocationException("Unable to expand the buffer. Max allowed buffer size is reached.");
     }
 
+    reallocRaw((int) newAllocationSize);
+    final int halfNewCapacity = data.capacity() / 2;
+    data.setZero(halfNewCapacity, halfNewCapacity);
+  }
+
+  @Override
+  public DrillBuf reallocRaw(int newAllocationSize) {
     logger.debug("Reallocating vector [{}]. # of bytes: [{}] -> [{}]", field, allocationSizeInBytes, newAllocationSize);
     if (newAllocationSize == 0) {
       throw new IllegalStateException("Attempt to reAlloc a zero-sized vector");
     }
-    final DrillBuf newBuf = allocator.buffer((int)newAllocationSize);
+    final DrillBuf newBuf = allocator.buffer(newAllocationSize);
     newBuf.setBytes(0, data, 0, data.capacity());
-    final int halfNewCapacity = newBuf.capacity() / 2;
-    newBuf.setZero(halfNewCapacity, halfNewCapacity);
     newBuf.writerIndex(data.writerIndex());
     data.release(1);
     data = newBuf;
-    allocationSizeInBytes = (int)newAllocationSize;
+    allocationSizeInBytes = newAllocationSize;
+    return newBuf;
   }
 
   /**
@@ -337,7 +349,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
   @Override
   public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
-    ((${minor.class}Vector) from).data.getBytes(fromIndex * ${type.width}, data, toIndex * ${type.width}, ${type.width});
+    ((${minor.class}Vector) from).data.getBytes(fromIndex * VALUE_WIDTH, data, toIndex * VALUE_WIDTH, VALUE_WIDTH);
   }
 
   public void decrementAllocationMonitor() {
@@ -423,7 +435,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       final String monthString = (Math.abs(months) == 1) ? " month " : " months ";
       final String dayString = (Math.abs(days) == 1) ? " day " : " days ";
 
-
       return(new StringBuilder().
              append(years).append(yearString).
              append(months).append(monthString).
@@ -621,26 +632,31 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   /**
-   * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
-   * vector are accessed by position from the logical start of the vector.  Values should be pushed
-   * onto the vector sequentially, but may be randomly accessed.
+   * ${minor.class}.Mutator implements a mutable vector of fixed width values.
+   * Elements in the vector are accessed by position from the logical start of
+   * the vector. Values should be pushed onto the vector sequentially, but may
+   * be randomly accessed.
    * <ul>
-   * <li>The width of each element is {@link #VALUE_WIDTH} (= ${type.width}) byte(s).</li>
+   * <li>The width of each element is {@link #VALUE_WIDTH} (= ${type.width})
+   * byte(s).</li>
    * <li>The equivalent Java primitive is '${minor.javaType!type.javaType}'</li>
    * </ul>
    *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
+   * NB: this class is automatically generated from ValueVectorTypes.tdd using
+   * FreeMarker.
    */
    public final class Mutator extends BaseDataValueVector.BaseMutator {
 
     private Mutator() {};
 
     /**
-     * Set the element at the given index to the given value.  Note that widths smaller than
-     * 32 bits are handled by the DrillBuf interface.
+     * Set the element at the given index to the given value. Note that widths
+     * smaller than 32 bits are handled by the DrillBuf interface.
      *
-     * @param index   position of the bit to set
-     * @param value   value to set
+     * @param index
+     *          position of the bit to set
+     * @param value
+     *          value to set
      */
 
   <#if (type.width > 8)>
@@ -655,37 +671,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       data.setBytes(index * VALUE_WIDTH, value, 0, VALUE_WIDTH);
     }
 
-    /**
-     * Set the value of a required or nullable vector. Enforces the value
-     * and size limits.
-     * @param index item to write
-     * @param value value to set
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
-     */
-
-    public void setScalar(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
-    /**
-     * Set the value of a repeated vector. Enforces only the size limit.
-     * @param index item to write
-     * @param value value to set
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
-     */
-
-    public void setArrayItem(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
     <#if minor.class == "Interval">
     public void set(int index, int months, int days, int milliseconds) {
       final int offsetIndex = index * VALUE_WIDTH;
@@ -701,20 +686,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, months, days, milliseconds);
     }
 
-    public void setScalar(int index, int months, int days, int milliseconds) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, months, days, milliseconds);
-    }
-
-    public void setArrayItem(int index, int months, int days, int milliseconds) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, months, days, milliseconds);
-    }
-
     protected void set(int index, ${minor.class}Holder holder) {
       set(index, holder.months, holder.days, holder.milliseconds);
     }
@@ -723,14 +694,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.months, holder.days, holder.milliseconds);
     }
 
-    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.months, holder.days, holder.milliseconds);
-    }
-
-    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(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);
     }
@@ -739,14 +702,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.months, holder.days, holder.milliseconds);
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.months, holder.days, holder.milliseconds);
-    }
-
-    public void setArrayItem(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(index, holder.months, holder.days, holder.milliseconds);
-    }
-
     <#elseif minor.class == "IntervalDay">
     public void set(int index, int days, int milliseconds) {
       final int offsetIndex = index * VALUE_WIDTH;
@@ -761,20 +716,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, days, milliseconds);
     }
 
-    public void setScalar(int index, int days, int milliseconds) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, days, milliseconds);
-    }
-
-    public void setArrayItem(int index, int days, int milliseconds) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, days, milliseconds);
-    }
-
     protected void set(int index, ${minor.class}Holder holder) {
       set(index, holder.days, holder.milliseconds);
     }
@@ -783,14 +724,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.days, holder.milliseconds);
     }
 
-    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.days, holder.milliseconds);
-    }
-
-    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(index, holder.days, holder.milliseconds);
-    }
-
     protected void set(int index, Nullable${minor.class}Holder holder) {
       set(index, holder.days, holder.milliseconds);
     }
@@ -799,14 +732,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.days, holder.milliseconds);
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.days, holder.milliseconds);
-    }
-
-    public void setArrayItem(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(index, holder.days, holder.milliseconds);
-    }
-
     <#elseif minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense">
     public void setSafe(int index, int start, DrillBuf buffer) {
       while(index >= getValueCapacity()) {
@@ -815,20 +740,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, start, buffer);
     }
 
-    public void setScalar(int index, int start, DrillBuf buffer) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, start, buffer);
-    }
-
-    public void setArrayItem(int index, int start, DrillBuf buffer) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, start, buffer);
-    }
-
     public void set(int index, ${minor.class}Holder holder) {
       set(index, holder.start, holder.buffer);
     }
@@ -837,14 +748,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.start, holder.buffer);
     }
 
-    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.start, holder.buffer);
-    }
-
-    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(index, holder.start, holder.buffer);
-    }
-
     void set(int index, Nullable${minor.class}Holder holder) {
       set(index, holder.start, holder.buffer);
     }
@@ -853,14 +756,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       setSafe(index, holder.start, holder.buffer);
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setScalar(index, holder.start, holder.buffer);
-    }
-
-    public void setArrayItem(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      setArrayItem(index, holder.start, holder.buffer);
-    }
-
       <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
     public void set(int index, BigDecimal value) {
       DecimalUtility.getSparseFromBigDecimal(value, data, index * VALUE_WIDTH,
@@ -874,20 +769,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, value);
     }
 
-    public void setScalar(int index, BigDecimal value) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
-    public void setArrayItem(int index, BigDecimal value) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
       </#if>
     public void set(int index, int start, DrillBuf buffer){
       data.setBytes(index * VALUE_WIDTH, buffer, start, VALUE_WIDTH);
@@ -912,42 +793,18 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       data.set${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH, value);
     }
 
-    public void setSafe(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      while(index >= getValueCapacity()) {
-        reAlloc();
-      }
-      set(index, value);
-    }
-
     /**
-     * Set the value of a required or nullable vector. Enforces the value
-     * and size limits.
+     * Set the value of a required or nullable vector. Grows the vector as needed.
+     * Does not enforce size limits; scalar fixed-width types can never overflow
+     * a vector.
      * @param index item to write
-     * @param value value to set
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
      */
 
-    public void setScalar(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, value);
-    }
-
-    /**
-     * Set the value of a repeated vector. Enforces only the size limit.
-     * @param index item to write
-     * @param value value to set
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
-     */
-
-    public void setArrayItem(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
+    public void setSafe(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      while(index >= getValueCapacity()) {
+        reAlloc();
       }
-      setSafe(index, value);
+      set(index, value);
     }
 
     protected void set(int index, ${minor.class}Holder holder) {
@@ -961,20 +818,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, holder);
     }
 
-    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
-    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
     protected void set(int index, Nullable${minor.class}Holder holder) {
       data.set${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH, holder.value);
     }
@@ -986,20 +829,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, holder);
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_SCALAR_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
-    public void setArrayItem(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
-      if (index >= MAX_VALUE_COUNT) {
-        throw new VectorOverflowException();
-      }
-      setSafe(index, holder);
-    }
-
     @Override
     public void generateTestData(int size) {
       setValueCount(size);
@@ -1028,30 +857,6 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
   </#if> <#-- type.width -->
-    /**
-     * Backfill missing offsets from the given last written position to the
-     * given current write position. Used by the "new" size-safe column
-     * writers to allow skipping values. The <tt>set()</tt> and <tt>setSafe()</tt>
-     * <b>do not</b> fill empties. See DRILL-5529 and DRILL-5530.
-     * @param lastWrite the position of the last valid write: the offset
-     * to be copied forward
-     * @param index the current write position filling occurs up to,
-     * but not including, this position
-     * @throws VectorOverflowException if the item was written, false if the index would
-     * overfill the vector
-     */
-
-    public void fillEmptiesBounded(int lastWrite, int index)
-            throws VectorOverflowException {
-  <#if type.width <= 8>
-      for (int i = lastWrite + 1; i <= index; i++) {
-        setSafe(i, <#if (type.width >= 4)>(${minor.javaType!type.javaType})</#if> 0);
-      }
-  <#else>
-      throw new UnsupportedOperationException("Cannot zero-fill ${minor.class} vectors.");
-  </#if>
-    }
-
     @Override
     public void setValueCount(int valueCount) {
       final int currentValueCapacity = getValueCapacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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 4f3eb17..fdb0200 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -47,7 +47,7 @@ package org.apache.drill.exec.vector;
  *
  * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
  */
-@SuppressWarnings("unused")
+
 public final class ${className} extends BaseDataValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector, NullableVector {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class);
 
@@ -180,6 +180,11 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   @Override
+  public DrillBuf reallocRaw(int newAllocationSize) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public void collectLedgers(Set<BufferLedger> ledgers) {
     bits.collectLedgers(ledgers);
     values.collectLedgers(ledgers);
@@ -304,7 +309,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   private class TransferImpl implements TransferPair {
-    Nullable${minor.class}Vector to;
+    private final Nullable${minor.class}Vector to;
 
     public TransferImpl(MaterializedField field, BufferAllocator allocator){
       to = new Nullable${minor.class}Vector(field, allocator);
@@ -336,12 +341,12 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   @Override
-  public Accessor getAccessor(){
+  public Accessor getAccessor() {
     return accessor;
   }
 
   @Override
-  public Mutator getMutator(){
+  public Mutator getMutator() {
     return mutator;
   }
 
@@ -548,16 +553,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       lastSet = index;
     }
 
-    public void setScalar(int index, byte[] value, int start, int length) throws VectorOverflowException {
-      if (index > lastSet + 1) {
-        fillEmpties(index); // Filling empties cannot overflow the vector
-      }
-      values.getMutator().setScalar(index, value, start, length);
-      bits.getMutator().setSafe(index, 1);
-      setCount++;
-      lastSet = index;
-    }
-
     public void setSafe(int index, ByteBuffer value, int start, int length) {
       if (index > lastSet + 1) {
         fillEmpties(index);
@@ -569,17 +564,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       lastSet = index;
     }
 
-    public void setScalar(int index, DrillBuf value, int start, int length) throws VectorOverflowException {
-      if (index > lastSet + 1) {
-        fillEmpties(index); // Filling empties cannot overflow the vector
-      }
-
-      values.getMutator().setScalar(index, value, start, length);
-      bits.getMutator().setSafe(index, 1);
-      setCount++;
-      lastSet = index;
-    }
-
     </#if>
     public void setNull(int index) {
       bits.getMutator().setSafe(index, 0);
@@ -593,10 +577,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       values.getMutator().set(index, holder);
     }
 
-    public void setNullBounded(int index) throws VectorOverflowException {
-      bits.getMutator().setScalar(index, 0);
-    }
-
     public void set(int index, Nullable${minor.class}Holder holder) {
       final ${valuesName}.Mutator valuesMutator = values.getMutator();
       <#if type.major == "VarLen">
@@ -648,18 +628,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       values.getMutator().setSafe(index<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
       setCount++;
       <#if type.major == "VarLen">lastSet = index;</#if>
-   }
-
-    public void setScalar(int index, int isSet<#list fields as field><#if field.include!true >, ${field.type} ${field.name}Field</#if></#list> ) throws VectorOverflowException {
-      <#if type.major == "VarLen">
-      if (index > lastSet + 1) {
-        fillEmpties(index);
-      }
-      </#if>
-      values.getMutator().setScalar(index<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
-      bits.getMutator().setSafe(index, isSet);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
     public void setSafe(int index, Nullable${minor.class}Holder value) {
@@ -674,18 +642,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
-    public void setScalar(int index, Nullable${minor.class}Holder value) throws VectorOverflowException {
-      <#if type.major == "VarLen">
-      if (index > lastSet + 1) {
-        fillEmpties(index);
-      }
-      </#if>
-      values.getMutator().setScalar(index, value);
-      bits.getMutator().setSafe(index, value.isSet);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-
     public void setSafe(int index, ${minor.class}Holder value) {
       <#if type.major == "VarLen">
       if (index > lastSet + 1) {
@@ -698,18 +654,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
-    public void setScalar(int index, ${minor.class}Holder value) throws VectorOverflowException {
-      <#if type.major == "VarLen">
-      if (index > lastSet + 1) {
-        fillEmpties(index);
-      }
-      </#if>
-      values.getMutator().setScalar(index, value);
-      bits.getMutator().setSafe(index, 1);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-
     <#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">
@@ -722,17 +666,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       setCount++;
     }
 
-    public void setScalar(int index, ${minor.javaType!type.javaType} value) throws VectorOverflowException {
-      <#if type.major == "VarLen">
-      if (index > lastSet + 1) {
-        fillEmpties(index);
-      }
-      </#if>
-      values.getMutator().setScalar(index, value);
-      bits.getMutator().setSafe(index, 1);
-      setCount++;
-    }
-
     </#if>
     <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
     public void set(int index, BigDecimal value) {
@@ -747,12 +680,6 @@ public final class ${className} extends BaseDataValueVector implements <#if type
       setCount++;
     }
 
-    public void setScalar(int index, BigDecimal value) throws VectorOverflowException {
-      values.getMutator().setScalar(index, value);
-      bits.getMutator().setSafe(index, 1);
-      setCount++;
-    }
-
     </#if>
     @Override
     public void setValueCount(int valueCount) {


[15/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
DRILL-5657: Size-aware vector writer structure

- Vector and accessor layer
- Row Set layer
- Tuple and column models
- Revised write-time metadata
- "Result set loader" layer

this closes #914


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

Branch: refs/heads/master
Commit: 40de8ca4f47533fa6593d1266403868ae1a2119f
Parents: eb0c403
Author: Paul Rogers <pr...@maprtech.com>
Authored: Thu Aug 17 22:41:30 2017 -0700
Committer: Paul Rogers <pr...@maprtech.com>
Committed: Wed Dec 20 21:17:48 2017 -0800

----------------------------------------------------------------------
 .../exec/physical/rowSet/ResultSetLoader.java   |  204 +++
 .../exec/physical/rowSet/ResultVectorCache.java |   33 +
 .../exec/physical/rowSet/RowSetLoader.java      |  153 +++
 .../exec/physical/rowSet/impl/ColumnState.java  |  358 +++++
 .../physical/rowSet/impl/NullProjectionSet.java |   41 +
 .../rowSet/impl/NullResultVectorCacheImpl.java  |   41 +
 .../physical/rowSet/impl/NullVectorState.java   |   52 +
 .../rowSet/impl/NullableVectorState.java        |  108 ++
 .../physical/rowSet/impl/OptionBuilder.java     |  134 ++
 .../rowSet/impl/PrimitiveColumnState.java       |  105 ++
 .../physical/rowSet/impl/ProjectionSet.java     |   48 +
 .../physical/rowSet/impl/ProjectionSetImpl.java |  136 ++
 .../rowSet/impl/RepeatedVectorState.java        |  168 +++
 .../rowSet/impl/ResultSetLoaderImpl.java        |  775 +++++++++++
 .../rowSet/impl/ResultVectorCacheImpl.java      |  186 +++
 .../physical/rowSet/impl/RowSetLoaderImpl.java  |   98 ++
 .../physical/rowSet/impl/SingleVectorState.java |  274 ++++
 .../exec/physical/rowSet/impl/TupleState.java   |  388 ++++++
 .../rowSet/impl/VectorContainerBuilder.java     |  257 ++++
 .../exec/physical/rowSet/impl/VectorState.java  |  102 ++
 .../physical/rowSet/impl/WriterIndexImpl.java   |  100 ++
 .../exec/physical/rowSet/impl/package-info.java |  304 +++++
 .../physical/rowSet/model/BaseTupleModel.java   |  117 ++
 .../physical/rowSet/model/ContainerVisitor.java |  115 ++
 .../physical/rowSet/model/MetadataProvider.java |   93 ++
 .../exec/physical/rowSet/model/ReaderIndex.java |   53 +
 .../physical/rowSet/model/SchemaInference.java  |   61 +
 .../exec/physical/rowSet/model/TupleModel.java  |  117 ++
 .../rowSet/model/hyper/BaseReaderBuilder.java   |  149 +++
 .../rowSet/model/hyper/package-info.java        |   30 +
 .../physical/rowSet/model/package-info.java     |   68 +
 .../rowSet/model/single/BaseReaderBuilder.java  |   89 ++
 .../rowSet/model/single/BaseWriterBuilder.java  |   72 +
 .../model/single/BuildVectorsFromMetadata.java  |   97 ++
 .../rowSet/model/single/VectorAllocator.java    |  112 ++
 .../rowSet/model/single/package-info.java       |   28 +
 .../exec/physical/rowSet/package-info.java      |  193 +++
 .../apache/drill/exec/record/BatchSchema.java   |   42 +-
 .../apache/drill/exec/record/RecordBatch.java   |    3 +-
 .../apache/drill/exec/record/TupleSchema.java   |  534 ++++++++
 .../exec/record/selection/SelectionVector2.java |   20 +-
 .../exec/cache/TestBatchSerialization.java      |   22 +-
 .../exec/physical/impl/TopN/TopNBatchTest.java  |   26 +-
 .../impl/validate/TestBatchValidator.java       |   64 +-
 .../physical/impl/xsort/TestExternalSort.java   |   12 +-
 .../impl/xsort/managed/SortTestUtilities.java   |    8 +-
 .../physical/impl/xsort/managed/TestCopier.java |  146 +-
 .../impl/xsort/managed/TestShortArrays.java     |    8 +-
 .../impl/xsort/managed/TestSortImpl.java        |   46 +-
 .../physical/impl/xsort/managed/TestSorter.java |   38 +-
 .../rowSet/impl/TestResultSetLoaderLimits.java  |  224 ++++
 .../impl/TestResultSetLoaderMapArray.java       |  481 +++++++
 .../rowSet/impl/TestResultSetLoaderMaps.java    |  810 +++++++++++
 .../impl/TestResultSetLoaderOmittedValues.java  |  379 ++++++
 .../impl/TestResultSetLoaderOverflow.java       |  680 ++++++++++
 .../impl/TestResultSetLoaderProjection.java     |  470 +++++++
 .../impl/TestResultSetLoaderProtocol.java       |  586 ++++++++
 .../rowSet/impl/TestResultSetLoaderTorture.java |  453 +++++++
 .../rowSet/impl/TestResultSetSchemaChange.java  |  245 ++++
 .../drill/exec/record/TestTupleSchema.java      |  509 +++++++
 .../drill/exec/record/TestVectorContainer.java  |  127 --
 .../exec/record/vector/TestValueVector.java     |   12 +
 .../apache/drill/exec/sql/TestInfoSchema.java   |    2 +-
 .../exec/store/easy/text/compliant/TestCsv.java |    6 +-
 .../java/org/apache/drill/test/ExampleTest.java |    4 +-
 .../org/apache/drill/test/OperatorFixture.java  |   30 +-
 .../org/apache/drill/test/QueryBuilder.java     |   12 +-
 .../apache/drill/test/QueryRowSetIterator.java  |    2 +-
 .../drill/test/rowSet/AbstractRowSet.java       |  109 +-
 .../drill/test/rowSet/AbstractSingleRowSet.java |  182 +--
 .../apache/drill/test/rowSet/DirectRowSet.java  |  171 +--
 .../drill/test/rowSet/HyperRowSetImpl.java      |  245 +---
 .../drill/test/rowSet/IndirectRowSet.java       |   38 +-
 .../org/apache/drill/test/rowSet/RowSet.java    |   81 +-
 .../apache/drill/test/rowSet/RowSetBuilder.java |   32 +-
 .../drill/test/rowSet/RowSetComparison.java     |  124 +-
 .../apache/drill/test/rowSet/RowSetPrinter.java |   30 +-
 .../apache/drill/test/rowSet/RowSetReader.java  |   54 +
 .../drill/test/rowSet/RowSetReaderImpl.java     |   76 ++
 .../apache/drill/test/rowSet/RowSetSchema.java  |  304 -----
 .../drill/test/rowSet/RowSetUtilities.java      |  101 +-
 .../apache/drill/test/rowSet/RowSetWriter.java  |  119 ++
 .../drill/test/rowSet/RowSetWriterImpl.java     |  155 +++
 .../apache/drill/test/rowSet/SchemaBuilder.java |   87 +-
 .../drill/test/rowSet/file/JsonFileBuilder.java |   35 +-
 .../drill/test/rowSet/test/DummyWriterTest.java |  169 +++
 .../drill/test/rowSet/test/PerformanceTool.java |  296 ++++
 .../drill/test/rowSet/test/RowSetTest.java      |  858 +++++++-----
 .../drill/test/rowSet/test/TestFillEmpties.java |  241 ++++
 .../test/rowSet/test/TestFixedWidthWriter.java  |  444 ++++++
 .../rowSet/test/TestOffsetVectorWriter.java     |  425 ++++++
 .../test/rowSet/test/TestScalarAccessors.java   | 1266 ++++++++++++++++++
 .../rowSet/test/TestVariableWidthWriter.java    |  418 ++++++
 .../drill/test/rowSet/test/VectorPrinter.java   |   72 +
 .../apache/drill/vector/TestFillEmpties.java    |   55 +-
 .../apache/drill/vector/TestVectorLimits.java   |  487 -------
 exec/jdbc-all/pom.xml                           |    2 +-
 .../src/main/java/io/netty/buffer/DrillBuf.java |   70 +-
 .../netty/buffer/PooledByteBufAllocatorL.java   |   62 +-
 .../drill/exec/memory/AllocationManager.java    |   89 +-
 .../main/codegen/templates/ColumnAccessors.java |  383 +++---
 .../codegen/templates/FixedValueVectors.java    |  293 +---
 .../codegen/templates/NullableValueVectors.java |   91 +-
 .../codegen/templates/RepeatedValueVectors.java |   71 +-
 .../src/main/codegen/templates/UnionVector.java |   44 +-
 .../templates/VariableLengthVectors.java        |  216 +--
 .../drill/exec/record/ColumnMetadata.java       |  114 ++
 .../drill/exec/record/MaterializedField.java    |   41 +-
 .../apache/drill/exec/record/TupleMetadata.java |   88 ++
 .../drill/exec/record/TupleNameSpace.java       |   89 ++
 .../drill/exec/vector/AllocationHelper.java     |    2 +-
 .../drill/exec/vector/BaseDataValueVector.java  |   16 +
 .../org/apache/drill/exec/vector/BitVector.java |   52 +-
 .../drill/exec/vector/FixedWidthVector.java     |    7 +-
 .../apache/drill/exec/vector/ObjectVector.java  |   26 +-
 .../drill/exec/vector/UntypedNullVector.java    |   59 +-
 .../apache/drill/exec/vector/ValueVector.java   |   53 +-
 .../drill/exec/vector/VariableWidthVector.java  |    4 +-
 .../apache/drill/exec/vector/VectorUtils.java   |   63 -
 .../apache/drill/exec/vector/ZeroVector.java    |    6 +-
 .../exec/vector/accessor/AccessorUtilities.java |  125 --
 .../drill/exec/vector/accessor/ArrayReader.java |  108 +-
 .../drill/exec/vector/accessor/ArrayWriter.java |   60 +-
 .../exec/vector/accessor/ColumnAccessor.java    |   40 -
 .../exec/vector/accessor/ColumnReader.java      |   64 -
 .../exec/vector/accessor/ColumnReaderIndex.java |   28 +
 .../exec/vector/accessor/ColumnWriter.java      |   45 -
 .../exec/vector/accessor/ColumnWriterIndex.java |   76 ++
 .../exec/vector/accessor/ObjectReader.java      |   60 +
 .../drill/exec/vector/accessor/ObjectType.java  |   28 +
 .../exec/vector/accessor/ObjectWriter.java      |  101 ++
 .../vector/accessor/ScalarElementReader.java    |   65 +
 .../exec/vector/accessor/ScalarReader.java      |   75 ++
 .../exec/vector/accessor/ScalarWriter.java      |   71 +-
 .../exec/vector/accessor/TupleAccessor.java     |   71 -
 .../drill/exec/vector/accessor/TupleReader.java |   36 +-
 .../drill/exec/vector/accessor/TupleWriter.java |  154 ++-
 .../drill/exec/vector/accessor/ValueType.java   |   31 +
 .../accessor/impl/AbstractArrayReader.java      |  128 --
 .../accessor/impl/AbstractArrayWriter.java      |  127 --
 .../accessor/impl/AbstractColumnAccessor.java   |   43 -
 .../accessor/impl/AbstractColumnReader.java     |  126 --
 .../accessor/impl/AbstractColumnWriter.java     |   87 --
 .../accessor/impl/AbstractTupleAccessor.java    |   38 -
 .../vector/accessor/impl/AccessorUtilities.java |   53 +
 .../accessor/impl/ColumnAccessorFactory.java    |  122 --
 .../accessor/impl/HierarchicalFormatter.java    |   38 +
 .../accessor/impl/HierarchicalPrinter.java      |  238 ++++
 .../vector/accessor/impl/TupleReaderImpl.java   |  151 ---
 .../vector/accessor/impl/TupleWriterImpl.java   |  162 ---
 .../exec/vector/accessor/package-info.java      |   79 +-
 .../accessor/reader/AbstractArrayReader.java    |  188 +++
 .../accessor/reader/AbstractObjectReader.java   |   52 +
 .../accessor/reader/AbstractTupleReader.java    |  189 +++
 .../accessor/reader/BaseElementReader.java      |  187 +++
 .../accessor/reader/BaseScalarReader.java       |  189 +++
 .../accessor/reader/ColumnReaderFactory.java    |  109 ++
 .../accessor/reader/ElementReaderIndex.java     |   24 +
 .../reader/FixedWidthElementReaderIndex.java    |   38 +
 .../exec/vector/accessor/reader/MapReader.java  |   43 +
 .../accessor/reader/ObjectArrayReader.java      |  159 +++
 .../accessor/reader/ScalarArrayReader.java      |  102 ++
 .../vector/accessor/reader/VectorAccessor.java  |   26 +
 .../vector/accessor/reader/package-info.java    |   26 +
 .../accessor/writer/AbstractArrayWriter.java    |  348 +++++
 .../writer/AbstractFixedWidthWriter.java        |  258 ++++
 .../accessor/writer/AbstractObjectWriter.java   |   72 +
 .../accessor/writer/AbstractScalarWriter.java   |  126 ++
 .../accessor/writer/AbstractTupleWriter.java    |  450 +++++++
 .../accessor/writer/BaseScalarWriter.java       |  272 ++++
 .../accessor/writer/BaseVarWidthWriter.java     |  157 +++
 .../accessor/writer/ColumnWriterFactory.java    |  196 +++
 .../exec/vector/accessor/writer/MapWriter.java  |  155 +++
 .../accessor/writer/NullableScalarWriter.java   |  190 +++
 .../accessor/writer/ObjectArrayWriter.java      |  143 ++
 .../accessor/writer/OffsetVectorWriter.java     |  283 ++++
 .../accessor/writer/ScalarArrayWriter.java      |  229 ++++
 .../vector/accessor/writer/WriterEvents.java    |  127 ++
 .../accessor/writer/dummy/DummyArrayWriter.java |   96 ++
 .../writer/dummy/DummyScalarWriter.java         |   89 ++
 .../accessor/writer/dummy/package-info.java     |   54 +
 .../vector/accessor/writer/package-info.java    |  151 +++
 .../exec/vector/complex/AbstractMapVector.java  |   13 +-
 .../vector/complex/BaseRepeatedValueVector.java |   13 +-
 .../drill/exec/vector/complex/ListVector.java   |    4 +-
 .../drill/exec/vector/complex/MapVector.java    |   24 +-
 .../exec/vector/complex/RepeatedListVector.java |   20 +-
 .../exec/vector/complex/RepeatedMapVector.java  |   21 +-
 188 files changed, 22717 insertions(+), 4811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
new file mode 100644
index 0000000..a4b260b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultSetLoader.java
@@ -0,0 +1,204 @@
+/*
+ * 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.physical.rowSet;
+
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.BaseValueVector;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+
+/**
+ * Builds a result set (series of zero or more row sets) based on a defined
+ * schema which may
+ * evolve (expand) over time. Automatically rolls "overflow" rows over
+ * when a batch fills.
+ * <p>
+ * Many of the methods in this interface verify that the loader is
+ * in the proper state. For example, an exception is thrown if the caller
+ * attempts to save a row before starting a batch. However, the per-column
+ * write methods are checked only through assertions that should enabled
+ * during testing, but will be disabled during production.
+ *
+ * @see {@link VectorContainerWriter}, the class which this class
+ * replaces
+ */
+
+public interface ResultSetLoader {
+
+  public static final int DEFAULT_ROW_COUNT = BaseValueVector.INITIAL_VALUE_ALLOCATION;
+
+  /**
+   * Current schema version. The version increments by one each time
+   * a column is added.
+   * @return the current schema version
+   */
+
+  int schemaVersion();
+
+  /**
+   * Adjust the number of rows to produce in the next batch. Takes
+   * affect after the next call to {@link #startBatch()}.
+   *
+   * @param count target batch row count
+   */
+
+  void setTargetRowCount(int count);
+
+  /**
+   * The number of rows produced by this loader (as configured in the loader
+   * options.)
+   *
+   * @return the target row count for batches that this loader produces
+   */
+
+  int targetRowCount();
+
+  /**
+   * The largest vector size produced by this loader (as specified by
+   * the value vector limit.)
+   *
+   * @return the largest vector size. Attempting to extend a vector beyond
+   * this limit causes automatic vector overflow and terminates the
+   * in-flight batch, even if the batch has not yet reached the target
+   * row count
+   */
+
+  int targetVectorSize();
+
+  /**
+   * Total number of batches created. Includes the current batch if
+   * the row count in this batch is non-zero.
+   * @return the number of batches produced including the current
+   * one
+   */
+
+  int batchCount();
+
+  /**
+   * Total number of rows loaded for all previous batches and the
+   * current batch.
+   * @return total row count
+   */
+
+  int totalRowCount();
+
+  /**
+   * Start a new row batch. Valid only when first started, or after the
+   * previous batch has been harvested.
+   */
+
+  void startBatch();
+
+  /**
+   * Writer for the top-level tuple (the entire row). Valid only when
+   * the mutator is actively writing a batch (after <tt>startBatch()</tt>
+   * but before </tt>harvest()</tt>.)
+   *
+   * @return writer for the top-level columns
+   */
+
+  RowSetLoader writer();
+  boolean writeable();
+
+  /**
+   * Load a row using column values passed as variable-length arguments. Expects
+   * map values to represented as an array.
+   * A schema of (a:int, b:map(c:varchar)) would be>
+   * set as <br><tt>loadRow(10, new Object[] {"foo"});</tt><br>
+   * Values of arrays can be expressed as a Java
+   * array. A schema of (a:int, b:int[]) can be set as<br>
+   * <tt>loadRow(10, new int[] {100, 200});</tt><br>.
+   * Primarily for testing, too slow for production code.
+   * <p>
+   * If the row consists of a single map or list, then the one value will be an
+   * <tt>Object</tt> array, creating an ambiguity. Use <tt>writer().set(0, value);</tt>
+   * in this case.
+   *
+   * @param values column values in column index order
+   * @return this loader
+   */
+
+  ResultSetLoader setRow(Object...values);
+
+  /**
+   * Return the output container, primarily to obtain the schema
+   * and set of vectors. Depending on when this is called, the
+   * data may or may not be populated: call
+   * {@link #harvest()} to obtain the container for a batch.
+   * <p>
+   * This method is useful when the schema is known and fixed.
+   * After declaring the schema, call this method to get the container
+   * that holds the vectors for use in planning projection, etc.
+   * <p>
+   * If the result set schema changes, then a call to this method will
+   * return the latest schema. But, if the schema changes during the
+   * overflow row, then this method will not see those changes until
+   * after harvesting the current batch. (This avoid the appearance
+   * of phantom columns in the output since the new column won't
+   * appear until the next batch.)
+   * <p>
+   * Never count on the data in the container; it may be empty, half
+   * written, or inconsistent. Always call
+   * {@link #harvest()} to obtain the container for a batch.
+   *
+   * @return the output container including schema and value
+   * vectors
+   */
+
+  VectorContainer outputContainer();
+
+  /**
+   * Harvest the current row batch, and reset the mutator
+   * to the start of the next row batch (which may already contain
+   * an overflow row.
+   * <p>
+   * The schema of the returned container is defined as:
+   * <ul>
+   * <li>The schema as passed in via the loader options, plus</li>
+   * <li>Columns added dynamically during write, minus</li>
+   * <li>Any columns not included in the project list, minus</li>
+   * <li>Any columns added in the overflow row.</li>
+   * </ul>
+   * That is, column order is as defined by the initial schema and column
+   * additions. In particular, the schema order is <b>not</b> defined by
+   * the projection list. (Another mechanism is required to reorder columns
+   * for the actual projection.)
+   *
+   * @return the row batch to send downstream
+   */
+
+  VectorContainer harvest();
+
+  /**
+   * The schema of the harvested batch. Valid until the start of the
+   * next batch.
+   *
+   * @return the extended schema of the harvested batch which includes
+   * any allocation hints used when creating the batch
+   */
+
+  TupleMetadata harvestSchema();
+
+  /**
+   * Called after all rows are returned, whether because no more data is
+   * available, or the caller wishes to cancel the current row batch
+   * and complete.
+   */
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultVectorCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultVectorCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultVectorCache.java
new file mode 100644
index 0000000..6e32b5d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/ResultVectorCache.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.physical.rowSet;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Interface for a cache that implements "vector persistence" across
+ * multiple result set loaders. Allows a single scan operator to offer
+ * the same set of vectors even when data is read by a set of readers.
+ */
+
+public interface ResultVectorCache {
+  BufferAllocator allocator();
+  ValueVector addOrGet(MaterializedField colSchema);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetLoader.java
new file mode 100644
index 0000000..070e9a9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetLoader.java
@@ -0,0 +1,153 @@
+/*
+ * 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.physical.rowSet;
+
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+
+/**
+ * Interface for writing values to a row set. Only available for newly-created
+ * single row sets.
+ * <p>
+ * Typical usage:
+ *
+ * <pre></code>
+ * void writeABatch() {
+ *   RowSetLoader writer = ...
+ *   while (! writer.isFull()) {
+ *     writer.start();
+ *     writer.scalar(0).setInt(10);
+ *     writer.scalar(1).setString("foo");
+ *     ...
+ *     writer.save();
+ *   }
+ * }</code></pre>
+ * Alternative usage:
+ *
+ * <pre></code>
+ * void writeABatch() {
+ *   RowSetLoader writer = ...
+ *   while (writer.start()) {
+ *     writer.scalar(0).setInt(10);
+ *     writer.scalar(1).setString("foo");
+ *     ...
+ *     writer.save();
+ *   }
+ * }</code></pre>
+ *
+ * The above writes until the batch is full, based on size or vector overflow.
+ * That is, the details of vector overflow are hidden from the code that calls
+ * the writer.
+ */
+
+public interface RowSetLoader extends TupleWriter {
+
+  ResultSetLoader loader();
+
+  /**
+   * Write a row of values, given by Java objects. Object type must match
+   * expected column type. Stops writing, and returns false, if any value causes
+   * vector overflow. Value format:
+   * <ul>
+   * <li>For scalars, the value as a suitable Java type (int or Integer, say,
+   * for <tt>INTEGER</tt> values.)</li>
+   * <li>For scalar arrays, an array of a suitable Java primitive type for
+   * scalars. For example, <tt>int[]</tt> for an <tt>INTEGER</tt> column.</li>
+   * <li>For a Map, an <tt>Object<tt> array with values encoded as above.
+   * (In fact, the list here is the same as the map format.</li>
+   * <li>For a list (repeated map, list of list), an <tt>Object</tt> array with
+   * values encoded as above. (So, for a repeated map, an outer <tt>Object</tt>
+   * map encodes the array, an inner one encodes the map members.</li>
+   * </ul>
+   *
+   * @param values
+   *          variable-length argument list of column values
+   */
+
+  RowSetLoader addRow(Object... values);
+
+  /**
+   * Indicates that no more rows fit into the current row batch and that the row
+   * batch should be harvested and sent downstream. Any overflow row is
+   * automatically saved for the next cycle. The value is undefined when a batch
+   * is not active.
+   * <p>
+   * Will be false on the first row, and all subsequent rows until either the
+   * maximum number of rows are written, or a vector overflows. After that, will
+   * return true. The method returns false as soon as any column writer
+   * overflows even in the middle of a row write. That is, this writer does not
+   * automatically handle overflow rows because that added complexity is seldom
+   * needed for tests.
+   *
+   * @return true if another row can be written, false if not
+   */
+
+  boolean isFull();
+
+  /**
+   * The number of rows in the current row set. Does not count any overflow row
+   * saved for the next batch.
+   *
+   * @return number of rows to be sent downstream
+   */
+
+  int rowCount();
+
+  /**
+   * The index of the current row. Same as the row count except in an overflow
+   * row in which case the row index will revert to zero as soon as any vector
+   * overflows. Note: this means that the index can change between columns in a
+   * single row. Applications usually don't use this index directly; rely on the
+   * writers to write to the proper location.
+   *
+   * @return the current write index
+   */
+
+  int rowIndex();
+
+  /**
+   * Prepare a new row for writing. Call this before each row.
+   * <p>
+   * Handles a very special case: that of discarding the last row written.
+   * A reader can read a row into vectors, then "sniff" the row to check,
+   * for example, against a filter. If the row is not wanted, simply omit
+   * the call to <tt>save()</tt> and the next all to <tt>start()</tt> will
+   * discard the unsaved row.
+   * <p>
+   * Note that the vectors still contain values in the
+   * discarded position; just the various pointers are unset. If
+   * the batch ends before the discarded values are overwritten, the
+   * discarded values just exist at the end of the vector. Since vectors
+   * start with garbage contents, the discarded values are simply a different
+   * kind of garbage. But, if the client writes a new row, then the new
+   * row overwrites the discarded row. This works because we only change
+   * the tail part of a vector; never the internals.
+   *
+   * @return true if another row can be added, false if the batch is full
+   */
+
+  boolean start();
+
+  /**
+   * Saves the current row and moves to the next row. Failing to call this
+   * method effectively abandons the in-flight row; something that may be useful
+   * to recover from partially-written rows that turn out to contain errors.
+   * Done automatically if using <tt>setRow()</tt>.
+   */
+
+  void save();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
new file mode 100644
index 0000000..f3626d9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ColumnState.java
@@ -0,0 +1,358 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.ArrayList;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.OffsetVectorState;
+import org.apache.drill.exec.physical.rowSet.impl.TupleState.MapState;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;
+import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
+
+/**
+ * Represents the write-time state for a column including the writer and the (optional)
+ * backing vector. Implements per-column operations such as vector overflow. If a column
+ * is a (possibly repeated) map, then the column state will hold a tuple state.
+ * <p>
+ * If a column is not projected, then the writer exists (to make life easier for the
+ * reader), but there will be no vector backing the writer.
+ * <p>
+ * Different columns need different kinds of vectors: a data vector, possibly an offset
+ * vector, or even a non-existent vector. The {@link VectorState} class abstracts out
+ * these diffrences.
+ */
+
+public abstract class ColumnState {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ColumnState.class);
+
+  public static abstract class BaseMapColumnState extends ColumnState {
+    protected final MapState mapState;
+
+    public BaseMapColumnState(ResultSetLoaderImpl resultSetLoader,
+         AbstractObjectWriter writer, VectorState vectorState,
+         ProjectionSet projectionSet) {
+      super(resultSetLoader, writer, vectorState);
+      mapState = new MapState(resultSetLoader, this, projectionSet);
+    }
+
+    @Override
+    public void rollover() {
+      super.rollover();
+      mapState.rollover();
+    }
+
+    @Override
+    public void startBatch() {
+      super.startBatch();
+      mapState.startBatch();
+    }
+
+    @Override
+    public void harvestWithLookAhead() {
+      super.harvestWithLookAhead();
+      mapState.harvestWithLookAhead();
+    }
+
+    @Override
+    public void close() {
+      super.close();
+      mapState.close();
+    }
+
+    public MapState mapState() { return mapState; }
+  }
+
+  public static class MapColumnState extends BaseMapColumnState {
+
+    public MapColumnState(ResultSetLoaderImpl resultSetLoader,
+        ColumnMetadata columnSchema,
+        ProjectionSet projectionSet) {
+      super(resultSetLoader,
+          ColumnWriterFactory.buildMap(columnSchema, null,
+              new ArrayList<AbstractObjectWriter>()),
+          new NullVectorState(),
+          projectionSet);
+    }
+
+    @Override
+    public void updateCardinality(int cardinality) {
+      super.updateCardinality(cardinality);
+      mapState.updateCardinality(cardinality);
+    }
+  }
+
+  public static class MapArrayColumnState extends BaseMapColumnState {
+
+    public MapArrayColumnState(ResultSetLoaderImpl resultSetLoader,
+        AbstractObjectWriter writer,
+        VectorState vectorState,
+        ProjectionSet projectionSet) {
+      super(resultSetLoader, writer,
+          vectorState,
+          projectionSet);
+    }
+
+    @SuppressWarnings("resource")
+    public static MapArrayColumnState build(ResultSetLoaderImpl resultSetLoader,
+        ColumnMetadata columnSchema,
+        ProjectionSet projectionSet) {
+
+      // Create the map's offset vector.
+
+      UInt4Vector offsetVector = new UInt4Vector(
+          BaseRepeatedValueVector.OFFSETS_FIELD,
+          resultSetLoader.allocator());
+
+      // Create the writer using the offset vector
+
+      AbstractObjectWriter writer = ColumnWriterFactory.buildMapArray(
+          columnSchema, offsetVector,
+          new ArrayList<AbstractObjectWriter>());
+
+      // Wrap the offset vector in a vector state
+
+      VectorState vectorState = new OffsetVectorState(
+            ((AbstractArrayWriter) writer.array()).offsetWriter(),
+            offsetVector,
+            (AbstractObjectWriter) writer.array().entry());
+
+      // Assemble it all into the column state.
+
+      return new MapArrayColumnState(resultSetLoader,
+                  writer, vectorState, projectionSet);
+    }
+
+    @Override
+    public void updateCardinality(int cardinality) {
+      super.updateCardinality(cardinality);
+      int childCardinality = cardinality * schema().expectedElementCount();
+      mapState.updateCardinality(childCardinality);
+    }
+  }
+
+  /**
+   * Columns move through various lifecycle states as identified by this
+   * enum. (Yes, sorry that the term "state" is used in two different ways
+   * here: the variables for a column and the point within the column
+   * lifecycle.
+   */
+
+  protected enum State {
+
+    /**
+     * Column is in the normal state of writing with no overflow
+     * in effect.
+     */
+
+    NORMAL,
+
+    /**
+     * Like NORMAL, but means that the data has overflowed and the
+     * column's data for the current row appears in the new,
+     * overflow batch. For a client that omits some columns, written
+     * columns will be in OVERFLOW state, unwritten columns in
+     * NORMAL state.
+     */
+
+    OVERFLOW,
+
+    /**
+     * Indicates that the column has data saved
+     * in the overflow batch.
+     */
+
+    LOOK_AHEAD,
+
+    /**
+     * Like LOOK_AHEAD, but indicates the special case that the column
+     * was added after overflow, so there is no vector for the column
+     * in the harvested batch.
+     */
+
+    NEW_LOOK_AHEAD
+  }
+
+  protected final ResultSetLoaderImpl resultSetLoader;
+  protected final int addVersion;
+  protected final VectorState vectorState;
+  protected State state;
+  protected AbstractObjectWriter writer;
+
+  /**
+   * Cardinality of the value itself. If this is an array,
+   * then this is the number of arrays. A separate number,
+   * the inner cardinality, is computed as the outer cardinality
+   * times the expected array count (from metadata.) The inner
+   * cardinality is the total number of array items in the
+   * vector.
+   */
+
+  protected int outerCardinality;
+
+  public ColumnState(ResultSetLoaderImpl resultSetLoader,
+      AbstractObjectWriter writer, VectorState vectorState) {
+    this.resultSetLoader = resultSetLoader;
+    this.vectorState = vectorState;
+    this.addVersion = resultSetLoader.bumpVersion();
+    state = resultSetLoader.hasOverflow() ?
+        State.NEW_LOOK_AHEAD : State.NORMAL;
+    this.writer = writer;
+  }
+
+  public AbstractObjectWriter writer() { return writer; }
+  public ColumnMetadata schema() { return writer.schema(); }
+
+  public ValueVector vector() { return vectorState.vector(); }
+
+  public void allocateVectors() {
+    assert outerCardinality != 0;
+    resultSetLoader.tallyAllocations(
+        vectorState.allocate(outerCardinality));
+  }
+
+  /**
+   * Prepare the column for a new row batch after overflow on the previous
+   * batch. Restore the look-ahead buffer to the
+   * active vector so we start writing where we left off.
+   */
+
+  public void startBatch() {
+    switch (state) {
+    case NORMAL:
+      resultSetLoader.tallyAllocations(vectorState.allocate(outerCardinality));
+      break;
+
+    case NEW_LOOK_AHEAD:
+
+      // Column is new, was not exchanged with backup vector
+
+      break;
+
+    case LOOK_AHEAD:
+
+      // Restore the look-ahead values to the main vector.
+
+      vectorState.startBatchWithLookAhead();
+      break;
+
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+
+    // In all cases, we are back to normal writing.
+
+    state = State.NORMAL;
+  }
+
+  /**
+   * A column within the row batch overflowed. Prepare to absorb the rest of the
+   * in-flight row by rolling values over to a new vector, saving the complete
+   * vector for later. This column could have a value for the overflow row, or
+   * for some previous row, depending on exactly when and where the overflow
+   * occurs.
+   */
+
+  public void rollover() {
+    assert state == State.NORMAL;
+
+    // If the source index is 0, then we could not fit this one
+    // value in the original vector. Nothing will be accomplished by
+    // trying again with an an overflow vector. Just fail.
+    //
+    // Note that this is a judgment call. It is possible to allow the
+    // vector to double beyond the limit, but that will require a bit
+    // of thought to get right -- and, of course, completely defeats
+    // the purpose of limiting vector size to avoid memory fragmentation...
+
+    if (resultSetLoader.writerIndex().vectorIndex() == 0) {
+      throw UserException
+        .memoryError("A single column value is larger than the maximum allowed size of 16 MB")
+        .build(logger);
+    }
+
+    // Otherwise, do the roll-over to a look-ahead vector.
+
+    vectorState.rollover(outerCardinality);
+
+    // Remember that we did this overflow processing.
+
+    state = State.OVERFLOW;
+  }
+
+  /**
+   * Writing of a row batch is complete. Prepare the vector for harvesting
+   * to send downstream. If this batch encountered overflow, set aside the
+   * look-ahead vector and put the full vector buffer back into the active
+   * vector.
+   */
+
+  public void harvestWithLookAhead() {
+    switch (state) {
+    case NEW_LOOK_AHEAD:
+
+      // If added after overflow, no data to save from the complete
+      // batch: the vector does not appear in the completed batch.
+
+      break;
+
+    case OVERFLOW:
+
+      // Otherwise, restore the original, full buffer and
+      // last write position.
+
+      vectorState.harvestWithLookAhead();
+
+      // Remember that we have look-ahead values stashed away in the
+      // backup vector.
+
+      state = State.LOOK_AHEAD;
+      break;
+
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+  }
+
+  public void close() {
+    vectorState.reset();
+  }
+
+  public void updateCardinality(int cardinality) {
+    outerCardinality = cardinality;
+  }
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("addVersion", addVersion)
+      .attribute("state", state)
+      .attributeIdentity("writer", writer)
+      .attribute("vectorState")
+      ;
+    vectorState.dump(format);
+    format.endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullProjectionSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullProjectionSet.java
new file mode 100644
index 0000000..2fcc813
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullProjectionSet.java
@@ -0,0 +1,41 @@
+/*
+ * 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.physical.rowSet.impl;
+
+/**
+ * Represents a wildcard: SELECT * when used at the root tuple.
+ * When used with maps, means selection of all map columns, either
+ * implicitly, or because the map itself is selected.
+ */
+
+public class NullProjectionSet implements ProjectionSet {
+
+  private boolean allProjected;
+
+  public NullProjectionSet(boolean allProjected) {
+    this.allProjected = allProjected;
+  }
+
+  @Override
+  public boolean isProjected(String colName) { return allProjected; }
+
+  @Override
+  public ProjectionSet mapProjection(String colName) {
+    return new NullProjectionSet(allProjected);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullResultVectorCacheImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullResultVectorCacheImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullResultVectorCacheImpl.java
new file mode 100644
index 0000000..930dc30
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullResultVectorCacheImpl.java
@@ -0,0 +1,41 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+public class NullResultVectorCacheImpl implements ResultVectorCache {
+
+  private final BufferAllocator allocator;
+
+  public NullResultVectorCacheImpl(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  @Override
+  public BufferAllocator allocator() { return allocator; }
+
+  @Override
+  public ValueVector addOrGet(MaterializedField colSchema) {
+    return TypeHelper.getNewVector(colSchema, allocator, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullVectorState.java
new file mode 100644
index 0000000..8372758
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullVectorState.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.physical.rowSet.impl;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Do-nothing vector state for a map column which has no actual vector
+ * associated with it.
+ */
+
+public class NullVectorState implements VectorState {
+
+  @Override public int allocate(int cardinality) { return 0; }
+  @Override public void rollover(int cardinality) { }
+  @Override public void harvestWithLookAhead() { }
+  @Override public void startBatchWithLookAhead() { }
+  @Override public void reset() { }
+  @Override public ValueVector vector() { return null; }
+
+  public static class UnmanagedVectorState extends NullVectorState {
+    ValueVector vector;
+
+    public UnmanagedVectorState(ValueVector vector) {
+      this.vector = vector;
+    }
+
+    @Override
+    public ValueVector vector() { return vector; }
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.startObject(this).endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.java
new file mode 100644
index 0000000..bf91032
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/NullableVectorState.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.physical.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.ValuesVectorState;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.apache.drill.exec.vector.accessor.writer.NullableScalarWriter;
+
+public class NullableVectorState implements VectorState {
+
+  public static class BitsVectorState extends ValuesVectorState {
+
+    public BitsVectorState(ColumnMetadata schema, AbstractScalarWriter writer, ValueVector mainVector) {
+      super(schema, writer, mainVector);
+    }
+
+    @Override
+    public int allocateVector(ValueVector vector, int cardinality) {
+      ((FixedWidthVector) vector).allocateNew(cardinality);
+      return vector.getBufferSize();
+    }
+  }
+
+  private final ColumnMetadata schema;
+  private final NullableScalarWriter writer;
+  private final NullableVector vector;
+  private final ValuesVectorState bitsState;
+  private final ValuesVectorState valuesState;
+
+  public NullableVectorState(AbstractObjectWriter writer, NullableVector vector) {
+    this.schema = writer.schema();
+    this.vector = vector;
+
+    this.writer = (NullableScalarWriter) writer.scalar();
+    bitsState = new BitsVectorState(schema, this.writer.bitsWriter(), vector.getBitsVector());
+    valuesState = new ValuesVectorState(schema, this.writer.baseWriter(), vector.getValuesVector());
+  }
+
+  @Override
+  public int allocate(int cardinality) {
+    return bitsState.allocate(cardinality) +
+           valuesState.allocate(cardinality);
+  }
+
+  @Override
+  public void rollover(int cardinality) {
+    bitsState.rollover(cardinality);
+    valuesState.rollover(cardinality);
+  }
+
+  @Override
+  public void harvestWithLookAhead() {
+    bitsState.harvestWithLookAhead();
+    valuesState.harvestWithLookAhead();
+  }
+
+  @Override
+  public void startBatchWithLookAhead() {
+    bitsState.startBatchWithLookAhead();
+    valuesState.startBatchWithLookAhead();
+  }
+
+  @Override
+  public void reset() {
+    bitsState.reset();
+    valuesState.reset();
+  }
+
+  @Override
+  public ValueVector vector() { return vector; }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("schema", schema)
+      .attributeIdentity("writer", writer)
+      .attributeIdentity("vector", vector)
+      .attribute("bitsState");
+    bitsState.dump(format);
+    format
+      .attribute("valuesState");
+    valuesState.dump(format);
+    format
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
new file mode 100644
index 0000000..a743052
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/OptionBuilder.java
@@ -0,0 +1,134 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.BaseValueVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Builder for the options for the row set loader. Reasonable defaults
+ * are provided for all options; use these options for test code or
+ * for clients that don't need special settings.
+ */
+
+public class OptionBuilder {
+  protected int vectorSizeLimit;
+  protected int rowCountLimit;
+  protected Collection<SchemaPath> projection;
+  protected ResultVectorCache vectorCache;
+  protected TupleMetadata schema;
+  protected long maxBatchSize;
+
+  public OptionBuilder() {
+    ResultSetOptions options = new ResultSetOptions();
+    vectorSizeLimit = options.vectorSizeLimit;
+    rowCountLimit = options.rowCountLimit;
+    maxBatchSize = options.maxBatchSize;
+  }
+
+  /**
+   * Specify the maximum number of rows per batch. Defaults to
+   * {@link BaseValueVector#INITIAL_VALUE_ALLOCATION}. Batches end either
+   * when this limit is reached, or when a vector overflows, whichever
+   * occurs first. The limit is capped at
+   * {@link ValueVector#MAX_ROW_COUNT}.
+   *
+   * @param limit the row count limit
+   * @return this builder
+   */
+
+  public OptionBuilder setRowCountLimit(int limit) {
+    rowCountLimit = Math.max(1,
+        Math.min(limit, ValueVector.MAX_ROW_COUNT));
+    return this;
+  }
+
+  public OptionBuilder setBatchSizeLimit(int bytes) {
+    maxBatchSize = bytes;
+    return this;
+  }
+
+  /**
+   * Record (batch) readers often read a subset of available table columns,
+   * but want to use a writer schema that includes all columns for ease of
+   * writing. (For example, a CSV reader must read all columns, even if the user
+   * wants a subset. The unwanted columns are simply discarded.)
+   * <p>
+   * This option provides a projection list, in the form of column names, for
+   * those columns which are to be projected. Only those columns will be
+   * backed by value vectors; non-projected columns will be backed by "null"
+   * writers that discard all values.
+   *
+   * @param projection the list of projected columns
+   * @return this builder
+   */
+
+  // TODO: Use SchemaPath in place of strings.
+
+  public OptionBuilder setProjection(Collection<SchemaPath> projection) {
+    this.projection = projection;
+    return this;
+  }
+
+  /**
+   * Downstream operators require "vector persistence": the same vector
+   * must represent the same column in every batch. For the scan operator,
+   * which creates multiple readers, this can be a challenge. The vector
+   * cache provides a transparent mechanism to enable vector persistence
+   * by returning the same vector for a set of independent readers. By
+   * default, the code uses a "null" cache which creates a new vector on
+   * each request. If a true cache is needed, the caller must provide one
+   * here.
+   */
+
+  public OptionBuilder setVectorCache(ResultVectorCache vectorCache) {
+    this.vectorCache = vectorCache;
+    return this;
+  }
+
+  /**
+   * Clients can use the row set builder in several ways:
+   * <ul>
+   * <li>Provide the schema up front, when known, by using this method to
+   * provide the schema.</li>
+   * <li>Discover the schema on the fly, adding columns during the write
+   * operation. Leave this method unset to start with an empty schema.</li>
+   * <li>A combination of the above.</li>
+   * </ul>
+   * @param schema the initial schema for the loader
+   * @return this builder
+   */
+
+  public OptionBuilder setSchema(TupleMetadata schema) {
+    this.schema = schema;
+    return this;
+  }
+
+  // TODO: No setter for vector length yet: is hard-coded
+  // at present in the value vector.
+
+  public ResultSetOptions build() {
+    return new ResultSetOptions(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/PrimitiveColumnState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/PrimitiveColumnState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/PrimitiveColumnState.java
new file mode 100644
index 0000000..c97ec18
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/PrimitiveColumnState.java
@@ -0,0 +1,105 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.ValuesVectorState;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * Primitive (non-map) column state. Handles all three cardinalities.
+ * Column metadata is hosted on the writer.
+ */
+
+public class PrimitiveColumnState extends ColumnState implements ColumnWriterListener {
+
+  public PrimitiveColumnState(ResultSetLoaderImpl resultSetLoader,
+      AbstractObjectWriter colWriter,
+      VectorState vectorState) {
+    super(resultSetLoader, colWriter, vectorState);
+    writer.bindListener(this);
+  }
+
+  public static PrimitiveColumnState newPrimitive(
+      ResultSetLoaderImpl resultSetLoader,
+      ValueVector vector,
+      AbstractObjectWriter writer) {
+    VectorState vectorState;
+    if (vector == null) {
+      vectorState = new NullVectorState();
+    } else {
+      vectorState = new ValuesVectorState(
+          writer.schema(),
+          (AbstractScalarWriter) writer.scalar(),
+          vector);
+    }
+    return new PrimitiveColumnState(resultSetLoader, writer,
+        vectorState);
+  }
+
+  public static PrimitiveColumnState newNullablePrimitive(
+      ResultSetLoaderImpl resultSetLoader,
+      ValueVector vector,
+      AbstractObjectWriter writer) {
+    VectorState vectorState;
+    if (vector == null) {
+      vectorState = new NullVectorState();
+    } else {
+      vectorState = new NullableVectorState(
+          writer,
+          (NullableVector) vector);
+    }
+    return new PrimitiveColumnState(resultSetLoader, writer,
+        vectorState);
+  }
+
+  public static PrimitiveColumnState newPrimitiveArray(
+      ResultSetLoaderImpl resultSetLoader,
+      ValueVector vector,
+      AbstractObjectWriter writer) {
+    VectorState vectorState;
+    if (vector == null) {
+      vectorState = new NullVectorState();
+    } else {
+      vectorState = new RepeatedVectorState(writer, (RepeatedValueVector) vector);
+    }
+    return new PrimitiveColumnState(resultSetLoader, writer,
+        vectorState);
+  }
+
+  @Override
+  public void overflowed(ScalarWriter writer) {
+    resultSetLoader.overflowed();
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public boolean canExpand(ScalarWriter writer, int delta) {
+    return resultSetLoader.canExpand(delta);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSet.java
new file mode 100644
index 0000000..9ea118f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSet.java
@@ -0,0 +1,48 @@
+/*
+ * 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.physical.rowSet.impl;
+
+/**
+ * Represents the set of columns projected for a tuple (row or map.)
+ * The projected columns might themselves be columns, so returns a
+ * projection set for such columns.
+ * <p>
+ * Three implementations exist:
+ * <ul>
+ * <li>Project all ({@link NullProjectionSet): used for a tuple when
+ * all columns are projected. Example: the root tuple (the row) in
+ * a <tt>SELECT *</tt> query.</li>
+ * <li>Project none  (also {@link NullProjectionSet): used when no
+ * columns are projected from a tuple, such as when a map itself is
+ * not projected, so none of its member columns are projected.</li>
+ * <li>Project some ({@link ProjectionSetImpl}: used in the
+ * <tt>SELECT a, c, e</tt> case in which the query identifies which
+ * columns to project (implicitly leaving out others, such as b and
+ * d in our example.)</li>
+ * </ul>
+ * <p>
+ * The result is that each tuple (row and map) has an associated
+ * projection set which the code can query to determine if a newly
+ * added column is wanted (and so should have a backing vector) or
+ * is unwanted (and can just receive a dummy writer.)
+ */
+
+interface ProjectionSet {
+  boolean isProjected(String colName);
+  ProjectionSet mapProjection(String colName);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSetImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSetImpl.java
new file mode 100644
index 0000000..e17f486
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ProjectionSetImpl.java
@@ -0,0 +1,136 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+
+/**
+ * Represents an explicit projection at some tuple level.
+ * <p>
+ * A column is projected if it is explicitly listed in the selection list.
+ * <p>
+ * If a column is a map, then the projection for the map's columns is based on
+ * two rules:
+ * <ol>
+ * <li>If the projection list includes at least one explicit mention of a map
+ * member, then include only those columns explicitly listed.</li>
+ * <li>If the projection at the parent level lists only the map column itself
+ * (which the projection can't know is a map), then assume this implies all
+ * columns, as if the entry where "map.*".</li>
+ * </ol>
+ * <p>
+ * Examples:<br>
+ * <code>m</code><br>
+ * If m turns out to be a map, project all members of m.<br>
+ * <code>m.a</code><br>
+ * Column m must be a map. Project only column a.<br>
+ * <code>m, m.a</code><br>
+ * Tricky case. We interpret this as projecting only the "a" element of map m.
+ * <p>
+ * The projection set is build from a list of columns, represented as
+ * {@link SchemaPath} objects, provided by the physical plan. The structure of
+ * <tt>SchemaPath</tt> is a bit awkward:
+ * <p>
+ * <ul>
+ * <li><tt>SchemaPath> is a wrapper for a column which directly holds the
+ * <tt>NameSegment</tt> for the top-level column.</li>
+ * <li><tt>NameSegment</tt> holds a name. This can be a top name such as
+ * `a`, or parts of a compound name such as `a`.`b`. Each <tt>NameSegment</tt>
+ * has a "child" that points to the option following parts of the name.</li>
+ * <li><PathSegment</tt> is the base class for the parts of a name.</tt>
+ * <li><tt>ArraySegment</tt> is the other kind of name part and represents
+ * an array index such as the "[1]" in `columns`[1].</li>
+ * <ul>
+ * The parser here consumes only names, this mechanism does not consider
+ * array indexes. As a result, there may be multiple projected columns that
+ * map to the same projection here: `columns`[1] and `columns`[2] both map to
+ * the name `columns`, for example.
+ */
+
+public class ProjectionSetImpl implements ProjectionSet {
+
+  Set<String> projection = new HashSet<>();
+  Map<String, ProjectionSetImpl> mapProjections = CaseInsensitiveMap
+      .newHashMap();
+
+  @Override
+  public boolean isProjected(String colName) {
+    return projection.contains(colName.toLowerCase());
+  }
+
+  @Override
+  public ProjectionSet mapProjection(String colName) {
+    ProjectionSet mapProj = mapProjections.get(colName.toLowerCase());
+    if (mapProj != null) {
+      return mapProj;
+    }
+
+    // No explicit information for the map. Members inherit the
+    // same projection as the map itself.
+
+    return new NullProjectionSet(isProjected(colName));
+  }
+
+  /**
+   * Parse a projection list. The list should consist of a list of column
+   * names; any wildcards should have been processed by the caller. An
+   * empty or null list means everything is projected (that is, an
+   * empty list here is equivalent to a wildcard in the SELECT
+   * statement.)
+   *
+   * @param projList
+   * @return
+   */
+  public static ProjectionSet parse(Collection<SchemaPath> projList) {
+    if (projList == null || projList.isEmpty()) {
+      return new NullProjectionSet(true);
+    }
+    ProjectionSetImpl projSet = new ProjectionSetImpl();
+    for (SchemaPath col : projList) {
+      projSet.addSegment(col.getRootSegment());
+    }
+    return projSet;
+  }
+
+  private void addSegment(NameSegment rootSegment) {
+    String rootKey = rootSegment.getPath().toLowerCase();
+    projection.add(rootKey);
+    PathSegment child = rootSegment.getChild();
+    if (child == null) {
+      return;
+    }
+    if (child.isArray()) {
+      // Ignore the [x] array suffix.
+      return;
+    }
+    ProjectionSetImpl map = mapProjections.get(rootKey);
+    if (map == null) {
+      map = new ProjectionSetImpl();
+      mapProjections.put(rootKey, map);
+    }
+    map.addSegment((NameSegment) child);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
new file mode 100644
index 0000000..98b6beb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RepeatedVectorState.java
@@ -0,0 +1,168 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.OffsetVectorState;
+import org.apache.drill.exec.physical.rowSet.impl.SingleVectorState.ValuesVectorState;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * Vector state for a scalar array (repeated scalar) vector. Manages both the
+ * offsets vector and data vector during overflow and other operations.
+ */
+
+public class RepeatedVectorState implements VectorState {
+  private final ColumnMetadata schema;
+  private final AbstractArrayWriter arrayWriter;
+  private final RepeatedValueVector vector;
+  private final OffsetVectorState offsetsState;
+  private final ValuesVectorState valuesState;
+
+  public RepeatedVectorState(AbstractObjectWriter writer, RepeatedValueVector vector) {
+    this.schema = writer.schema();
+
+    // Get the repeated vector
+
+    this.vector = vector;
+
+    // Create the values state using the value (data) portion of the repeated
+    // vector, and the scalar (value) portion of the array writer.
+
+    arrayWriter = (AbstractArrayWriter) writer.array();
+    AbstractScalarWriter colWriter = (AbstractScalarWriter) arrayWriter.scalar();
+    valuesState = new ValuesVectorState(schema, colWriter, vector.getDataVector());
+
+    // Create the offsets state with the offset vector portion of the repeated
+    // vector, and the offset writer portion of the array writer.
+
+    offsetsState = new OffsetVectorState(arrayWriter.offsetWriter(),
+        vector.getOffsetVector(),
+        (AbstractObjectWriter) arrayWriter.entry());
+  }
+
+  @Override
+  public ValueVector vector() { return vector; }
+
+  @Override
+  public int allocate(int cardinality) {
+    return offsetsState.allocate(cardinality) +
+           valuesState.allocate(childCardinality(cardinality));
+  }
+
+  private int childCardinality(int cardinality) {
+    return cardinality * schema.expectedElementCount();
+  }
+
+  /**
+   * The column is a scalar or an array of scalars. We need to roll over both the column
+   * values and the offsets that point to those values. The index provided is
+   * the index into the offset vector. We use that to obtain the index of the
+   * values to roll-over.
+   * <p>
+   * Data structure:
+   * <p><pre></code>
+   * RepeatedVectorState (this class)
+   * +- OffsetVectorState
+   * .  +- OffsetVectorWriter (A)
+   * .  +- Offset vector (B)
+   * .  +- Backup (e.g. look-ahead) offset vector
+   * +- ValuesVectorState
+   * .  +- Scalar (element) writer (C)
+   * .  +- Data (elements) vector (D)
+   * .  +- Backup elements vector
+   * +- Array Writer
+   * .  +- ColumnWriterIndex (for array as a whole)
+   * .  +- OffsetVectorWriter (A)
+   * .  .  +- Offset vector (B)
+   * .  +- ArrayElementWriterIndex
+   * .  +- ScalarWriter (D)
+   * .  .  +- ArrayElementWriterIndex
+   * .  .  +- Scalar vector (D)
+   * </code></pre>
+   * <p>
+   * The top group of objects point into the writer objects in the second
+   * group. Letters in parens show the connections.
+   * <p>
+   * To perform the roll-over, we must:
+   * <ul>
+   * <li>Copy values from the current vectors to a set of new, look-ahead
+   * vectors.</li>
+   * <li>Swap buffers between the main and "backup" vectors, effectively
+   * moving the "full" batch to the sidelines, putting the look-ahead vectors
+   * into play in order to finish writing the current row.</li>
+   * <li>Update the writers to point to the look-ahead buffers, including
+   * the initial set of data copied into those vectors.</li>
+   * <li>Update the vector indexes to point to the next write positions
+   * after the values copied during roll-over.</li>
+   * </ul>
+   *
+   * @param cardinality the number of outer elements to create in the look-ahead
+   * vector
+   */
+
+  @Override
+  public void rollover(int cardinality) {
+
+    // Swap out the two vectors. The index presented to the caller
+    // is that of the data vector: the next position in the data
+    // vector to be set into the data vector writer index.
+
+    valuesState.rollover(childCardinality(cardinality));
+    offsetsState.rollover(cardinality);
+  }
+
+  @Override
+  public void harvestWithLookAhead() {
+    offsetsState.harvestWithLookAhead();
+    valuesState.harvestWithLookAhead();
+  }
+
+  @Override
+  public void startBatchWithLookAhead() {
+    offsetsState.startBatchWithLookAhead();
+    valuesState.startBatchWithLookAhead();
+  }
+
+  @Override
+  public void reset() {
+    offsetsState.reset();
+    valuesState.reset();
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("schema", schema)
+      .attributeIdentity("writer", arrayWriter)
+      .attributeIdentity("vector", vector)
+      .attribute("offsetsState");
+    offsetsState.dump(format);
+    format
+      .attribute("valuesState");
+    valuesState.dump(format);
+    format
+      .endObject();
+  }
+}


[11/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
new file mode 100644
index 0000000..b23eb0d
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMaps.java
@@ -0,0 +1,810 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test (non-array) map support in the result set loader and related classes.
+ */
+
+public class TestResultSetLoaderMaps extends SubOperatorTest {
+
+  @Test
+  public void testBasics() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("c", MinorType.INT)
+          .add("d", MinorType.VARCHAR)
+          .buildMap()
+        .add("e", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Verify structure and schema
+
+    assertEquals(5, rsLoader.schemaVersion());
+    TupleMetadata actualSchema = rootWriter.schema();
+    assertEquals(3, actualSchema.size());
+    assertTrue(actualSchema.metadata(1).isMap());
+    assertEquals(2, actualSchema.metadata("m").mapSchema().size());
+    assertEquals(2, actualSchema.column("m").getChildren().size());
+
+    rsLoader.startBatch();
+
+    // Write a row the way that clients will do.
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    TupleWriter mWriter = rootWriter.tuple("m");
+    ScalarWriter cWriter = mWriter.scalar("c");
+    ScalarWriter dWriter = mWriter.scalar("d");
+    ScalarWriter eWriter = rootWriter.scalar("e");
+
+    rootWriter.start();
+    aWriter.setInt(10);
+    cWriter.setInt(110);
+    dWriter.setString("fred");
+    eWriter.setString("pebbles");
+    rootWriter.save();
+
+    // Try adding a duplicate column.
+
+    try {
+      mWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL));
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+
+    // Write another using the test-time conveniences
+
+    rootWriter.addRow(20, new Object[] {210, "barney"}, "bam-bam");
+
+    // Harvest the batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(5, rsLoader.schemaVersion());
+    assertEquals(2, actual.rowCount());
+
+    // Validate data
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {110, "fred"}, "pebbles")
+        .addRow(20, new Object[] {210, "barney"}, "bam-bam")
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+    rsLoader.close();
+  }
+
+  /**
+   * Create schema with a map, then add columns to the map
+   * after delivering the first batch. The new columns should appear
+   * in the second-batch output.
+   */
+
+  @Test
+  public void testMapEvolution() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("b", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(3, rsLoader.schemaVersion());
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(10, new Object[] {"fred"})
+      .addRow(20, new Object[] {"barney"});
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(3, rsLoader.schemaVersion());
+    assertEquals(2, actual.rowCount());
+
+    // Validate first batch
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {"fred"})
+        .addRow(20, new Object[] {"barney"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // Add three columns in the second batch. One before
+    // the batch starts, one before the first row, and one after
+    // the first row.
+
+    TupleWriter mapWriter = rootWriter.tuple("m");
+    mapWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED));
+
+    rsLoader.startBatch();
+    mapWriter.addColumn(SchemaBuilder.columnSchema("d", MinorType.BIGINT, DataMode.REQUIRED));
+
+    rootWriter.addRow(30, new Object[] {"wilma", 130, 130_000L});
+
+    mapWriter.addColumn(SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.REQUIRED));
+    rootWriter.addRow(40, new Object[] {"betty", 140, 140_000L, "bam-bam"});
+
+    actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(6, rsLoader.schemaVersion());
+    assertEquals(2, actual.rowCount());
+
+    // Validate first batch
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("b", MinorType.VARCHAR)
+          .add("c", MinorType.INT)
+          .add("d", MinorType.BIGINT)
+          .add("e", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(30, new Object[] {"wilma", 130, 130_000L, ""})
+        .addRow(40, new Object[] {"betty", 140, 140_000L, "bam-bam"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test adding a map to a loader after writing the first row.
+   */
+
+  @Test
+  public void testMapAddition() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(1, rsLoader.schemaVersion());
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Start without the map. Add a map after the first row.
+
+    rsLoader.startBatch();
+    rootWriter.addRow(10);
+
+    int mapIndex = rootWriter.addColumn(SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REQUIRED));
+    TupleWriter mapWriter = rootWriter.tuple(mapIndex);
+
+    // Add a column to the map with the same name as the top-level column.
+    // Verifies that the name spaces are independent.
+
+    mapWriter.addColumn(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rootWriter
+      .addRow(20, new Object[]{"fred"})
+      .addRow(30, new Object[]{"barney"});
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(3, rsLoader.schemaVersion());
+    assertEquals(3, actual.rowCount());
+
+    // Validate first batch
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("a", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, new Object[] {""})
+        .addRow(20, new Object[] {"fred"})
+        .addRow(30, new Object[] {"barney"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test adding an empty map to a loader after writing the first row.
+   * Then add columns in another batch. Yes, this is a bizarre condition,
+   * but we must check it anyway for robustness.
+   */
+
+  @Test
+  public void testEmptyMapAddition() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(1, rsLoader.schemaVersion());
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Start without the map. Add a map after the first row.
+
+    rsLoader.startBatch();
+    rootWriter.addRow(10);
+
+    int mapIndex = rootWriter.addColumn(SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REQUIRED));
+    TupleWriter mapWriter = rootWriter.tuple(mapIndex);
+
+    rootWriter
+      .addRow(20, new Object[]{})
+      .addRow(30, new Object[]{});
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(2, rsLoader.schemaVersion());
+    assertEquals(3, actual.rowCount());
+
+    // Validate first batch
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .buildMap()
+        .buildSchema();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, new Object[] {})
+        .addRow(20, new Object[] {})
+        .addRow(30, new Object[] {})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // Now add another column to the map
+
+    rsLoader.startBatch();
+    mapWriter.addColumn(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rootWriter
+      .addRow(40, new Object[]{"fred"})
+      .addRow(50, new Object[]{"barney"});
+
+    actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(3, rsLoader.schemaVersion());
+    assertEquals(2, actual.rowCount());
+
+    // Validate first batch
+
+    expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("a", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(40, new Object[] {"fred"})
+        .addRow(50, new Object[] {"barney"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Create nested maps. Then, add columns to each map
+   * on the fly. Use required, variable-width columns since
+   * those require the most processing and are most likely to
+   * fail if anything is out of place.
+   */
+
+  @Test
+  public void testNestedMapsRequired() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m1")
+          .add("b", MinorType.VARCHAR)
+          .addMap("m2")
+            .add("c", MinorType.VARCHAR)
+            .buildMap()
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(5, rsLoader.schemaVersion());
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    rootWriter.addRow(10, new Object[] {"b1", new Object[] {"c1"}});
+
+    // Validate first batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(5, rsLoader.schemaVersion());
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {"b1", new Object[] {"c1"}})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // Now add columns in the second batch.
+
+    rsLoader.startBatch();
+    rootWriter.addRow(20, new Object[] {"b2", new Object[] {"c2"}});
+
+    TupleWriter m1Writer = rootWriter.tuple("m1");
+    m1Writer.addColumn(SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.REQUIRED));
+    TupleWriter m2Writer = m1Writer.tuple("m2");
+    m2Writer.addColumn(SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rootWriter.addRow(30, new Object[] {"b3", new Object[] {"c3", "e3"}, "d3"});
+
+    // And another set while the write proceeds.
+
+    m1Writer.addColumn(SchemaBuilder.columnSchema("f", MinorType.VARCHAR, DataMode.REQUIRED));
+    m2Writer.addColumn(SchemaBuilder.columnSchema("g", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rootWriter.addRow(40, new Object[] {"b4", new Object[] {"c4", "e4", "g4"}, "d4", "e4"});
+
+    // Validate second batch
+
+    actual = fixture.wrap(rsLoader.harvest());
+    assertEquals(9, rsLoader.schemaVersion());
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m1")
+          .add("b", MinorType.VARCHAR)
+          .addMap("m2")
+            .add("c", MinorType.VARCHAR)
+            .add("e", MinorType.VARCHAR)
+            .add("g", MinorType.VARCHAR)
+            .buildMap()
+          .add("d", MinorType.VARCHAR)
+          .add("f", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(20, new Object[] {"b2", new Object[] {"c2", "",   ""  }, "",    "" })
+        .addRow(30, new Object[] {"b3", new Object[] {"c3", "e3", ""  }, "d3",  "" })
+        .addRow(40, new Object[] {"b4", new Object[] {"c4", "e4", "g4"}, "d4", "e4"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Create nested maps. Then, add columns to each map
+   * on the fly. This time, with nullable types.
+   */
+
+  @Test
+  public void testNestedMapsNullable() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m1")
+          .addNullable("b", MinorType.VARCHAR)
+          .addMap("m2")
+            .addNullable("c", MinorType.VARCHAR)
+            .buildMap()
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    rootWriter.addRow(10, new Object[] {"b1", new Object[] {"c1"}});
+
+    // Validate first batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {"b1", new Object[] {"c1"}})
+        .build();
+//    actual.print();
+//    expected.print();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // Now add columns in the second batch.
+
+    rsLoader.startBatch();
+    rootWriter.addRow(20, new Object[] {"b2", new Object[] {"c2"}});
+
+    TupleWriter m1Writer = rootWriter.tuple("m1");
+    m1Writer.addColumn(SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.OPTIONAL));
+    TupleWriter m2Writer = m1Writer.tuple("m2");
+    m2Writer.addColumn(SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.OPTIONAL));
+
+    rootWriter.addRow(30, new Object[] {"b3", new Object[] {"c3", "e3"}, "d3"});
+
+    // And another set while the write proceeds.
+
+    m1Writer.addColumn(SchemaBuilder.columnSchema("f", MinorType.VARCHAR, DataMode.OPTIONAL));
+    m2Writer.addColumn(SchemaBuilder.columnSchema("g", MinorType.VARCHAR, DataMode.OPTIONAL));
+
+    rootWriter.addRow(40, new Object[] {"b4", new Object[] {"c4", "e4", "g4"}, "d4", "e4"});
+
+    // Validate second batch
+
+    actual = fixture.wrap(rsLoader.harvest());
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m1")
+          .addNullable("b", MinorType.VARCHAR)
+          .addMap("m2")
+            .addNullable("c", MinorType.VARCHAR)
+            .addNullable("e", MinorType.VARCHAR)
+            .addNullable("g", MinorType.VARCHAR)
+            .buildMap()
+          .addNullable("d", MinorType.VARCHAR)
+          .addNullable("f", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(20, new Object[] {"b2", new Object[] {"c2", null, null}, null, null})
+        .addRow(30, new Object[] {"b3", new Object[] {"c3", "e3", null}, "d3", null})
+        .addRow(40, new Object[] {"b4", new Object[] {"c4", "e4", "g4"}, "d4", "e4"})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test a map that contains a scalar array. No reason to suspect that this
+   * will have problem as the array writer is fully tested in the accessor
+   * subsystem. Still, need to test the cardinality methods of the loader
+   * layer.
+   */
+
+  @Test
+  public void testMapWithArray() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .addArray("c", MinorType.INT)
+          .addArray("d", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Write some rows
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(10, new Object[] {new int[] {110, 120, 130},
+                                new String[] {"d1.1", "d1.2", "d1.3", "d1.4"}})
+      .addRow(20, new Object[] {new int[] {210}, new String[] {}})
+      .addRow(30, new Object[] {new int[] {}, new String[] {"d3.1"}})
+      ;
+
+    // Validate first batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {new int[] {110, 120, 130},
+                                  new String[] {"d1.1", "d1.2", "d1.3", "d1.4"}})
+        .addRow(20, new Object[] {new int[] {210}, new String[] {}})
+        .addRow(30, new Object[] {new int[] {}, new String[] {"d3.1"}})
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // Add another array after the first row in the second batch.
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(40, new Object[] {new int[] {410, 420}, new String[] {"d4.1", "d4.2"}})
+      .addRow(50, new Object[] {new int[] {510}, new String[] {"d5.1"}})
+      ;
+
+    TupleWriter mapWriter = rootWriter.tuple("m");
+    mapWriter.addColumn(SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.REPEATED));
+    rootWriter
+      .addRow(60, new Object[] {new int[] {610, 620}, new String[] {"d6.1", "d6.2"}, new String[] {"e6.1", "e6.2"}})
+      .addRow(70, new Object[] {new int[] {710}, new String[] {}, new String[] {"e7.1", "e7.2"}})
+      ;
+
+    // Validate first batch. The new array should have been back-filled with
+    // empty offsets for the missing rows.
+
+    actual = fixture.wrap(rsLoader.harvest());
+//    System.out.println(actual.schema().toString());
+    expected = fixture.rowSetBuilder(actual.schema())
+        .addRow(40, new Object[] {new int[] {410, 420}, new String[] {"d4.1", "d4.2"}, new String[] {}})
+        .addRow(50, new Object[] {new int[] {510}, new String[] {"d5.1"}, new String[] {}})
+        .addRow(60, new Object[] {new int[] {610, 620}, new String[] {"d6.1", "d6.2"}, new String[] {"e6.1", "e6.2"}})
+        .addRow(70, new Object[] {new int[] {710}, new String[] {}, new String[] {"e7.1", "e7.2"}})
+        .build();
+//    expected.print();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Create a schema with a map, then trigger an overflow on one of the columns
+   * in the map. Proper overflow handling should occur regardless of nesting
+   * depth.
+   */
+
+  @Test
+  public void testMapWithOverflow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m1")
+          .add("b", MinorType.INT)
+          .addMap("m2")
+            .add("c", MinorType.INT) // Before overflow, written
+            .add("d", MinorType.VARCHAR)
+            .add("e", MinorType.INT) // After overflow, not yet written
+            .buildMap()
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (! rootWriter.isFull()) {
+      rootWriter.addRow(count, new Object[] {count * 10, new Object[] {count * 100, value, count * 1000}});
+      count++;
+    }
+
+    // Our row count should include the overflow row
+
+    int expectedCount = ValueVector.MAX_BUFFER_SIZE / value.length;
+    assertEquals(expectedCount + 1, count);
+
+    // Loader's row count should include only "visible" rows
+
+    assertEquals(expectedCount, rootWriter.rowCount());
+
+    // Total count should include invisible and look-ahead rows.
+
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+
+    // Result should exclude the overflow row
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(expectedCount, result.rowCount());
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(1, result.rowCount());
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test the case in which a new column is added during the overflow row. Unlike
+   * the top-level schema case, internally we must create a copy of the map, and
+   * move vectors across only when the result is to include the schema version
+   * of the target column. For overflow, the new column is added after the
+   * first batch; it is added in the second batch that contains the overflow
+   * row in which the column was added.
+   */
+
+  @Test
+  public void testMapOverflowWithNewColumn() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("b", MinorType.INT)
+          .add("c", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(4, rsLoader.schemaVersion());
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Can't use the shortcut to populate rows when doing a schema
+    // change.
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    TupleWriter mWriter = rootWriter.tuple("m");
+    ScalarWriter bWriter = mWriter.scalar("b");
+    ScalarWriter cWriter = mWriter.scalar("c");
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      aWriter.setInt(count);
+      bWriter.setInt(count * 10);
+      cWriter.setBytes(value, value.length);
+      if (rootWriter.isFull()) {
+
+        // Overflow just occurred. Add another column.
+
+        mWriter.addColumn(SchemaBuilder.columnSchema("d", MinorType.INT, DataMode.OPTIONAL));
+        mWriter.scalar("d").setInt(count * 100);
+      }
+      rootWriter.save();
+      count++;
+    }
+
+    // Result set should include the original columns, but not d.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+
+    assertEquals(4, rsLoader.schemaVersion());
+    assertTrue(schema.isEquivalent(result.schema()));
+    BatchSchema expectedSchema = new BatchSchema(SelectionVectorMode.NONE, schema.toFieldList());
+    assertTrue(expectedSchema.isEquivalent(result.batchSchema()));
+
+    // Use a reader to validate row-by-row. Too large to create an expected
+    // result set.
+
+    RowSetReader reader = result.reader();
+    TupleReader mapReader = reader.tuple("m");
+    int rowId = 0;
+    while (reader.next()) {
+      assertEquals(rowId, reader.scalar("a").getInt());
+      assertEquals(rowId * 10, mapReader.scalar("b").getInt());
+      assertTrue(Arrays.equals(value, mapReader.scalar("c").getBytes()));
+      rowId++;
+    }
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    assertEquals(1, rootWriter.rowCount());
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(1, result.rowCount());
+
+    reader = result.reader();
+    mapReader = reader.tuple("m");
+    while (reader.next()) {
+      assertEquals(rowId, reader.scalar("a").getInt());
+      assertEquals(rowId * 10, mapReader.scalar("b").getInt());
+      assertTrue(Arrays.equals(value, mapReader.scalar("c").getBytes()));
+      assertEquals(rowId * 100, mapReader.scalar("d").getInt());
+    }
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Version of the {#link TestResultSetLoaderProtocol#testOverwriteRow()} test
+   * that uses nested columns.
+   */
+
+  @Test
+  public void testOverwriteRow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .add("b", MinorType.INT)
+          .add("c", MinorType.VARCHAR)
+        .buildMap()
+      .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Can't use the shortcut to populate rows when doing overwrites.
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    TupleWriter mWriter = rootWriter.tuple("m");
+    ScalarWriter bWriter = mWriter.scalar("b");
+    ScalarWriter cWriter = mWriter.scalar("c");
+
+    // Write 100,000 rows, overwriting 99% of them. This will cause vector
+    // overflow and data corruption if overwrite does not work; but will happily
+    // produce the correct result if everything works as it should.
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (count < 100_000) {
+      rootWriter.start();
+      count++;
+      aWriter.setInt(count);
+      bWriter.setInt(count * 10);
+      cWriter.setBytes(value, value.length);
+      if (count % 100 == 0) {
+        rootWriter.save();
+      }
+    }
+
+    // Verify using a reader.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(count / 100, result.rowCount());
+    RowSetReader reader = result.reader();
+    TupleReader mReader = reader.tuple("m");
+    int rowId = 1;
+    while (reader.next()) {
+      assertEquals(rowId * 100, reader.scalar("a").getInt());
+      assertEquals(rowId * 1000, mReader.scalar("b").getInt());
+      assertTrue(Arrays.equals(value, mReader.scalar("c").getBytes()));
+      rowId++;
+    }
+
+    result.clear();
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
new file mode 100644
index 0000000..2c4c87b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOmittedValues.java
@@ -0,0 +1,379 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+public class TestResultSetLoaderOmittedValues extends SubOperatorTest {
+
+  /**
+   * Test "holes" in the middle of a batch, and unset columns at
+   * the end. Ending the batch should fill in missing values.
+   */
+
+  @Test
+  public void testOmittedValuesAtEnd() {
+
+    // Create columns up front
+
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .addNullable("c", MinorType.VARCHAR)
+        .add("d", MinorType.INT)
+        .addNullable("e", MinorType.INT)
+        .addArray("f", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    int rowCount = 0;
+    ScalarWriter arrayWriter;
+    for (int i = 0; i < 2;  i++) { // Row 0, 1
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setInt(rowCount);
+      rootWriter.scalar(1).setString("b_" + rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.scalar(3).setInt(rowCount * 10);
+      rootWriter.scalar(4).setInt(rowCount * 100);
+      arrayWriter = rootWriter.column(5).array().scalar();
+      arrayWriter.setString("f_" + rowCount + "-1");
+      arrayWriter.setString("f_" + rowCount + "-2");
+      rootWriter.save();
+    }
+
+    // Holes in half the columns
+
+    for (int i = 0; i < 2;  i++) { // Rows 2, 3
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setInt(rowCount);
+      rootWriter.scalar(1).setString("b_" + rowCount);
+      rootWriter.scalar(3).setInt(rowCount * 10);
+      arrayWriter = rootWriter.column(5).array().scalar();
+      arrayWriter.setString("f_" + rowCount + "-1");
+      arrayWriter.setString("f_" + rowCount + "-2");
+      rootWriter.save();
+    }
+
+    // Holes in the other half
+
+    for (int i = 0; i < 2;  i++) { // Rows 4, 5
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setInt(rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.scalar(4).setInt(rowCount * 100);
+      rootWriter.save();
+    }
+
+    // All columns again.
+
+    for (int i = 0; i < 2;  i++) { // Rows 6, 7
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setInt(rowCount);
+      rootWriter.scalar(1).setString("b_" + rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.scalar(3).setInt(rowCount * 10);
+      rootWriter.scalar(4).setInt(rowCount * 100);
+      arrayWriter = rootWriter.column(5).array().scalar();
+      arrayWriter.setString("f_" + rowCount + "-1");
+      arrayWriter.setString("f_" + rowCount + "-2");
+      rootWriter.save();
+    }
+
+    // Omit all but the key column at end
+
+    for (int i = 0; i < 2;  i++) { // Rows 8, 9
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setInt(rowCount);
+      rootWriter.save();
+    }
+
+    // Harvest the row and verify.
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+//    actual.print();
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .addNullable("c", MinorType.VARCHAR)
+        .add("3", MinorType.INT)
+        .addNullable("e", MinorType.INT)
+        .addArray("f", MinorType.VARCHAR)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(  1, "b_1", "c_1",  10,  100, new String[] {"f_1-1",  "f_1-2"})
+        .addRow(  2, "b_2", "c_2",  20,  200, new String[] {"f_2-1",  "f_2-2"})
+        .addRow(  3, "b_3", null,   30, null, new String[] {"f_3-1",  "f_3-2"})
+        .addRow(  4, "b_4", null,   40, null, new String[] {"f_4-1",  "f_4-2"})
+        .addRow(  5, "",    "c_5",   0,  500, new String[] {})
+        .addRow(  6, "",    "c_6",   0,  600, new String[] {})
+        .addRow(  7, "b_7", "c_7",  70,  700, new String[] {"f_7-1",  "f_7-2"})
+        .addRow(  8, "b_8", "c_8",  80,  800, new String[] {"f_8-1",  "f_8-2"})
+        .addRow(  9, "",    null,    0, null, new String[] {})
+        .addRow( 10, "",    null,    0, null, new String[] {})
+        .build();
+
+    new RowSetComparison(expected)
+        .verifyAndClearAll(actual);
+    rsLoader.close();
+  }
+
+  /**
+   * Test "holes" at the end of a batch when batch overflows. Completed
+   * batch must be finalized correctly, new batch initialized correct,
+   * for the missing values.
+   */
+
+  @Test
+  public void testOmittedValuesAtEndWithOverflow() {
+    TupleMetadata schema = new SchemaBuilder()
+        // Row index
+        .add("a", MinorType.INT)
+        // Column that forces overflow
+        .add("b", MinorType.VARCHAR)
+        // Column with all holes
+        .addNullable("c", MinorType.VARCHAR)
+        // Column with some holes
+        .addNullable("d", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Fill the batch. Column d has some values. Column c is worst case: no values.
+
+    rsLoader.startBatch();
+    byte value[] = new byte[533];
+    Arrays.fill(value, (byte) 'X');
+    int rowNumber = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rowNumber++;
+      rootWriter.scalar(0).setInt(rowNumber);
+      rootWriter.scalar(1).setBytes(value, value.length);
+      if (rowNumber < 10_000) {
+        rootWriter.scalar(3).setString("d-" + rowNumber);
+      }
+      rootWriter.save();
+      assertEquals(rowNumber, rsLoader.totalRowCount());
+    }
+
+    // Harvest and verify
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(rowNumber - 1, result.rowCount());
+    RowSetReader reader = result.reader();
+    int rowIndex = 0;
+    while (reader.next()) {
+      int expectedRowNumber = 1 + rowIndex;
+      assertEquals(expectedRowNumber, reader.scalar(0).getInt());
+      assertTrue(reader.scalar(2).isNull());
+      if (expectedRowNumber < 10_000) {
+        assertEquals("d-" + expectedRowNumber, reader.scalar(3).getString());
+      } else {
+        assertTrue(reader.scalar(3).isNull());
+      }
+      rowIndex++;
+    }
+
+    // Start count for this batch is one less than current
+    // count, because of the overflow row.
+
+    int startRowNumber = rowNumber;
+
+    // Write a few more rows to the next batch
+
+    rsLoader.startBatch();
+    for (int i = 0; i < 10; i++) {
+      rootWriter.start();
+      rowNumber++;
+      rootWriter.scalar(0).setInt(rowNumber);
+      rootWriter.scalar(1).setBytes(value, value.length);
+      if (i > 5) {
+        rootWriter.scalar(3).setString("d-" + rowNumber);
+      }
+      rootWriter.save();
+      assertEquals(rowNumber, rsLoader.totalRowCount());
+    }
+
+    // Verify that holes were preserved.
+
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(rowNumber, rsLoader.totalRowCount());
+    assertEquals(rowNumber - startRowNumber + 1, result.rowCount());
+//    result.print();
+    reader = result.reader();
+    rowIndex = 0;
+    while (reader.next()) {
+      int expectedRowNumber = startRowNumber + rowIndex;
+      assertEquals(expectedRowNumber, reader.scalar(0).getInt());
+      assertTrue(reader.scalar(2).isNull());
+      if (rowIndex > 6) {
+        assertEquals("d-" + expectedRowNumber, reader.scalar(3).getString());
+      } else {
+        assertTrue("Row " + rowIndex + " col d should be null", reader.scalar(3).isNull());
+      }
+      rowIndex++;
+    }
+    assertEquals(rowIndex, 11);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test that omitting the call to saveRow() effectively discards
+   * the row. Note that the vectors still contain values in the
+   * discarded position; just the various pointers are unset. If
+   * the batch ends before the discarded values are overwritten, the
+   * discarded values just exist at the end of the vector. Since vectors
+   * start with garbage contents, the discarded values are simply a different
+   * kind of garbage. But, if the client writes a new row, then the new
+   * row overwrites the discarded row. This works because we only change
+   * the tail part of a vector; never the internals.
+   */
+
+  @Test
+  public void testSkipRows() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    int rowNumber = 0;
+    for (int i = 0; i < 14; i++) {
+      rootWriter.start();
+      rowNumber++;
+      rootWriter.scalar(0).setInt(rowNumber);
+      if (i % 3 == 0) {
+        rootWriter.scalar(1).setNull();
+      } else {
+        rootWriter.scalar(1).setString("b-" + rowNumber);
+      }
+      if (i % 2 == 0) {
+        rootWriter.save();
+      }
+    }
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+//    result.print();
+    SingleRowSet expected = fixture.rowSetBuilder(result.batchSchema())
+        .addRow( 1, null)
+        .addRow( 3, "b-3")
+        .addRow( 5, "b-5")
+        .addRow( 7, null)
+        .addRow( 9, "b-9")
+        .addRow(11, "b-11")
+        .addRow(13, null)
+        .build();
+//    expected.print();
+    new RowSetComparison(expected)
+      .verifyAndClearAll(result);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test that discarding a row works even if that row happens to be an
+   * overflow row.
+   */
+
+  @Test
+  public void testSkipOverflowRow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setInt(count);
+      rootWriter.scalar(1).setBytes(value, value.length);
+
+      // Relies on fact that isFull becomes true right after
+      // a vector overflows; don't have to wait for saveRow().
+      // Keep all rows, but discard the overflow row.
+
+      if (! rootWriter.isFull()) {
+        rootWriter.save();
+      }
+      count++;
+    }
+
+    // Discard the results.
+
+    rsLoader.harvest().zeroVectors();
+
+    // Harvest the next batch. Will be empty (because overflow row
+    // was discarded.)
+
+    rsLoader.startBatch();
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(0, result.rowCount());
+    result.clear();
+
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
new file mode 100644
index 0000000..0146cfe
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderOverflow.java
@@ -0,0 +1,680 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+
+/**
+ * Exercise the vector overflow functionality for the result set loader.
+ */
+
+public class TestResultSetLoaderOverflow extends SubOperatorTest {
+
+  /**
+   * Test that the writer detects a vector overflow. The offending column
+   * value should be moved to the next batch.
+   */
+
+  @Test
+  public void testVectorSizeLimit() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("s", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+
+    // Number of rows should be driven by vector size.
+    // Our row count should include the overflow row
+
+    int expectedCount = ValueVector.MAX_BUFFER_SIZE / value.length;
+    assertEquals(expectedCount + 1, count);
+
+    // Loader's row count should include only "visible" rows
+
+    assertEquals(expectedCount, rootWriter.rowCount());
+
+    // Total count should include invisible and look-ahead rows.
+
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+
+    // Result should exclude the overflow row
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(expectedCount, result.rowCount());
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(1, result.rowCount());
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test that the writer detects a vector overflow. The offending column
+   * value should be moved to the next batch.
+   */
+
+  @Test
+  public void testBatchSizeLimit() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("s", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .setBatchSizeLimit(
+            8 * 1024 * 1024 + // Data
+            2 * ValueVector.MAX_ROW_COUNT * 4) // Offsets, doubled because of +1
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+
+    // Our row count should include the overflow row
+
+    int expectedCount = 8 * 1024 * 1024 / value.length;
+    assertEquals(expectedCount + 1, count);
+
+    // Loader's row count should include only "visible" rows
+
+    assertEquals(expectedCount, rootWriter.rowCount());
+
+    // Total count should include invisible and look-ahead rows.
+
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+
+    // Result should exclude the overflow row
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(expectedCount, result.rowCount());
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(1, result.rowCount());
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Load a batch to overflow. Then, close the loader with the overflow
+   * batch unharvested. The Loader should release the memory allocated
+   * to the unused overflow vectors.
+   */
+
+  @Test
+  public void testCloseWithOverflow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("s", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+
+    assertTrue(count < ValueVector.MAX_ROW_COUNT);
+
+    // Harvest the full batch
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    result.clear();
+
+    // Close without harvesting the overflow batch.
+
+    rsLoader.close();
+  }
+
+  /**
+   * Case where a single array fills up the vector to the maximum size
+   * limit. Overflow won't work here; the attempt will fail with a user
+   * exception.
+   */
+
+  @Test
+  public void testOversizeArray() {
+    TupleMetadata schema = new SchemaBuilder()
+        .addArray("s", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Create a single array as the column value in the first row. When
+    // this overflows, an exception is thrown since overflow is not possible.
+
+    rsLoader.startBatch();
+    byte value[] = new byte[473];
+    Arrays.fill(value, (byte) 'X');
+    rootWriter.start();
+    ScalarWriter array = rootWriter.array(0).scalar();
+    try {
+      for (int i = 0; i < ValueVector.MAX_ROW_COUNT; i++) {
+        array.setBytes(value, value.length);
+      }
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("column value is larger than the maximum"));
+    }
+    rsLoader.close();
+  }
+
+  /**
+   * Test a row with a single array column which overflows. Verifies
+   * that all the fiddly bits about offset vectors and so on works
+   * correctly. Run this test (the simplest case) if you change anything
+   * about the array handling code.
+   */
+
+  @Test
+  public void testSizeLimitOnArray() {
+    TupleMetadata schema = new SchemaBuilder()
+        .addArray("s", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Fill batch with rows of with a single array, three values each. Tack on
+    // a suffix to each so we can be sure the proper data is written and moved
+    // to the overflow batch.
+
+    rsLoader.startBatch();
+    byte value[] = new byte[473];
+    Arrays.fill(value, (byte) 'X');
+    String strValue = new String(value, Charsets.UTF_8);
+    int count = 0;
+    int rowSize = 0;
+    int totalSize = 0;
+    int valuesPerArray = 13;
+    while (rootWriter.start()) {
+      totalSize += rowSize;
+      rowSize = 0;
+      ScalarWriter array = rootWriter.array(0).scalar();
+      for (int i = 0; i < valuesPerArray; i++) {
+        String cellValue = strValue + (count + 1) + "." + i;
+        array.setString(cellValue);
+        rowSize += cellValue.length();
+      }
+      rootWriter.save();
+      count++;
+    }
+
+    // Row count should include the overflow row.
+
+    int expectedCount = count - 1;
+
+    // Size without overflow row should fit in the vector, size
+    // with overflow should not.
+
+    assertTrue(totalSize <= ValueVector.MAX_BUFFER_SIZE);
+    assertTrue(totalSize + rowSize > ValueVector.MAX_BUFFER_SIZE);
+
+    // Result should exclude the overflow row. Last row
+    // should hold the last full array.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(expectedCount, result.rowCount());
+    RowSetReader reader = result.reader();
+    reader.set(expectedCount - 1);
+    ScalarElementReader arrayReader = reader.column(0).elements();
+    assertEquals(valuesPerArray, arrayReader.size());
+    for (int i = 0; i < valuesPerArray; i++) {
+      String cellValue = strValue + (count - 1) + "." + i;
+      assertEquals(cellValue, arrayReader.getString(i));
+    }
+    result.clear();
+
+    // Next batch should start with the overflow row.
+    // The only row in this next batch should be the whole
+    // array being written at the time of overflow.
+
+    rsLoader.startBatch();
+//    VectorPrinter.printStrings((VarCharVector) ((VarCharColumnWriter) rootWriter.array(0).scalar()).vector(), 0, 5);
+//    ((ResultSetLoaderImpl) rsLoader).dump(new HierarchicalPrinter());
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+    result = fixture.wrap(rsLoader.harvest());
+//    VectorPrinter.printStrings((VarCharVector) ((VarCharColumnWriter) rootWriter.array(0).scalar()).vector(), 0, 5);
+    assertEquals(1, result.rowCount());
+    reader = result.reader();
+    reader.next();
+    arrayReader = reader.column(0).elements();
+    assertEquals(valuesPerArray, arrayReader.size());
+    for (int i = 0; i < valuesPerArray; i++) {
+      String cellValue = strValue + (count) + "." + i;
+      assertEquals(cellValue, arrayReader.getString(i));
+    }
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test the complete set of array overflow cases:
+   * <ul>
+   * <li>Array a is written before the column that has overflow,
+   * and must be copied, in its entirety, to the overflow row.</li>
+   * <li>Column b causes the overflow.</li>
+   * <li>Column c is written after the overflow, and should go
+   * to the look-ahead row.</li>
+   * <li>Column d is written for a while, then has empties before
+   * the overflow row, but is written in the overflow row.<li>
+   * <li>Column e is like d, but is not written in the overflow
+   * row.</li>
+   */
+
+  @Test
+  public void testArrayOverflowWithOtherArrays() {
+    TupleMetadata schema = new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .addArray("b", MinorType.VARCHAR)
+        .addArray("c", MinorType.INT)
+        .addArray("d", MinorType.INT)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Fill batch with rows of with a single array, three values each. Tack on
+    // a suffix to each so we can be sure the proper data is written and moved
+    // to the overflow batch.
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    String strValue = new String(value, Charsets.UTF_8);
+
+    int aCount = 3;
+    int bCount = 11;
+    int cCount = 5;
+    int dCount = 7;
+
+    int cCutoff = ValueVector.MAX_BUFFER_SIZE / value.length / bCount / 2;
+
+    ScalarWriter aWriter = rootWriter.array("a").scalar();
+    ScalarWriter bWriter = rootWriter.array("b").scalar();
+    ScalarWriter cWriter = rootWriter.array("c").scalar();
+    ScalarWriter dWriter = rootWriter.array("d").scalar();
+
+    int count = 0;
+    rsLoader.startBatch();
+    while (rootWriter.start()) {
+      if (rootWriter.rowCount() == 2952) {
+        count = count + 0;
+      }
+      for (int i = 0; i < aCount; i++) {
+        aWriter.setInt(count * aCount + i);
+      }
+      for (int i = 0; i < bCount; i++) {
+        String cellValue = strValue + (count * bCount + i);
+        bWriter.setString(cellValue);
+      }
+      if (count < cCutoff) {
+        for (int i = 0; i < cCount; i++) {
+          cWriter.setInt(count * cCount + i);
+        }
+      }
+
+      // Relies on fact that isFull becomes true right after
+      // a vector overflows; don't have to wait for saveRow().
+
+      if (count < cCutoff || rootWriter.isFull()) {
+        for (int i = 0; i < dCount; i++) {
+          dWriter.setInt(count * dCount + i);
+        }
+      }
+      rootWriter.save();
+      count++;
+    }
+
+    // Verify
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(count - 1, result.rowCount());
+
+    RowSetReader reader = result.reader();
+    ScalarElementReader aReader = reader.array("a").elements();
+    ScalarElementReader bReader = reader.array("b").elements();
+    ScalarElementReader cReader = reader.array("c").elements();
+    ScalarElementReader dReader = reader.array("d").elements();
+
+    while (reader.next()) {
+      int rowId = reader.rowIndex();
+      assertEquals(aCount, aReader.size());
+      for (int i = 0; i < aCount; i++) {
+        assertEquals(rowId * aCount + i, aReader.getInt(i));
+      }
+      assertEquals(bCount, bReader.size());
+      for (int i = 0; i < bCount; i++) {
+        String cellValue = strValue + (rowId * bCount + i);
+        assertEquals(cellValue, bReader.getString(i));
+      }
+      if (rowId < cCutoff) {
+        assertEquals(cCount, cReader.size());
+        for (int i = 0; i < cCount; i++) {
+          assertEquals(rowId * cCount + i, cReader.getInt(i));
+        }
+        assertEquals(dCount, dReader.size());
+        for (int i = 0; i < dCount; i++) {
+          assertEquals(rowId * dCount + i, dReader.getInt(i));
+        }
+      } else {
+        assertEquals(0, cReader.size());
+        assertEquals(0, dReader.size());
+      }
+    }
+    result.clear();
+    int firstCount = count - 1;
+
+    // One row is in the batch. Write more, skipping over the
+    // initial few values for columns c and d. Column d has a
+    // roll-over value, c has an empty roll-over.
+
+    rsLoader.startBatch();
+    for (int j = 0; j < 5; j++) {
+      rootWriter.start();
+      for (int i = 0; i < aCount; i++) {
+        aWriter.setInt(count * aCount + i);
+      }
+      for (int i = 0; i < bCount; i++) {
+        String cellValue = strValue + (count * bCount + i);
+        bWriter.setString(cellValue);
+      }
+      if (j > 3) {
+        for (int i = 0; i < cCount; i++) {
+          cWriter.setInt(count * cCount + i);
+        }
+        for (int i = 0; i < dCount; i++) {
+          dWriter.setInt(count * dCount + i);
+        }
+      }
+      rootWriter.save();
+      count++;
+    }
+
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(6, result.rowCount());
+
+    reader = result.reader();
+    aReader = reader.array("a").elements();
+    bReader = reader.array("b").elements();
+    cReader = reader.array("c").elements();
+    dReader = reader.array("d").elements();
+
+    int j = 0;
+    while (reader.next()) {
+      int rowId = firstCount + reader.rowIndex();
+      assertEquals(aCount, aReader.size());
+      for (int i = 0; i < aCount; i++) {
+        assertEquals("Index " + i, rowId * aCount + i, aReader.getInt(i));
+      }
+      assertEquals(bCount, bReader.size());
+      for (int i = 0; i < bCount; i++) {
+        String cellValue = strValue + (rowId * bCount + i);
+        assertEquals(cellValue, bReader.getString(i));
+      }
+      if (j > 4) {
+        assertEquals(cCount, cReader.size());
+        for (int i = 0; i < cCount; i++) {
+          assertEquals(rowId * cCount + i, cReader.getInt(i));
+        }
+      } else {
+        assertEquals(0, cReader.size());
+      }
+      if (j == 0 || j > 4) {
+        assertEquals(dCount, dReader.size());
+        for (int i = 0; i < dCount; i++) {
+          assertEquals(rowId * dCount + i, dReader.getInt(i));
+        }
+      } else {
+        assertEquals(0, dReader.size());
+      }
+      j++;
+    }
+    result.clear();
+
+    rsLoader.close();
+  }
+
+  /**
+   * Create an array that contains more than 64K values. Drill has no numeric
+   * limit on array lengths. (Well, it does, but the limit is about 2 billion
+   * which, even for bytes, is too large to fit into a vector...)
+   */
+
+  @Test
+  public void testLargeArray() {
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator());
+    RowSetLoader rootWriter = rsLoader.writer();
+    MaterializedField field = SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REPEATED);
+    rootWriter.addColumn(field);
+
+    // Create a single array as the column value in the first row. When
+    // this overflows, an exception is thrown since overflow is not possible.
+
+    rsLoader.startBatch();
+    rootWriter.start();
+    ScalarWriter array = rootWriter.array(0).scalar();
+    try {
+      for (int i = 0; i < Integer.MAX_VALUE; i++) {
+        array.setInt(i+1);
+      }
+      fail();
+    } catch (UserException e) {
+      // Expected
+    }
+    rsLoader.close();
+  }
+
+  /**
+   * Test the case that an array has "missing values" before the overflow.
+   */
+
+  @Test
+  public void testMissingArrayValues() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .addArray("c", MinorType.INT)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+
+    int blankAfter = ValueVector.MAX_BUFFER_SIZE / 512 * 2 / 3;
+    ScalarWriter cWriter = rootWriter.array("c").scalar();
+
+    rsLoader.startBatch();
+    int rowId = 0;
+    while (rootWriter.start()) {
+      rootWriter.scalar("a").setInt(rowId);
+      rootWriter.scalar("b").setBytes(value, value.length);
+      if (rowId < blankAfter) {
+        for (int i = 0; i < 3; i++) {
+          cWriter.setInt(rowId * 3 + i);
+        }
+      }
+      rootWriter.save();
+      rowId++;
+    }
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(rowId - 1, result.rowCount());
+    RowSetReader reader = result.reader();
+    ScalarElementReader cReader = reader.array("c").elements();
+    while (reader.next()) {
+      assertEquals(reader.rowIndex(), reader.scalar("a").getInt());
+      assertTrue(Arrays.equals(value, reader.scalar("b").getBytes()));
+      if (reader.rowIndex() < blankAfter) {
+        assertEquals(3, cReader.size());
+        for (int i = 0; i < 3; i++) {
+          assertEquals(reader.rowIndex() * 3 + i, cReader.getInt(i));
+        }
+      } else {
+        assertEquals(0, cReader.size());
+      }
+    }
+    result.clear();
+    rsLoader.close();
+  }
+
+  @Test
+  public void testOverflowWithNullables() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("n", MinorType.INT)
+        .addNullable("a", MinorType.VARCHAR)
+        .addNullable("b", MinorType.VARCHAR)
+        .addNullable("c", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setInt(count);
+      rootWriter.scalar(1).setNull();
+      rootWriter.scalar(2).setBytes(value, value.length);
+      rootWriter.scalar(3).setNull();
+      rootWriter.save();
+      count++;
+    }
+
+    // Result should exclude the overflow row
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(count - 1, result.rowCount());
+
+    RowSetReader reader = result.reader();
+    while (reader.next()) {
+      assertEquals(reader.rowIndex(), reader.scalar(0).getInt());
+      assertTrue(reader.scalar(1).isNull());
+      assertTrue(Arrays.equals(value, reader.scalar(2).getBytes()));
+      assertTrue(reader.scalar(3).isNull());
+    }
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    result = fixture.wrap(rsLoader.harvest());
+    reader = result.reader();
+    assertEquals(1, result.rowCount());
+    assertTrue(reader.next());
+    assertEquals(count - 1, reader.scalar(0).getInt());
+    assertTrue(reader.scalar(1).isNull());
+    assertTrue(Arrays.equals(value, reader.scalar(2).getBytes()));
+    assertTrue(reader.scalar(3).isNull());
+    result.clear();
+
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
new file mode 100644
index 0000000..5c6ff7b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProjection.java
@@ -0,0 +1,470 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Test of the basics of the projection mechanism.
+ */
+
+public class TestResultSetLoaderProjection extends SubOperatorTest {
+
+  @Test
+  public void testProjectionMap() {
+
+    // Null map means everything is projected
+
+    {
+      ProjectionSet projSet = ProjectionSetImpl.parse(null);
+      assertTrue(projSet instanceof NullProjectionSet);
+      assertTrue(projSet.isProjected("foo"));
+    }
+
+    // Empty list means everything is projected
+
+    {
+      ProjectionSet projSet = ProjectionSetImpl.parse(new ArrayList<SchemaPath>());
+      assertTrue(projSet instanceof NullProjectionSet);
+      assertTrue(projSet.isProjected("foo"));
+    }
+
+    // Simple non-map columns
+
+    {
+      List<SchemaPath> projCols = new ArrayList<>();
+      projCols.add(SchemaPath.getSimplePath("foo"));
+      projCols.add(SchemaPath.getSimplePath("bar"));
+      ProjectionSet projSet = ProjectionSetImpl.parse(projCols);
+      assertTrue(projSet instanceof ProjectionSetImpl);
+      assertTrue(projSet.isProjected("foo"));
+      assertTrue(projSet.isProjected("bar"));
+      assertFalse(projSet.isProjected("mumble"));
+    }
+
+    // Whole-map projection (note, fully projected maps are
+    // identical to projected simple columns at this level of
+    // abstraction.)
+
+    {
+      List<SchemaPath> projCols = new ArrayList<>();
+      projCols.add(SchemaPath.getSimplePath("map"));
+      ProjectionSet projSet = ProjectionSetImpl.parse(projCols);
+      assertTrue(projSet instanceof ProjectionSetImpl);
+      assertTrue(projSet.isProjected("map"));
+      assertFalse(projSet.isProjected("another"));
+      ProjectionSet mapProj = projSet.mapProjection("map");
+      assertNotNull(mapProj);
+      assertTrue(mapProj instanceof NullProjectionSet);
+      assertTrue(mapProj.isProjected("foo"));
+      assertNotNull(projSet.mapProjection("another"));
+      assertFalse(projSet.mapProjection("another").isProjected("anyCol"));
+    }
+
+    // Selected map projection, multiple levels, full projection
+    // at leaf level.
+
+    {
+      List<SchemaPath> projCols = new ArrayList<>();
+      projCols.add(SchemaPath.getCompoundPath("map", "a"));
+      projCols.add(SchemaPath.getCompoundPath("map", "b"));
+      projCols.add(SchemaPath.getCompoundPath("map", "map2", "x"));
+      ProjectionSet projSet = ProjectionSetImpl.parse(projCols);
+      assertTrue(projSet instanceof ProjectionSetImpl);
+      assertTrue(projSet.isProjected("map"));
+
+      // Map: an explicit map at top level
+
+      ProjectionSet mapProj = projSet.mapProjection("map");
+      assertTrue(mapProj instanceof ProjectionSetImpl);
+      assertTrue(mapProj.isProjected("a"));
+      assertTrue(mapProj.isProjected("b"));
+      assertTrue(mapProj.isProjected("map2"));
+      assertFalse(projSet.isProjected("bogus"));
+
+      // Map b: an implied nested map
+
+      ProjectionSet bMapProj = mapProj.mapProjection("b");
+      assertNotNull(bMapProj);
+      assertTrue(bMapProj instanceof NullProjectionSet);
+      assertTrue(bMapProj.isProjected("foo"));
+
+      // Map2, an nested map, has an explicit projection
+
+      ProjectionSet map2Proj = mapProj.mapProjection("map2");
+      assertNotNull(map2Proj);
+      assertTrue(map2Proj instanceof ProjectionSetImpl);
+      assertTrue(map2Proj.isProjected("x"));
+      assertFalse(map2Proj.isProjected("bogus"));
+    }
+  }
+
+  /**
+   * Test imposing a selection mask between the client and the underlying
+   * vector container.
+   */
+
+  @Test
+  public void testProjectionStatic() {
+    List<SchemaPath> selection = Lists.newArrayList(
+        SchemaPath.getSimplePath("c"),
+        SchemaPath.getSimplePath("b"),
+        SchemaPath.getSimplePath("e"));
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.INT)
+        .add("c", MinorType.INT)
+        .add("d", MinorType.INT)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setProjection(selection)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+
+    doProjectionTest(rsLoader);
+  }
+
+  @Test
+  public void testProjectionDynamic() {
+    List<SchemaPath> selection = Lists.newArrayList(
+        SchemaPath.getSimplePath("c"),
+        SchemaPath.getSimplePath("b"),
+        SchemaPath.getSimplePath("e"));
+    ResultSetOptions options = new OptionBuilder()
+        .setProjection(selection)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED));
+    rootWriter.addColumn(SchemaBuilder.columnSchema("b", MinorType.INT, DataMode.REQUIRED));
+    rootWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED));
+    rootWriter.addColumn(SchemaBuilder.columnSchema("d", MinorType.INT, DataMode.REQUIRED));
+
+    doProjectionTest(rsLoader);
+  }
+
+  private void doProjectionTest(ResultSetLoader rsLoader) {
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // All columns appear, including non-projected ones.
+
+    TupleMetadata actualSchema = rootWriter.schema();
+    assertEquals(4, actualSchema.size());
+    assertEquals("a", actualSchema.column(0).getName());
+    assertEquals("b", actualSchema.column(1).getName());
+    assertEquals("c", actualSchema.column(2).getName());
+    assertEquals("d", actualSchema.column(3).getName());
+    assertEquals(0, actualSchema.index("A"));
+    assertEquals(3, actualSchema.index("d"));
+    assertEquals(-1, actualSchema.index("e"));
+
+    // Non-projected columns identify themselves via metadata
+
+    assertFalse(actualSchema.metadata("a").isProjected());
+    assertTrue(actualSchema.metadata("b").isProjected());
+    assertTrue(actualSchema.metadata("c").isProjected());
+    assertFalse(actualSchema.metadata("d").isProjected());
+
+    // Write some data. Doesn't need much.
+
+    rsLoader.startBatch();
+    for (int i = 1; i < 3; i++) {
+      rootWriter.start();
+      rootWriter.scalar(0).setInt(i * 5);
+      rootWriter.scalar(1).setInt(i);
+      rootWriter.scalar(2).setInt(i * 10);
+      rootWriter.scalar(3).setInt(i * 20);
+      rootWriter.save();
+    }
+
+    // Verify. Result should only have the projected
+    // columns, only if defined by the loader, in the order
+    // of definition.
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("b", MinorType.INT)
+        .add("c", MinorType.INT)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(1, 10)
+        .addRow(2, 20)
+        .build();
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+//    actual.print();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(actual);
+    rsLoader.close();
+  }
+
+  @Test
+  public void testMapProjection() {
+    List<SchemaPath> selection = Lists.newArrayList(
+        SchemaPath.getSimplePath("m1"),
+        SchemaPath.getCompoundPath("m2", "d"));
+    TupleMetadata schema = new SchemaBuilder()
+        .addMap("m1")
+          .add("a", MinorType.INT)
+          .add("b", MinorType.INT)
+          .buildMap()
+        .addMap("m2")
+          .add("c", MinorType.INT)
+          .add("d", MinorType.INT)
+          .buildMap()
+        .addMap("m3")
+          .add("e", MinorType.INT)
+          .add("f", MinorType.INT)
+          .buildMap()
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setProjection(selection)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Verify the projected columns
+
+    TupleMetadata actualSchema = rootWriter.schema();
+    ColumnMetadata m1Md = actualSchema.metadata("m1");
+    assertTrue(m1Md.isMap());
+    assertTrue(m1Md.isProjected());
+    assertEquals(2, m1Md.mapSchema().size());
+    assertTrue(m1Md.mapSchema().metadata("a").isProjected());
+    assertTrue(m1Md.mapSchema().metadata("b").isProjected());
+
+    ColumnMetadata m2Md = actualSchema.metadata("m2");
+    assertTrue(m2Md.isMap());
+    assertTrue(m2Md.isProjected());
+    assertEquals(2, m2Md.mapSchema().size());
+    assertFalse(m2Md.mapSchema().metadata("c").isProjected());
+    assertTrue(m2Md.mapSchema().metadata("d").isProjected());
+
+    ColumnMetadata m3Md = actualSchema.metadata("m3");
+    assertTrue(m3Md.isMap());
+    assertFalse(m3Md.isProjected());
+    assertEquals(2, m3Md.mapSchema().size());
+    assertFalse(m3Md.mapSchema().metadata("e").isProjected());
+    assertFalse(m3Md.mapSchema().metadata("f").isProjected());
+
+    // Write a couple of rows.
+
+    rsLoader.startBatch();
+    rootWriter.start();
+    rootWriter.tuple("m1").scalar("a").setInt(1);
+    rootWriter.tuple("m1").scalar("b").setInt(2);
+    rootWriter.tuple("m2").scalar("c").setInt(3);
+    rootWriter.tuple("m2").scalar("d").setInt(4);
+    rootWriter.tuple("m3").scalar("e").setInt(5);
+    rootWriter.tuple("m3").scalar("f").setInt(6);
+    rootWriter.save();
+
+    rootWriter.start();
+    rootWriter.tuple("m1").scalar("a").setInt(11);
+    rootWriter.tuple("m1").scalar("b").setInt(12);
+    rootWriter.tuple("m2").scalar("c").setInt(13);
+    rootWriter.tuple("m2").scalar("d").setInt(14);
+    rootWriter.tuple("m3").scalar("e").setInt(15);
+    rootWriter.tuple("m3").scalar("f").setInt(16);
+    rootWriter.save();
+
+    // Verify. Only the projected columns appear in the result set.
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+      .addMap("m1")
+        .add("a", MinorType.INT)
+        .add("b", MinorType.INT)
+        .buildMap()
+      .addMap("m2")
+        .add("d", MinorType.INT)
+        .buildMap()
+      .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow(new Object[] {1, 2}, new Object[] {4})
+      .addRow(new Object[] {11, 12}, new Object[] {14})
+      .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(fixture.wrap(rsLoader.harvest()));
+    rsLoader.close();
+  }
+
+  /**
+   * Test a map array. Use the convenience methods to set values.
+   * Only the projected array members should appear in the harvested
+   * results.
+   */
+
+  @Test
+  public void testMapArrayProjection() {
+    List<SchemaPath> selection = Lists.newArrayList(
+        SchemaPath.getSimplePath("m1"),
+        SchemaPath.getCompoundPath("m2", "d"));
+    TupleMetadata schema = new SchemaBuilder()
+        .addMapArray("m1")
+          .add("a", MinorType.INT)
+          .add("b", MinorType.INT)
+          .buildMap()
+        .addMapArray("m2")
+          .add("c", MinorType.INT)
+          .add("d", MinorType.INT)
+          .buildMap()
+        .addMapArray("m3")
+          .add("e", MinorType.INT)
+          .add("f", MinorType.INT)
+          .buildMap()
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setProjection(selection)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Write a couple of rows.
+
+    rsLoader.startBatch();
+    rootWriter.addRow(
+        new Object[] { new Object[] {10, 20}, new Object[] {11, 21}},
+        new Object[] { new Object[] {30, 40}, new Object[] {31, 42}},
+        new Object[] { new Object[] {50, 60}, new Object[] {51, 62}});
+    rootWriter.addRow(
+        new Object[] { new Object[] {110, 120}, new Object[] {111, 121}},
+        new Object[] { new Object[] {130, 140}, new Object[] {131, 142}},
+        new Object[] { new Object[] {150, 160}, new Object[] {151, 162}});
+
+    // Verify. Only the projected columns appear in the result set.
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+      .addMapArray("m1")
+        .add("a", MinorType.INT)
+        .add("b", MinorType.INT)
+        .buildMap()
+      .addMapArray("m2")
+        .add("d", MinorType.INT)
+        .buildMap()
+      .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+      .addRow(
+          new Object[] { new Object[] {10, 20}, new Object[] {11, 21}},
+          new Object[] { new Object[] {40}, new Object[] {42}})
+      .addRow(
+          new Object[] { new Object[] {110, 120}, new Object[] {111, 121}},
+          new Object[] { new Object[] {140}, new Object[] {142}})
+      .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(fixture.wrap(rsLoader.harvest()));
+    rsLoader.close();
+  }
+
+  /**
+   * Verify that the projection code plays nice with vector overflow. Overflow
+   * is the most complex operation in this subsystem with many specialized
+   * methods that must work together flawlessly. This test ensures that
+   * non-projected columns stay in the background and don't interfere
+   * with overflow logic.
+   */
+
+  @Test
+  public void testProjectWithOverflow() {
+    List<SchemaPath> selection = Lists.newArrayList(
+        SchemaPath.getSimplePath("small"),
+        SchemaPath.getSimplePath("dummy"));
+    TupleMetadata schema = new SchemaBuilder()
+        .add("big", MinorType.VARCHAR)
+        .add("small", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setProjection(selection)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    byte big[] = new byte[600];
+    Arrays.fill(big, (byte) 'X');
+    byte small[] = new byte[512];
+    Arrays.fill(small, (byte) 'X');
+
+    rsLoader.startBatch();
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(big, big.length);
+      rootWriter.scalar(1).setBytes(small, small.length);
+      rootWriter.save();
+      count++;
+    }
+
+    // Number of rows should be driven by size of the
+    // projected vector ("small"), not by the larger, unprojected
+    // "big" vector.
+    // Our row count should include the overflow row
+
+    int expectedCount = ValueVector.MAX_BUFFER_SIZE / small.length;
+    assertEquals(expectedCount + 1, count);
+
+    // Loader's row count should include only "visible" rows
+
+    assertEquals(expectedCount, rootWriter.rowCount());
+
+    // Total count should include invisible and look-ahead rows.
+
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+
+    // Result should exclude the overflow row
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(expectedCount, result.rowCount());
+    result.clear();
+
+    // Next batch should start with the overflow row
+
+    rsLoader.startBatch();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(expectedCount + 1, rsLoader.totalRowCount());
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(1, result.rowCount());
+    result.clear();
+
+    rsLoader.close();
+  }
+}


[14/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
new file mode 100644
index 0000000..b875e7e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultSetLoaderImpl.java
@@ -0,0 +1,775 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.Collection;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.TupleState.RowState;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Implementation of the result set loader.
+ * @see {@link ResultSetLoader}
+ */
+
+public class ResultSetLoaderImpl implements ResultSetLoader {
+
+  /**
+   * Read-only set of options for the result set loader.
+   */
+
+  public static class ResultSetOptions {
+    public final int vectorSizeLimit;
+    public final int rowCountLimit;
+    public final ResultVectorCache vectorCache;
+    public final Collection<SchemaPath> projection;
+    public final TupleMetadata schema;
+    public final long maxBatchSize;
+
+    public ResultSetOptions() {
+      vectorSizeLimit = ValueVector.MAX_BUFFER_SIZE;
+      rowCountLimit = DEFAULT_ROW_COUNT;
+      projection = null;
+      vectorCache = null;
+      schema = null;
+      maxBatchSize = -1;
+    }
+
+    public ResultSetOptions(OptionBuilder builder) {
+      this.vectorSizeLimit = builder.vectorSizeLimit;
+      this.rowCountLimit = builder.rowCountLimit;
+      this.projection = builder.projection;
+      this.vectorCache = builder.vectorCache;
+      this.schema = builder.schema;
+      this.maxBatchSize = builder.maxBatchSize;
+    }
+
+    public void dump(HierarchicalFormatter format) {
+      format
+        .startObject(this)
+        .attribute("vectorSizeLimit", vectorSizeLimit)
+        .attribute("rowCountLimit", rowCountLimit)
+        .attribute("projection", projection)
+        .endObject();
+    }
+  }
+
+  private enum State {
+    /**
+     * Before the first batch.
+     */
+
+    START,
+
+    /**
+     * Writing to a batch normally.
+     */
+
+    ACTIVE,
+
+    /**
+     * Batch overflowed a vector while writing. Can continue
+     * to write to a temporary "overflow" batch until the
+     * end of the current row.
+     */
+
+    OVERFLOW,
+
+    /**
+     * Temporary state to avoid batch-size related overflow while
+     * an overflow is in progress.
+     */
+
+    IN_OVERFLOW,
+
+    /**
+     * Batch is full due to reaching the row count limit
+     * when saving a row.
+     * No more writes allowed until harvesting the current batch.
+     */
+
+    FULL_BATCH,
+
+    /**
+     * Current batch was harvested: data is gone. No lookahead
+     * batch exists.
+     */
+
+    HARVESTED,
+
+    /**
+     * Current batch was harvested and its data is gone. However,
+     * overflow occurred during that batch and the data exists
+     * in the overflow vectors.
+     * <p>
+     * This state needs special consideration. The column writer
+     * structure maintains its state (offsets, etc.) from the OVERFLOW
+     * state, but the buffers currently in the vectors are from the
+     * complete batch. <b>No writes can be done in this state!</b>
+     * The writer state does not match the data in the buffers.
+     * The code here does what it can to catch this state. But, if
+     * some client tries to write to a column writer in this state,
+     * bad things will happen. Doing so is invalid (the write is outside
+     * of a batch), so this is not a terrible restriction.
+     * <p>
+     * Said another way, the current writer state is invalid with respect
+     * to the active buffers, but only if the writers try to act on the
+     * buffers. Since the writers won't do so, this temporary state is
+     * fine. The correct buffers are restored once a new batch is started
+     * and the state moves to ACTIVE.
+     */
+
+    LOOK_AHEAD,
+
+    /**
+     * Mutator is closed: no more operations are allowed.
+     */
+
+    CLOSED
+  }
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResultSetLoaderImpl.class);
+
+  /**
+   * Options provided to this loader.
+   */
+
+  private final ResultSetOptions options;
+
+  /**
+   * Allocator for vectors created by this loader.
+   */
+
+  final BufferAllocator allocator;
+
+  /**
+   * Internal structure used to work with the vectors (real or dummy) used
+   * by this loader.
+   */
+
+  final RowState rootState;
+
+  /**
+   * Top-level writer index that steps through the rows as they are written.
+   * When an overflow batch is in effect, indexes into that batch instead.
+   * Since a batch is really a tree of tuples, in which some branches of
+   * the tree are arrays, the root indexes here feeds into array indexes
+   * within the writer structure that points to the current position within
+   * an array column.
+   */
+
+  private final WriterIndexImpl writerIndex;
+
+  /**
+   * The row-level writer for stepping through rows as they are written,
+   * and for accessing top-level columns.
+   */
+
+  private final RowSetLoaderImpl rootWriter;
+
+  /**
+   * Vector cache for this loader.
+   * @see {@link OptionBuilder#setVectorCache()}.
+   */
+
+  private final ResultVectorCache vectorCache;
+
+  /**
+   * Tracks the state of the row set loader. Handling vector overflow requires
+   * careful stepping through a variety of states as the write proceeds.
+   */
+
+  private State state = State.START;
+
+  /**
+   * Track the current schema as seen by the writer. Each addition of a column
+   * anywhere in the schema causes the active schema version to increase by one.
+   * This allows very easy checks for schema changes: save the prior version number
+   * and compare it against the current version number.
+   */
+
+  private int activeSchemaVersion;
+
+  /**
+   * Track the current schema as seen by the consumer of the batches that this
+   * loader produces. The harvest schema version can be behind the active schema
+   * version in the case in which new columns are added to the overflow row.
+   * Since the overflow row won't be visible to the harvested batch, that batch
+   * sees the schema as it existed at a prior version: the harvest schema
+   * version.
+   */
+
+  private int harvestSchemaVersion;
+
+  /**
+   * Builds the harvest vector container that includes only the columns that
+   * are included in the harvest schema version. That is, it excludes columns
+   * added while writing the overflow row.
+   */
+
+  private VectorContainerBuilder containerBuilder;
+
+  /**
+   * Counts the batches harvested (sent downstream) from this loader. Does
+   * not include the current, in-flight batch.
+   */
+
+  private int harvestBatchCount;
+
+  /**
+   * Counts the rows included in previously-harvested batches. Does not
+   * include the number of rows in the current batch.
+   */
+
+  private int previousRowCount;
+
+  /**
+   * Number of rows in the harvest batch. If an overflow batch is in effect,
+   * then this is the number of rows in the "main" batch before the overflow;
+   * that is the number of rows in the batch that will be harvested. If no
+   * overflow row is in effect, then this number is undefined (and should be
+   * zero.)
+   */
+
+  private int pendingRowCount;
+
+  /**
+   * The number of rows per batch. Starts with the configured amount. Can be
+   * adjusted between batches, perhaps based on the actual observed size of
+   * input data.
+   */
+
+  private int targetRowCount;
+
+  /**
+   * Total bytes allocated to the current batch.
+   */
+
+  protected int accumulatedBatchSize;
+
+  protected final ProjectionSet projectionSet;
+
+  public ResultSetLoaderImpl(BufferAllocator allocator, ResultSetOptions options) {
+    this.allocator = allocator;
+    this.options = options;
+    targetRowCount = options.rowCountLimit;
+    writerIndex = new WriterIndexImpl(this);
+
+    if (options.vectorCache == null) {
+      vectorCache = new NullResultVectorCacheImpl(allocator);
+    } else {
+      vectorCache = options.vectorCache;
+    }
+
+    // If projection, build the projection map.
+
+    projectionSet = ProjectionSetImpl.parse(options.projection);
+
+    // Build the row set model depending on whether a schema is provided.
+
+    rootState = new RowState(this);
+    rootWriter = rootState.rootWriter();
+
+    // If no schema, columns will be added incrementally as they
+    // are discovered. Start with an empty model.
+
+    if (options.schema != null) {
+
+      // Schema provided. Populate a model (and create vectors) for the
+      // provided schema. The schema can be extended later, but normally
+      // won't be if known up front.
+
+      logger.debug("Schema: " + options.schema.toString());
+      rootState.buildSchema(options.schema);
+    }
+  }
+
+  private void updateCardinality() {
+    rootState.updateCardinality(targetRowCount());
+  }
+
+  public ResultSetLoaderImpl(BufferAllocator allocator) {
+    this(allocator, new ResultSetOptions());
+  }
+
+  public BufferAllocator allocator() { return allocator; }
+
+  protected int bumpVersion() {
+
+    // Update the active schema version. We cannot update the published
+    // schema version at this point because a column later in this same
+    // row might cause overflow, and any new columns in this row will
+    // be hidden until a later batch. But, if we are between batches,
+    // then it is fine to add the column to the schema.
+
+    activeSchemaVersion++;
+    switch (state) {
+    case HARVESTED:
+    case START:
+    case LOOK_AHEAD:
+      harvestSchemaVersion = activeSchemaVersion;
+      break;
+    default:
+      break;
+
+    }
+    return activeSchemaVersion;
+  }
+
+  @Override
+  public int schemaVersion() { return harvestSchemaVersion; }
+
+  @Override
+  public void startBatch() {
+    switch (state) {
+    case HARVESTED:
+    case START:
+      logger.trace("Start batch");
+      accumulatedBatchSize = 0;
+      updateCardinality();
+      rootState.startBatch();
+      checkInitialAllocation();
+
+      // The previous batch ended without overflow, so start
+      // a new batch, and reset the write index to 0.
+
+      writerIndex.reset();
+      rootWriter.startWrite();
+      break;
+
+    case LOOK_AHEAD:
+
+      // A row overflowed so keep the writer index at its current value
+      // as it points to the second row in the overflow batch. However,
+      // the last write position of each writer must be restored on
+      // a column-by-column basis, which is done by the visitor.
+
+      logger.trace("Start batch after overflow");
+      rootState.startBatch();
+
+      // Note: no need to do anything with the writers; they were left
+      // pointing to the correct positions in the look-ahead batch.
+      // The above simply puts the look-ahead vectors back "under"
+      // the writers.
+
+      break;
+
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+
+    // Update the visible schema with any pending overflow batch
+    // updates.
+
+    harvestSchemaVersion = activeSchemaVersion;
+    pendingRowCount = 0;
+    state = State.ACTIVE;
+  }
+
+  @Override
+  public RowSetLoader writer() {
+    if (state == State.CLOSED) {
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+    return rootWriter;
+  }
+
+  @Override
+  public ResultSetLoader setRow(Object... values) {
+    startRow();
+    writer().setTuple(values);
+    saveRow();
+    return this;
+  }
+
+  /**
+   * Called before writing a new row. Implementation of
+   * {@link RowSetLoader#start()}.
+   */
+
+  protected void startRow() {
+    switch (state) {
+    case ACTIVE:
+
+      // Update the visible schema with any pending overflow batch
+      // updates.
+
+      harvestSchemaVersion = activeSchemaVersion;
+      rootWriter.startRow();
+      break;
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+  }
+
+  /**
+   * Finalize the current row. Implementation of
+   * {@link RowSetLoader#save()}.
+   */
+
+  protected void saveRow() {
+    switch (state) {
+    case ACTIVE:
+      rootWriter.endArrayValue();
+      rootWriter.saveRow();
+      if (! writerIndex.next()) {
+        state = State.FULL_BATCH;
+      }
+
+      // No overflow row. Advertise the schema version to the client.
+
+      harvestSchemaVersion = activeSchemaVersion;
+      break;
+
+    case OVERFLOW:
+
+      // End the value of the look-ahead row in the look-ahead vectors.
+
+      rootWriter.endArrayValue();
+      rootWriter.saveRow();
+
+      // Advance the writer index relative to the look-ahead batch.
+
+      writerIndex.next();
+
+      // Stay in the overflow state. Doing so will cause the writer
+      // to report that it is full.
+      //
+      // Also, do not change the harvest schema version. We will
+      // expose to the downstream operators the schema in effect
+      // at the start of the row. Columns added within the row won't
+      // appear until the next batch.
+
+      break;
+
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+  }
+
+  /**
+   * Implementation of {@link RowSetLoader#isFull()}
+   * @return true if the batch is full (reached vector capacity or the
+   * row count limit), false if more rows can be added
+   */
+
+  protected boolean isFull() {
+    switch (state) {
+    case ACTIVE:
+      return ! writerIndex.valid();
+    case OVERFLOW:
+    case FULL_BATCH:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  @Override
+  public boolean writeable() {
+    return state == State.ACTIVE || state == State.OVERFLOW;
+  }
+
+  private boolean isBatchActive() {
+    return state == State.ACTIVE || state == State.OVERFLOW ||
+           state == State.FULL_BATCH ;
+  }
+
+  /**
+   * Implementation for {#link {@link RowSetLoader#rowCount()}.
+   *
+   * @return the number of rows to be sent downstream for this
+   * batch. Does not include the overflow row.
+   */
+
+  protected int rowCount() {
+    switch (state) {
+    case ACTIVE:
+    case FULL_BATCH:
+      return writerIndex.size();
+    case OVERFLOW:
+      return pendingRowCount;
+    default:
+      return 0;
+    }
+  }
+
+  protected WriterIndexImpl writerIndex() { return writerIndex; }
+
+  @Override
+  public void setTargetRowCount(int rowCount) {
+    targetRowCount = Math.max(1, rowCount);
+  }
+
+  @Override
+  public int targetRowCount() { return targetRowCount; }
+
+  @Override
+  public int targetVectorSize() { return options.vectorSizeLimit; }
+
+  protected void overflowed() {
+    logger.trace("Vector overflow");
+
+    // If we see overflow when we are already handling overflow, it means
+    // that a single value is too large to fit into an entire vector.
+    // Fail the query.
+    //
+    // Note that this is a judgment call. It is possible to allow the
+    // vector to double beyond the limit, but that will require a bit
+    // of thought to get right -- and, of course, completely defeats
+    // the purpose of limiting vector size to avoid memory fragmentation...
+    //
+    // Individual columns handle the case in which overflow occurs on the
+    // first row of the main batch. This check handles the pathological case
+    // in which we successfully overflowed, but then another column
+    // overflowed during the overflow row -- that indicates that that one
+    // column can't fit in an empty vector. That is, this check is for a
+    // second-order overflow.
+
+    if (state == State.OVERFLOW) {
+      throw UserException
+          .memoryError("A single column value is larger than the maximum allowed size of 16 MB")
+          .build(logger);
+    }
+    if (state != State.ACTIVE) {
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+    state = State.IN_OVERFLOW;
+
+    // Preserve the number of rows in the now-complete batch.
+
+    pendingRowCount = writerIndex.vectorIndex();
+
+    // Roll-over will allocate new vectors. Update with the latest
+    // array cardinality.
+
+    updateCardinality();
+
+//    rootWriter.dump(new HierarchicalPrinter());
+
+    // Wrap up the completed rows into a batch. Sets
+    // vector value counts. The rollover data still exists so
+    // it can be moved, but it is now past the recorded
+    // end of the vectors (though, obviously, not past the
+    // physical end.)
+
+    rootWriter.preRollover();
+
+    // Roll over vector values.
+
+    accumulatedBatchSize = 0;
+    rootState.rollover();
+
+    // Adjust writer state to match the new vector values. This is
+    // surprisingly easy if we not that the current row is shifted to
+    // the 0 position in the new vector, so we just shift all offsets
+    // downward by the current row position at each repeat level.
+
+    rootWriter.postRollover();
+
+    // The writer index is reset back to 0. Because of the above roll-over
+    // processing, some vectors may now already have values in the 0 slot.
+    // However, the vector that triggered overflow has not yet written to
+    // the current record, and so will now write to position 0. After the
+    // completion of the row, all 0-position values should be written (or
+    // at least those provided by the client.)
+    //
+    // For arrays, the writer might have written a set of values
+    // (v1, v2, v3), and v4 might have triggered the overflow. In this case,
+    // the array values have been moved, offset vectors adjusted, the
+    // element writer adjusted, so that v4 will be written to index 3
+    // to produce (v1, v2, v3, v4, v5, ...) in the look-ahead vector.
+
+    writerIndex.rollover();
+    checkInitialAllocation();
+
+    // Remember that overflow is in effect.
+
+    state = State.OVERFLOW;
+  }
+
+  protected boolean hasOverflow() { return state == State.OVERFLOW; }
+
+  @Override
+  public VectorContainer harvest() {
+    int rowCount;
+    switch (state) {
+    case ACTIVE:
+    case FULL_BATCH:
+      rowCount = harvestNormalBatch();
+      logger.trace("Harvesting {} rows", rowCount);
+      break;
+    case OVERFLOW:
+      rowCount = harvestOverflowBatch();
+      logger.trace("Harvesting {} rows after overflow", rowCount);
+      break;
+    default:
+      throw new IllegalStateException("Unexpected state: " + state);
+    }
+
+    // Build the output container
+
+    VectorContainer container = outputContainer();
+    container.setRecordCount(rowCount);
+
+    // Finalize: update counts, set state.
+
+    harvestBatchCount++;
+    previousRowCount += rowCount;
+    return container;
+  }
+
+  private int harvestNormalBatch() {
+
+    // Wrap up the vectors: final fill-in, set value count, etc.
+
+    rootWriter.endBatch();
+    harvestSchemaVersion = activeSchemaVersion;
+    state = State.HARVESTED;
+    return writerIndex.size();
+  }
+
+  private int harvestOverflowBatch() {
+    rootState.harvestWithLookAhead();
+    state = State.LOOK_AHEAD;
+    return pendingRowCount;
+  }
+
+  @Override
+  public VectorContainer outputContainer() {
+    // Build the output container.
+
+    if (containerBuilder == null) {
+      containerBuilder = new VectorContainerBuilder(this);
+    }
+    containerBuilder.update(harvestSchemaVersion);
+    return containerBuilder.container();
+  }
+
+  @Override
+  public TupleMetadata harvestSchema() {
+    return containerBuilder.schema();
+  }
+
+  @Override
+  public void close() {
+    if (state == State.CLOSED) {
+      return;
+    }
+    rootState.close();
+
+    // Do not close the vector cache; the caller owns that and
+    // will, presumably, reuse those vectors for another writer.
+
+    state = State.CLOSED;
+  }
+
+  @Override
+  public int batchCount() {
+    return harvestBatchCount + (rowCount() == 0 ? 0 : 1);
+  }
+
+  @Override
+  public int totalRowCount() {
+    int total = previousRowCount;
+    if (isBatchActive()) {
+      total += pendingRowCount + writerIndex.size();
+    }
+    return total;
+  }
+
+  public ResultVectorCache vectorCache() { return vectorCache; }
+  public RowState rootState() { return rootState; }
+
+  /**
+   * Return whether a vector within the current batch can expand. Limits
+   * are enforce only if a limit was provided in the options.
+   *
+   * @param delta increase in vector size
+   * @return true if the vector can expand, false if an overflow
+   * event should occur
+   */
+
+  public boolean canExpand(int delta) {
+    accumulatedBatchSize += delta;
+    return state == State.IN_OVERFLOW ||
+           options.maxBatchSize <= 0 ||
+           accumulatedBatchSize <= options.maxBatchSize;
+  }
+
+  /**
+   * Accumulate the initial vector allocation sizes.
+   *
+   * @param allocationBytes number of bytes allocated to a vector
+   * in the batch setup step
+   */
+
+  public void tallyAllocations(int allocationBytes) {
+    accumulatedBatchSize += allocationBytes;
+  }
+
+  /**
+   * Log and check the initial vector allocation. If a batch size
+   * limit is set, warn if the initial allocation exceeds the limit.
+   * This will occur if the target row count is incorrect for the
+   * data size.
+   */
+
+  private void checkInitialAllocation() {
+    if (options.maxBatchSize < 0) {
+      logger.debug("Initial vector allocation: {}, no batch limit specified",
+          accumulatedBatchSize);
+    }
+    else if (accumulatedBatchSize > options.maxBatchSize) {
+      logger.warn("Initial vector allocation: {}, but batch size limit is: {}",
+          accumulatedBatchSize, options.maxBatchSize);
+    } else {
+      logger.debug("Initial vector allocation: {}, batch size limit: {}",
+          accumulatedBatchSize, options.maxBatchSize);
+    }
+  }
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("options");
+    options.dump(format);
+    format
+      .attribute("index", writerIndex.vectorIndex())
+      .attribute("state", state)
+      .attribute("activeSchemaVersion", activeSchemaVersion)
+      .attribute("harvestSchemaVersion", harvestSchemaVersion)
+      .attribute("pendingRowCount", pendingRowCount)
+      .attribute("targetRowCount", targetRowCount)
+      ;
+    format.attribute("root");
+    rootState.dump(format);
+    format.attribute("rootWriter");
+    rootWriter.dump(format);
+    format.endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCacheImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCacheImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCacheImpl.java
new file mode 100644
index 0000000..c7288b2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/ResultVectorCacheImpl.java
@@ -0,0 +1,186 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.ResultVectorCache;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.ValueVector;
+
+/**
+ * Manages an inventory of value vectors used across row batch readers.
+ * Drill semantics for batches is complex. Each operator logically returns
+ * a batch of records on each call of the Drill Volcano iterator protocol
+ * <tt>next()</tt> operation. However, the batches "returned" are not
+ * separate objects. Instead, Drill enforces the following semantics:
+ * <ul>
+ * <li>If a <tt>next()</tt> call returns <tt>OK</tt> then the set of vectors
+ * in the "returned" batch must be identical to those in the prior batch. Not
+ * just the same type; they must be the same <tt>ValueVector</tt> objects.
+ * (The buffers within the vectors will be different.)</li>
+ * <li>If the set of vectors changes in any way (add a vector, remove a
+ * vector, change the type of a vector), then the <tt>next()</tt> call
+ * <b>must</b> return <tt>OK_NEW_SCHEMA</tt>.</ul>
+ * </ul>
+ * These rules create interesting constraints for the scan operator.
+ * Conceptually, each batch is distinct. But, it must share vectors. The
+ * {@link ResultSetLoader} class handles this by managing the set of vectors
+ * used by a single reader.
+ * <p>
+ * Readers are independent: each may read a distinct schema (as in JSON.)
+ * Yet, the Drill protocol requires minimizing spurious <tt>OK_NEW_SCHEMA</tt>
+ * events. As a result, two readers run by the same scan operator must
+ * share the same set of vectors, despite the fact that they may have
+ * different schemas and thus different <tt>ResultSetLoader</tt>s.
+ * <p>
+ * The purpose of this inventory is to persist vectors across readers, even
+ * when, say, reader B does not use a vector that reader A created.
+ * <p>
+ * The semantics supported by this class include:
+ * <ul>
+ * <li>Ability to "pre-declare" columns based on columns that appear in
+ * an explicit select list. This ensures that the columns are known (but
+ * not their types).</li>
+ * <li>Ability to reuse a vector across readers if the column retains the same
+ * name and type (minor type and mode.)</li>
+ * <li>Ability to flush unused vectors for readers with changing schemas
+ * if a schema change occurs.</li>
+ * <li>Support schema "hysteresis"; that is, the a "sticky" schema that
+ * minimizes spurious changes. Once a vector is declared, it can be included
+ * in all subsequent batches (provided the column is nullable or an array.)</li>
+ * </ul>
+ */
+public class ResultVectorCacheImpl implements ResultVectorCache {
+
+  /**
+   * State of a projected vector. At first all we have is a name.
+   * Later, we'll discover the type.
+   */
+
+  private static class VectorState {
+    protected final String name;
+    protected ValueVector vector;
+    protected boolean touched;
+
+    public VectorState(String name) {
+      this.name = name;
+    }
+
+    public boolean satisfies(MaterializedField colSchema) {
+      if (vector == null) {
+        return false;
+      }
+      MaterializedField vectorSchema = vector.getField();
+      return vectorSchema.getType().equals(colSchema.getType());
+    }
+  }
+
+  private final BufferAllocator allocator;
+  private final Map<String, VectorState> vectors = new HashMap<>();
+
+  public ResultVectorCacheImpl(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  @Override
+  public BufferAllocator allocator() { return allocator; }
+
+  public void predefine(List<String> selected) {
+    for (String colName : selected) {
+      addVector(colName);
+    }
+  }
+
+  private VectorState addVector(String colName) {
+    VectorState vs = new VectorState(colName);
+    vectors.put(vs.name, vs);
+    return vs;
+  }
+
+  public void newBatch() {
+    for (VectorState vs : vectors.values()) {
+      vs.touched = false;
+    }
+  }
+
+  public void trimUnused() {
+    List<VectorState> unused = new ArrayList<>();
+    for (VectorState vs : vectors.values()) {
+      if (! vs.touched) {
+        unused.add(vs);
+      }
+    }
+    if (unused.isEmpty()) {
+      return;
+    }
+    for (VectorState vs : unused) {
+      vectors.remove(vs.name);
+    }
+  }
+
+  @Override
+  public ValueVector addOrGet(MaterializedField colSchema) {
+    VectorState vs = vectors.get(colSchema.getName());
+
+    // If the vector is found, and is of the right type, reuse it.
+
+    if (vs != null && vs.satisfies(colSchema)) {
+      return vs.vector;
+    }
+
+    // If no vector, this is a late schema. Create the vector.
+
+    if (vs == null) {
+      vs = addVector(colSchema.getName());
+
+    // Else, if the vector changed type, close the old one.
+
+    } else if (vs.vector != null) {
+      vs.vector.close();
+      vs.vector = null;
+    }
+
+    // Create the new vector.
+
+    vs.touched = true;
+    vs.vector = TypeHelper.getNewVector(colSchema, allocator, null);
+    return vs.vector;
+  }
+
+  public MajorType getType(String name) {
+    VectorState vs = vectors.get(name);
+    if (vs == null || vs.vector == null) {
+      return null;
+    }
+    return vs.vector.getField().getType();
+  }
+
+  public void close() {
+    for (VectorState vs : vectors.values()) {
+      vs.vector.close();
+    }
+    vectors.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.java
new file mode 100644
index 0000000..ec61ae7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/RowSetLoaderImpl.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.physical.rowSet.impl;
+
+import java.util.ArrayList;
+
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
+
+/**
+ * Implementation of the row set loader. Provides row-level operations, leaving the
+ * result set loader to provide batch-level operations. However, all control
+ * operations are actually delegated to the result set loader, which handles
+ * the details of working with overflow rows.
+ */
+
+public class RowSetLoaderImpl extends AbstractTupleWriter implements RowSetLoader {
+
+  private final ResultSetLoaderImpl rsLoader;
+
+  protected RowSetLoaderImpl(ResultSetLoaderImpl rsLoader, TupleMetadata schema) {
+    super(schema, new ArrayList<AbstractObjectWriter>());
+    this.rsLoader = rsLoader;
+    bindIndex(rsLoader.writerIndex());
+  }
+
+  @Override
+  public ResultSetLoader loader() { return rsLoader; }
+
+  @Override
+  public RowSetLoader addRow(Object...values) {
+    if (! start()) {
+      throw new IllegalStateException("Batch is full.");
+    }
+    setObject(values);
+    save();
+    return this;
+  }
+
+  @Override
+  public int rowIndex() { return rsLoader.writerIndex().vectorIndex(); }
+
+  @Override
+  public void save() { rsLoader.saveRow(); }
+
+  @Override
+  public boolean start() {
+    if (rsLoader.isFull()) {
+
+      // Full batch? Return false.
+
+      return false;
+    } else if (state == State.IN_ROW) {
+
+      // Already in a row? Rewind the to start of the row.
+
+      restartRow();
+    } else {
+
+      // Otherwise, advance to the next row.
+
+      rsLoader.startRow();
+    }
+    return true;
+  }
+
+  public void endBatch() {
+    if (state == State.IN_ROW) {
+      restartRow();
+      state = State.IN_WRITE;
+    }
+    endWrite();
+  }
+
+  @Override
+  public boolean isFull( ) { return rsLoader.isFull(); }
+
+  @Override
+  public int rowCount() { return rsLoader.rowCount(); }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
new file mode 100644
index 0000000..f6bc5f3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/SingleVectorState.java
@@ -0,0 +1,274 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VariableWidthVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.apache.drill.exec.vector.accessor.writer.OffsetVectorWriter;
+
+/**
+ * Base class for a single vector. Handles the bulk of work for that vector.
+ * Subclasses are specialized for offset vectors or values vectors.
+ * (The "single vector" name contrasts with classes that manage compound
+ * vectors, such as a data and offsets vector.)
+ */
+
+public abstract class SingleVectorState implements VectorState {
+
+  /**
+   * State for a scalar value vector. The vector might be for a simple (non-array)
+   * vector, or might be the payload part of a scalar array (repeated scalar)
+   * vector.
+   */
+
+  public static class ValuesVectorState extends SingleVectorState {
+
+    private final ColumnMetadata schema;
+
+    public ValuesVectorState(ColumnMetadata schema, AbstractScalarWriter writer, ValueVector mainVector) {
+      super(writer, mainVector);
+      this.schema = schema;
+    }
+
+    @Override
+    public int allocateVector(ValueVector vector, int cardinality) {
+      if (schema.isVariableWidth()) {
+
+        // Cap the allocated size to the maximum.
+
+        int size = (int) Math.min(ValueVector.MAX_BUFFER_SIZE, (long) cardinality * schema.expectedWidth());
+        ((VariableWidthVector) vector).allocateNew(size, cardinality);
+      } else {
+        ((FixedWidthVector) vector).allocateNew(cardinality);
+      }
+      return vector.getBufferSize();
+    }
+
+    @Override
+    protected void copyOverflow(int sourceStartIndex, int sourceEndIndex) {
+      int newIndex = 0;
+      ResultSetLoaderImpl.logger.trace("Vector {} of type {}: copy {} values from {} to {}",
+          mainVector.getField().toString(),
+          mainVector.getClass().getSimpleName(),
+          Math.max(0, sourceEndIndex - sourceStartIndex + 1),
+          sourceStartIndex, newIndex);
+
+      // Copy overflow values from the full vector to the new
+      // look-ahead vector. Uses vector-level operations for convenience.
+      // These aren't very efficient, but overflow does not happen very
+      // often.
+
+      for (int src = sourceStartIndex; src <= sourceEndIndex; src++, newIndex++) {
+        mainVector.copyEntry(newIndex, backupVector, src);
+      }
+    }
+  }
+
+  /**
+   * Special case for an offset vector. Offset vectors are managed like any other
+   * vector with respect to overflow and allocation. This means that the loader
+   * classes avoid the use of the RepeatedVector class methods, instead working
+   * with the offsets vector (here) or the values vector to allow the needed
+   * fine control over overflow operations.
+   */
+
+  public static class OffsetVectorState extends SingleVectorState {
+
+    private final AbstractObjectWriter childWriter;
+
+    public OffsetVectorState(AbstractScalarWriter writer, ValueVector mainVector,
+        AbstractObjectWriter childWriter) {
+      super(writer, mainVector);
+      this.childWriter = childWriter;
+    }
+
+    @Override
+    public int allocateVector(ValueVector toAlloc, int cardinality) {
+      ((UInt4Vector) toAlloc).allocateNew(cardinality);
+      return toAlloc.getBufferSize();
+    }
+
+    public int rowStartOffset() {
+      return ((OffsetVectorWriter) writer).rowStartOffset();
+    }
+
+    @Override
+    protected void copyOverflow(int sourceStartIndex, int sourceEndIndex) {
+
+      if (sourceStartIndex > sourceEndIndex) {
+        return;
+      }
+
+      // This is an offset vector. The data to copy is one greater
+      // than the row index.
+
+      sourceStartIndex++;
+      sourceEndIndex++;
+
+      // Copy overflow values from the full vector to the new
+      // look-ahead vector. Since this is an offset vector, values must
+      // be adjusted as they move across.
+      //
+      // Indexing can be confusing. Offset vectors have values offset
+      // from their row by one position. The offset vector position for
+      // row i has the start value for row i. The offset vector position for
+      // i+1 has the start of the next value. The difference between the
+      // two is the element length. As a result, the offset vector always has
+      // one more value than the number of rows, and position 0 is always 0.
+      //
+      // The index passed in here is that of the row that overflowed. That
+      // offset vector position contains the offset of the start of the data
+      // for the current row. We must subtract that offset from each copied
+      // value to adjust the offset for the destination.
+
+      UInt4Vector.Accessor sourceAccessor = ((UInt4Vector) backupVector).getAccessor();
+      UInt4Vector.Mutator destMutator = ((UInt4Vector) mainVector).getMutator();
+      int offset = childWriter.events().writerIndex().rowStartIndex();
+      int newIndex = 1;
+      ResultSetLoaderImpl.logger.trace("Offset vector: copy {} values from {} to {} with offset {}",
+          Math.max(0, sourceEndIndex - sourceStartIndex + 1),
+          sourceStartIndex, newIndex, offset);
+      assert offset == sourceAccessor.get(sourceStartIndex - 1);
+
+      // Position zero is special and will be filled in by the writer
+      // later.
+
+      for (int src = sourceStartIndex; src <= sourceEndIndex; src++, newIndex++) {
+        destMutator.set(newIndex, sourceAccessor.get(src) - offset);
+      }
+//      VectorPrinter.printOffsets((UInt4Vector) backupVector, sourceStartIndex - 1, sourceEndIndex - sourceStartIndex + 3);
+//      VectorPrinter.printOffsets((UInt4Vector) mainVector, 0, newIndex);
+    }
+  }
+
+  protected final AbstractScalarWriter writer;
+  protected final ValueVector mainVector;
+  protected ValueVector backupVector;
+
+  public SingleVectorState(AbstractScalarWriter writer, ValueVector mainVector) {
+    this.writer = writer;
+    this.mainVector = mainVector;
+  }
+
+  @Override
+  public ValueVector vector() { return mainVector; }
+
+  @Override
+  public int allocate(int cardinality) {
+    return allocateVector(mainVector, cardinality);
+  }
+
+  protected abstract int allocateVector(ValueVector vector, int cardinality);
+
+  /**
+   * A column within the row batch overflowed. Prepare to absorb the rest of
+   * the in-flight row by rolling values over to a new vector, saving the
+   * complete vector for later. This column could have a value for the overflow
+   * row, or for some previous row, depending on exactly when and where the
+   * overflow occurs.
+   *
+   * @param sourceStartIndex the index of the row that caused the overflow, the
+   * values of which should be copied to a new "look-ahead" vector. If the
+   * vector is an array, then the overflowIndex is the position of the first
+   * element to be moved, and multiple elements may need to move
+   */
+
+  @Override
+  public void rollover(int cardinality) {
+
+    int sourceStartIndex = writer.writerIndex().rowStartIndex();
+
+    // Remember the last write index for the original vector.
+    // This tells us the end of the set of values to move, while the
+    // sourceStartIndex above tells us the start.
+
+    int sourceEndIndex = writer.lastWriteIndex();
+
+    // Switch buffers between the backup vector and the writer's output
+    // vector. Done this way because writers are bound to vectors and
+    // we wish to keep the binding.
+
+    if (backupVector == null) {
+      backupVector = TypeHelper.getNewVector(mainVector.getField(), mainVector.getAllocator(), null);
+    }
+    assert cardinality > 0;
+    allocateVector(backupVector, cardinality);
+    mainVector.exchange(backupVector);
+
+    // Copy overflow values from the full vector to the new
+    // look-ahead vector.
+
+    copyOverflow(sourceStartIndex, sourceEndIndex);
+
+    // At this point, the writer is positioned to write to the look-ahead
+    // vector at the position after the copied values. The original vector
+    // is saved along with a last write position that is no greater than
+    // the retained values.
+  }
+
+  protected abstract void copyOverflow(int sourceStartIndex, int sourceEndIndex);
+
+  /**
+    * Exchange the data from the backup vector and the main vector, putting
+    * the completed buffers back into the main vectors, and stashing the
+    * overflow buffers away in the backup vector.
+    * Restore the main vector's last write position.
+    */
+
+  @Override
+  public void harvestWithLookAhead() {
+    mainVector.exchange(backupVector);
+  }
+
+  /**
+   * The previous full batch has been sent downstream and the client is
+   * now ready to start writing to the next batch. Initialize that new batch
+   * with the look-ahead values saved during overflow of the previous batch.
+   */
+
+  @Override
+  public void startBatchWithLookAhead() {
+    mainVector.exchange(backupVector);
+    backupVector.clear();
+  }
+
+  @Override
+  public void reset() {
+    mainVector.clear();
+    if (backupVector != null) {
+      backupVector.clear();
+    }
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attributeIdentity("writer", writer)
+      .attributeIdentity("mainVector", mainVector)
+      .attributeIdentity("backupVector", backupVector)
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
new file mode 100644
index 0000000..de41ee4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
@@ -0,0 +1,388 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.rowSet.impl.ColumnState.BaseMapColumnState;
+import org.apache.drill.exec.physical.rowSet.impl.ColumnState.MapArrayColumnState;
+import org.apache.drill.exec.physical.rowSet.impl.ColumnState.MapColumnState;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.TupleSchema.AbstractColumnMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
+import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;
+
+/**
+ * Represents the loader state for a tuple: a row or a map. This is "state" in
+ * the sense of variables that are carried along with each tuple. Handles
+ * write-time issues such as defining new columns, allocating memory, handling
+ * overflow, assembling the output version of the map, and so on. Each
+ * row and map in the result set has a tuple state instances associated
+ * with it.
+ * <p>
+ * Here, by "tuple" we mean a container of vectors, each of which holds
+ * a variety of values. So, the "tuple" here is structural, not a specific
+ * set of values, but rather the collection of vectors that hold tuple
+ * values.
+ */
+
+public abstract class TupleState implements TupleWriterListener {
+
+  /**
+   * Handles the details of the top-level tuple, the data row itself.
+   * Note that by "row" we mean the set of vectors that define the
+   * set of rows.
+   */
+
+  public static class RowState extends TupleState {
+
+    /**
+     * The row-level writer for stepping through rows as they are written,
+     * and for accessing top-level columns.
+     */
+
+    private final RowSetLoaderImpl writer;
+
+    public RowState(ResultSetLoaderImpl rsLoader) {
+      super(rsLoader, rsLoader.projectionSet);
+      writer = new RowSetLoaderImpl(rsLoader, schema);
+      writer.bindListener(this);
+    }
+
+    public RowSetLoaderImpl rootWriter() { return writer; }
+
+    @Override
+    public AbstractTupleWriter writer() { return writer; }
+
+    @Override
+    public int innerCardinality() { return resultSetLoader.targetRowCount();}
+  }
+
+  /**
+   * Represents a tuple defined as a Drill map: single or repeated. Note that
+   * the map vector does not exist here; it is assembled only when "harvesting"
+   * a batch. This design supports the obscure case in which a new column
+   * is added during an overflow row, so exists within this abstraction,
+   * but is not published to the map that makes up the output.
+   */
+
+  public static class MapState extends TupleState {
+
+    protected final BaseMapColumnState mapColumnState;
+    protected int outerCardinality;
+
+    public MapState(ResultSetLoaderImpl rsLoader,
+        BaseMapColumnState mapColumnState,
+        ProjectionSet projectionSet) {
+      super(rsLoader, projectionSet);
+      this.mapColumnState = mapColumnState;
+      mapColumnState.writer().bindListener(this);
+    }
+
+    /**
+     * Return the tuple writer for the map. If this is a single
+     * map, then it is the writer itself. If this is a map array,
+     * then the tuple is nested inside the array.
+     */
+
+    @Override
+    public AbstractTupleWriter writer() {
+      AbstractObjectWriter objWriter = mapColumnState.writer();
+      TupleWriter tupleWriter;
+      if (objWriter.type() == ObjectType.ARRAY) {
+        tupleWriter = objWriter.array().tuple();
+      } else {
+        tupleWriter = objWriter.tuple();
+      }
+      return (AbstractTupleWriter) tupleWriter;
+    }
+
+    /**
+     * In order to allocate the correct-sized vectors, the map must know
+     * its member cardinality: the number of elements in each row. This
+     * is 1 for a single map, but may be any number for a map array. Then,
+     * this value is recursively pushed downward to compute the cardinality
+     * of lists of maps that contains lists of maps, and so on.
+     */
+
+    @Override
+    public void updateCardinality(int outerCardinality) {
+      this.outerCardinality = outerCardinality;
+      super.updateCardinality(outerCardinality);
+    }
+
+    @Override
+    public int innerCardinality() {
+      return outerCardinality * mapColumnState.schema().expectedElementCount();
+    }
+
+    @Override
+    public void dump(HierarchicalFormatter format) {
+      format
+        .startObject(this)
+        .attribute("column", mapColumnState.schema().name())
+        .attribute("cardinality", outerCardinality)
+        .endObject();
+    }
+  }
+
+  protected final ResultSetLoaderImpl resultSetLoader;
+  protected final List<ColumnState> columns = new ArrayList<>();
+  protected final TupleSchema schema = new TupleSchema();
+  protected final ProjectionSet projectionSet;
+
+  protected TupleState(ResultSetLoaderImpl rsLoader, ProjectionSet projectionSet) {
+    this.resultSetLoader = rsLoader;
+    this.projectionSet = projectionSet;
+  }
+
+  public abstract int innerCardinality();
+
+  /**
+   * Returns an ordered set of the columns which make up the tuple.
+   * Column order is the same as that defined by the map's schema,
+   * to allow indexed access. New columns always appear at the end
+   * of the list to preserve indexes.
+   *
+   * @return ordered list of column states for the columns within
+   * this tuple
+   */
+
+  public List<ColumnState> columns() { return columns; }
+
+  public TupleMetadata schema() { return writer().schema(); }
+
+  public abstract AbstractTupleWriter writer();
+
+  @Override
+  public ObjectWriter addColumn(TupleWriter tupleWriter, MaterializedField column) {
+    return addColumn(tupleWriter, TupleSchema.fromField(column));
+  }
+
+  @Override
+  public ObjectWriter addColumn(TupleWriter tupleWriter, ColumnMetadata columnSchema) {
+
+    // Verify name is not a (possibly case insensitive) duplicate.
+
+    TupleMetadata tupleSchema = schema();
+    String colName = columnSchema.name();
+    if (tupleSchema.column(colName) != null) {
+      throw new IllegalArgumentException("Duplicate column: " + colName);
+    }
+
+    return addColumn(columnSchema);
+  }
+
+  /**
+   * Implementation of the work to add a new column to this tuple given a
+   * schema description of the column.
+   *
+   * @param columnSchema schema of the column
+   * @return writer for the new column
+   */
+
+  private AbstractObjectWriter addColumn(ColumnMetadata columnSchema) {
+
+    // Indicate projection in the metadata.
+
+    ((AbstractColumnMetadata) columnSchema).setProjected(
+        projectionSet.isProjected(columnSchema.name()));
+
+    // Build the column
+
+    ColumnState colState;
+    if (columnSchema.isMap()) {
+      colState = buildMap(columnSchema);
+    } else {
+      colState = buildPrimitive(columnSchema);
+    }
+    columns.add(colState);
+    colState.updateCardinality(innerCardinality());
+    colState.allocateVectors();
+    return colState.writer();
+  }
+
+  /**
+   * Build a primitive column. Check if the column is projected. If not,
+   * allocate a dummy writer for the column. If projected, then allocate
+   * a vector, a writer, and the column state which binds the two together
+   * and manages the column.
+   *
+   * @param columnSchema schema of the new primitive column
+   * @return column state for the new column
+   */
+
+  @SuppressWarnings("resource")
+  private ColumnState buildPrimitive(ColumnMetadata columnSchema) {
+    ValueVector vector;
+    if (columnSchema.isProjected()) {
+
+      // Create the vector for the column.
+
+      vector = resultSetLoader.vectorCache().addOrGet(columnSchema.schema());
+    } else {
+
+      // Column is not projected. No materialized backing for the column.
+
+      vector = null;
+    }
+
+    // Create the writer. Will be returned to the tuple writer.
+
+    AbstractObjectWriter colWriter = ColumnWriterFactory.buildColumnWriter(columnSchema, vector);
+
+    if (columnSchema.isArray()) {
+      return PrimitiveColumnState.newPrimitiveArray(resultSetLoader, vector, colWriter);
+    } else {
+      return PrimitiveColumnState.newPrimitive(resultSetLoader, vector, colWriter);
+    }
+  }
+
+  /**
+   * Build a new map (single or repeated) column. No map vector is created
+   * here, instead we create a tuple state to hold the columns, and defer the
+   * map vector (or vector container) until harvest time.
+   *
+   * @param columnSchema description of the map column
+   * @return column state for the map column
+   */
+
+  private ColumnState buildMap(ColumnMetadata columnSchema) {
+
+    // When dynamically adding columns, must add the (empty)
+    // map by itself, then add columns to the map via separate
+    // calls.
+
+    assert columnSchema.isMap();
+    assert columnSchema.mapSchema().size() == 0;
+
+    // Create the writer. Will be returned to the tuple writer.
+
+    ProjectionSet childProjection = projectionSet.mapProjection(columnSchema.name());
+    if (columnSchema.isArray()) {
+      return MapArrayColumnState.build(resultSetLoader,
+          columnSchema,
+          childProjection);
+    } else {
+      return new MapColumnState(resultSetLoader,
+          columnSchema,
+          childProjection);
+    }
+  }
+
+  /**
+   * When creating a schema up front, provide the schema of the desired tuple,
+   * then build vectors and writers to match. Allows up-front schema definition
+   * in addition to on-the-fly schema creation handled elsewhere.
+   *
+   * @param schema desired tuple schema to be materialized
+   */
+
+  public void buildSchema(TupleMetadata schema) {
+    for (int i = 0; i < schema.size(); i++) {
+      ColumnMetadata colSchema = schema.metadata(i);
+      AbstractObjectWriter colWriter;
+      if (colSchema.isMap()) {
+        colWriter = addColumn(colSchema.cloneEmpty());
+        BaseMapColumnState mapColState = (BaseMapColumnState) columns.get(columns.size() - 1);
+        mapColState.mapState().buildSchema(colSchema.mapSchema());
+      } else {
+        colWriter = addColumn(colSchema);
+      }
+      writer().addColumnWriter(colWriter);
+    }
+  }
+
+  public void updateCardinality(int cardinality) {
+    for (ColumnState colState : columns) {
+      colState.updateCardinality(cardinality);
+    }
+  }
+
+  /**
+   * A column within the row batch overflowed. Prepare to absorb the rest of the
+   * in-flight row by rolling values over to a new vector, saving the complete
+   * vector for later. This column could have a value for the overflow row, or
+   * for some previous row, depending on exactly when and where the overflow
+   * occurs.
+   */
+
+  public void rollover() {
+    for (ColumnState colState : columns) {
+      colState.rollover();
+    }
+  }
+
+  /**
+   * Writing of a row batch is complete, and an overflow occurred. Prepare the
+   * vector for harvesting to send downstream. Set aside the look-ahead vector
+   * and put the full vector buffer back into the active vector.
+   */
+
+  public void harvestWithLookAhead() {
+    for (ColumnState colState : columns) {
+      colState.harvestWithLookAhead();
+    }
+  }
+
+  /**
+   * Start a new batch by shifting the overflow buffers back into the main
+   * write vectors and updating the writers.
+   */
+
+  public void startBatch() {
+    for (ColumnState colState : columns) {
+      colState.startBatch();
+    }
+  }
+
+  /**
+   * Clean up state (such as backup vectors) associated with the state
+   * for each vector.
+   */
+
+  public void close() {
+    for (ColumnState colState : columns) {
+      colState.close();
+    }
+  }
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attributeArray("columns");
+    for (int i = 0; i < columns.size(); i++) {
+      format.element(i);
+      columns.get(i).dump(format);
+    }
+    format
+      .endArray()
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
new file mode 100644
index 0000000..faa68cb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorContainerBuilder.java
@@ -0,0 +1,257 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import java.util.List;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.impl.ColumnState.BaseMapColumnState;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+
+/**
+ * Builds the harvest vector container that includes only the columns that
+ * are included in the harvest schema version. That is, it excludes columns
+ * added while writing an overflow row.
+ * <p>
+ * Because a Drill row is actually a hierarchy, walks the internal hierarchy
+ * and builds a corresponding output hierarchy.
+ * <ul>
+ * <li>The root node is the row itself (vector container),</li>
+ * <li>Internal nodes are maps (structures),</li>
+ * <li>Leaf notes are primitive vectors (which may be arrays).</li>
+ * </ul>
+ * The basic algorithm is to identify the version of the output schema,
+ * then add any new columns added up to that version. This object maintains
+ * the output container across batches, meaning that updates are incremental:
+ * we need only add columns that are new since the last update. And, those new
+ * columns will always appear directly after all existing columns in the row
+ * or in a map.
+ * <p>
+ * As special case occurs when columns are added in the overflow row. These
+ * columns <i>do not</i> appear in the output container for the main part
+ * of the batch; instead they appear in the <i>next</i> output container
+ * that includes the overflow row.
+ * <p>
+ * Since the container here may contain a subset of the internal columns, an
+ * interesting case occurs for maps. The maps in the output container are
+ * <b>not</b> the same as those used internally. Since a map column can contain
+ * either one list of columns or another, the internal and external maps must
+ * differ. The set of child vectors (except for child maps) are shared.
+ */
+
+public class VectorContainerBuilder {
+
+  /**
+   * Drill vector containers and maps are both tuples, but they irritatingly
+   * have completely different APIs for working with their child vectors.
+   * This class acts as a proxy to wrap the two APIs to provide a common
+   * view for the use of the container builder.
+   */
+
+  public static abstract class TupleProxy {
+    protected TupleMetadata schema;
+
+    public TupleProxy(TupleMetadata schema) {
+      this.schema = schema;
+    }
+
+    protected abstract int size();
+    protected abstract ValueVector vector(int index);
+    protected abstract void add(ValueVector vector);
+
+    protected TupleProxy mapProxy(int index) {
+      return new MapProxy(
+          schema.metadata(index).mapSchema(),
+          (AbstractMapVector) vector(index));
+    }
+  }
+
+  /**
+   * Proxy wrapper class for a vector container.
+   */
+
+  protected static class ContainerProxy extends TupleProxy {
+
+    private VectorContainer container;
+
+    protected ContainerProxy(TupleMetadata schema, VectorContainer container) {
+      super(schema);
+      this.container = container;
+    }
+
+    @Override
+    protected int size() {
+      return container.getNumberOfColumns();
+    }
+
+    @Override
+    protected ValueVector vector(int index) {
+      return container.getValueVector(index).getValueVector();
+    }
+
+    @Override
+    protected void add(ValueVector vector) {
+      container.add(vector);
+    }
+  }
+
+  /**
+   * Proxy wrapper for a map container.
+   */
+
+  protected static class MapProxy extends TupleProxy {
+
+    private AbstractMapVector mapVector;
+
+    protected MapProxy(TupleMetadata schema, AbstractMapVector mapVector) {
+      super(schema);
+      this.mapVector = mapVector;
+    }
+
+    @Override
+    protected int size() {
+      return mapVector.size();
+    }
+
+    @Override
+    protected ValueVector vector(int index) {
+      return mapVector.getChildByOrdinal(index);
+    }
+
+    @Override
+    protected void add(ValueVector vector) {
+      mapVector.putChild(vector.getField().getName(), vector);
+    }
+  }
+
+  private final ResultSetLoaderImpl resultSetLoader;
+  private int outputSchemaVersion = -1;
+  private TupleMetadata schema;
+  private VectorContainer container;
+
+  public VectorContainerBuilder(ResultSetLoaderImpl rsLoader) {
+    this.resultSetLoader = rsLoader;
+    container = new VectorContainer(rsLoader.allocator);
+    schema = new TupleSchema();
+  }
+
+  public void update(int targetVersion) {
+    if (outputSchemaVersion >= targetVersion) {
+      return;
+    }
+    outputSchemaVersion = targetVersion;
+    updateTuple(resultSetLoader.rootState(), new ContainerProxy(schema, container));
+    container.buildSchema(SelectionVectorMode.NONE);
+  }
+
+  public VectorContainer container() { return container; }
+
+  public int outputSchemaVersion() { return outputSchemaVersion; }
+
+  public BufferAllocator allocator() {
+     return resultSetLoader.allocator();
+  }
+
+  private void updateTuple(TupleState sourceModel, TupleProxy destProxy) {
+    int prevCount = destProxy.size();
+    List<ColumnState> cols = sourceModel.columns();
+    int currentCount = cols.size();
+
+    // Scan any existing maps for column additions
+
+    for (int i = 0; i < prevCount; i++) {
+      ColumnState colState = cols.get(i);
+      if (! colState.schema().isProjected()) {
+        continue;
+      }
+      if (colState.schema().isMap()) {
+        updateTuple((TupleState) ((BaseMapColumnState) colState).mapState(), destProxy.mapProxy(i));
+      }
+    }
+
+    // Add new columns, which may be maps
+
+    for (int i = prevCount; i < currentCount; i++) {
+      ColumnState colState = cols.get(i);
+      if (! colState.schema().isProjected()) {
+        continue;
+      }
+
+      // If the column was added after the output schema version cutoff,
+      // skip that column for now.
+
+      if (colState.addVersion > outputSchemaVersion) {
+        break;
+      }
+      if (colState.schema().isMap()) {
+        buildMap(destProxy, (BaseMapColumnState) colState);
+      } else {
+        destProxy.add(colState.vector());
+        destProxy.schema.addColumn(colState.schema());
+        assert destProxy.size() == destProxy.schema.size();
+      }
+    }
+  }
+
+  @SuppressWarnings("resource")
+  private void buildMap(TupleProxy parentTuple, BaseMapColumnState colModel) {
+
+    // Creating the map vector will create its contained vectors if we
+    // give it a materialized field with children. So, instead pass a clone
+    // without children so we can add them.
+
+    ColumnMetadata mapColSchema = colModel.schema().cloneEmpty();
+
+    // Don't get the map vector from the vector cache. Map vectors may
+    // have content that varies from batch to batch. Only the leaf
+    // vectors can be cached.
+
+    AbstractMapVector mapVector;
+    if (mapColSchema.isArray()) {
+
+      // A repeated map shares an offset vector with the internal
+      // repeated map.
+
+      UInt4Vector offsets = (UInt4Vector) colModel.vector();
+      mapVector = new RepeatedMapVector(mapColSchema.schema(), offsets, null);
+    } else {
+      mapVector = new MapVector(mapColSchema.schema(), allocator(), null);
+    }
+
+    // Add the map vector and schema to the parent tuple
+
+    parentTuple.add(mapVector);
+    int index = parentTuple.schema.addColumn(mapColSchema);
+    assert parentTuple.size() == parentTuple.size();
+
+    // Update the tuple, which will add the new columns in the map
+
+    updateTuple(colModel.mapState(), parentTuple.mapProxy(index));
+  }
+
+  public TupleMetadata schema() { return schema; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorState.java
new file mode 100644
index 0000000..4a1c698
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/VectorState.java
@@ -0,0 +1,102 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Handles batch and overflow operation for a (possibly compound) vector.
+ * <p>
+ * The data model is the following:
+ * <ul>
+ * <li>Column model<ul>
+ *   <li>Value vector itself</li>
+ *   <li>Column writer</li>
+ *   <li>Column schema</li>
+ *   <li>Column coordinator (this class)</li>
+ * </ul></li></ul>
+ * The vector state coordinates events between the result set loader
+ * on the one side and the vectors, writers and schema on the other.
+ * For example:
+ * <pre><code>
+ * Result Set       Vector
+ *   Loader   <-->  State   <-->    Vectors
+ * </code></pre>
+ * Events from the row set loader deal with allocation, roll-over,
+ * harvesting completed batches and so on. Events from the writer,
+ * via the tuple model deal with adding columns and column
+ * overflow.
+ */
+
+public interface VectorState {
+
+  /**
+   * Allocate a new vector with the number of elements given. If the vector
+   * is an array, then the cardinality given is the number of arrays.
+   * @param cardinality number of elements desired in the allocated
+   * vector
+   *
+   * @return the number of bytes allocated
+   */
+
+  int allocate(int cardinality);
+
+  /**
+   * A vector has overflowed. Create a new look-ahead vector of the given
+   * cardinality, then copy the overflow values from the main vector to the
+   * look-ahead vector.
+   *
+   * @param cardinality the number of elements in the new vector. If this
+   * vector is an array, then this is the number of arrays
+   * @return the new next write position for the vector index associated
+   * with the writer for this vector
+   */
+
+  void rollover(int cardinality);
+
+  /**
+   * A batch is being harvested after an overflow. Put the full batch
+   * back into the main vector so it can be harvested.
+   */
+
+  void harvestWithLookAhead();
+
+  /**
+   * A new batch is starting while an look-ahead vector exists. Move
+   * the look-ahead buffers into the main vector to prepare for writing
+   * the rest of the batch.
+   */
+
+  void startBatchWithLookAhead();
+
+  /**
+   * Clear the vector(s) associated with this state.
+   */
+
+  void reset();
+
+  /**
+   * Underlying vector: the one presented to the consumer of the
+   * result set loader.
+   */
+
+  ValueVector vector();
+
+  void dump(HierarchicalFormatter format);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/WriterIndexImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/WriterIndexImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/WriterIndexImpl.java
new file mode 100644
index 0000000..2158dd1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/WriterIndexImpl.java
@@ -0,0 +1,100 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+
+/**
+ * Writer index that points to each row in the row set. The index starts at
+ * the 0th row and advances one row on each increment. This allows writers to
+ * start positioned at the first row. Writes happen in the current row.
+ * Calling <tt>next()</tt> advances to the next position, effectively saving
+ * the current row. The most recent row can be abandoned easily simply by not
+ * calling <tt>next()</tt>. This means that the number of completed rows is
+ * the same as the row index.
+ * <p>
+ * The writer index enforces the row count limit for a new batch. The
+ * limit is set by the result set loader and can vary from batch to batch
+ * if the client chooses in order to adjust the row count based on actual
+ * data size.
+ */
+
+class WriterIndexImpl implements ColumnWriterIndex {
+
+  private final ResultSetLoader rsLoader;
+  private int rowIndex = 0;
+
+  public WriterIndexImpl(ResultSetLoader rsLoader) {
+    this.rsLoader = rsLoader;
+  }
+
+  @Override
+  public int vectorIndex() { return rowIndex; }
+
+  @Override
+  public int rowStartIndex() { return rowIndex; }
+
+  public boolean next() {
+    if (++rowIndex < rsLoader.targetRowCount()) {
+      return true;
+    } else {
+      // Should not call next() again once batch is full.
+      rowIndex = rsLoader.targetRowCount();
+      return false;
+    }
+  }
+
+  public int size() {
+
+    // The index always points to the next slot past the
+    // end of valid rows.
+
+    return rowIndex;
+  }
+
+  public boolean valid() { return rowIndex < rsLoader.targetRowCount(); }
+
+  @Override
+  public void rollover() {
+
+    // The top level index always rolls over to 0 --
+    // the first row position in the new vectors.
+
+    reset();
+  }
+
+  public void reset() { rowIndex = 0; }
+
+  @Override
+  public void nextElement() { }
+
+  @Override
+  public ColumnWriterIndex outerIndex() { return null; }
+
+  @Override
+  public String toString() {
+    return new StringBuilder()
+      .append("[")
+      .append(getClass().getSimpleName())
+      .append(" rowIndex = ")
+      .append(rowIndex)
+      .append("]")
+      .toString();
+  }
+}


[08/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
index 39b0128..f0b3321 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/SchemaBuilder.java
@@ -17,15 +17,16 @@
  */
 package org.apache.drill.test.rowSet;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.ColumnMetadata;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.record.TupleSchema.MapColumnMetadata;
 
 /**
  * Builder of a row set schema expressed as a list of materialized
@@ -59,8 +60,6 @@ public class SchemaBuilder {
    * need scale and precision, and so on.
    */
 
-  // TODO: Add map methods
-
   public static class ColumnBuilder {
     private final String name;
     private final MajorType.Builder typeBuilder;
@@ -78,7 +77,11 @@ public class SchemaBuilder {
     }
 
     public ColumnBuilder setWidth(int width) {
-      typeBuilder.setPrecision(width);
+      return setPrecision(width);
+    }
+
+    public ColumnBuilder setPrecision(int precision) {
+      typeBuilder.setPrecision(precision);
       return this;
     }
 
@@ -101,10 +104,14 @@ public class SchemaBuilder {
   public static class MapBuilder extends SchemaBuilder {
     private final SchemaBuilder parent;
     private final String memberName;
+    private final DataMode mode;
 
-    public MapBuilder(SchemaBuilder parent, String memberName) {
+    public MapBuilder(SchemaBuilder parent, String memberName, DataMode mode) {
       this.parent = parent;
       this.memberName = memberName;
+      // Optional maps not supported in Drill
+      assert mode != DataMode.OPTIONAL;
+      this.mode = mode;
     }
 
     @Override
@@ -114,11 +121,14 @@ public class SchemaBuilder {
 
     @Override
     public SchemaBuilder buildMap() {
-      MaterializedField col = columnSchema(memberName, MinorType.MAP, DataMode.REQUIRED);
-      for (MaterializedField childCol : columns) {
-        col.addChild(childCol);
+      // TODO: Use the map schema directly rather than
+      // rebuilding it as is done here.
+
+      MaterializedField col = columnSchema(memberName, MinorType.MAP, mode);
+      for (ColumnMetadata md : schema) {
+        col.addChild(md.schema());
       }
-      parent.finishMap(col);
+      parent.finishMap(TupleSchema.newMap(col, schema));
       return parent;
     }
 
@@ -128,7 +138,7 @@ public class SchemaBuilder {
     }
   }
 
-  protected List<MaterializedField> columns = new ArrayList<>( );
+  protected TupleSchema schema = new TupleSchema();
   private SelectionVectorMode svMode = SelectionVectorMode.NONE;
 
   public SchemaBuilder() { }
@@ -144,59 +154,60 @@ public class SchemaBuilder {
     }
   }
 
-  public SchemaBuilder add(String pathName, MajorType type) {
-    return add(MaterializedField.create(pathName, type));
+  public SchemaBuilder add(String name, MajorType type) {
+    return add(MaterializedField.create(name, type));
   }
 
   public SchemaBuilder add(MaterializedField col) {
-    columns.add(col);
+    schema.add(col);
     return this;
   }
 
   /**
    * Create a column schema using the "basic three" properties of name, type and
    * cardinality (AKA "data mode.") Use the {@link ColumnBuilder} for to set
-   * other schema attributes.
+   * other schema attributes. Name is relative to the enclosing map or tuple;
+   * it is not the fully qualified path name.
    */
 
-  public static MaterializedField columnSchema(String pathName, MinorType type, DataMode mode) {
-    return MaterializedField.create(pathName,
+  public static MaterializedField columnSchema(String name, MinorType type, DataMode mode) {
+    return MaterializedField.create(name,
         MajorType.newBuilder()
           .setMinorType(type)
           .setMode(mode)
           .build());
   }
 
-  public SchemaBuilder add(String pathName, MinorType type, DataMode mode) {
-    return add(columnSchema(pathName, type, mode));
+  public SchemaBuilder add(String name, MinorType type, DataMode mode) {
+    return add(columnSchema(name, type, mode));
   }
 
-  public SchemaBuilder add(String pathName, MinorType type) {
-    return add(pathName, type, DataMode.REQUIRED);
+  public SchemaBuilder add(String name, MinorType type) {
+    return add(name, type, DataMode.REQUIRED);
   }
 
-  public SchemaBuilder add(String pathName, MinorType type, int width) {
-    MaterializedField field = new SchemaBuilder.ColumnBuilder(pathName, type)
+  public SchemaBuilder add(String name, MinorType type, int width) {
+    MaterializedField field = new SchemaBuilder.ColumnBuilder(name, type)
         .setMode(DataMode.REQUIRED)
         .setWidth(width)
         .build();
     return add(field);
   }
 
-  public SchemaBuilder addNullable(String pathName, MinorType type) {
-    return add(pathName, type, DataMode.OPTIONAL);
+  public SchemaBuilder addNullable(String name, MinorType type) {
+    return add(name, type, DataMode.OPTIONAL);
   }
 
-  public SchemaBuilder addNullable(String pathName, MinorType type, int width) {
-    MaterializedField field = new SchemaBuilder.ColumnBuilder(pathName, type)
+  public SchemaBuilder addNullable(String name, MinorType type, int width) {
+    MaterializedField field = new SchemaBuilder.ColumnBuilder(name, type)
         .setMode(DataMode.OPTIONAL)
         .setWidth(width)
         .build();
     return add(field);
   }
 
-  public SchemaBuilder addArray(String pathName, MinorType type) {
-    return add(pathName, type, DataMode.REPEATED);
+  public SchemaBuilder addArray(String name, MinorType type) {
+    return add(name, type, DataMode.REPEATED);
   }
 
   /**
@@ -209,7 +220,11 @@ public class SchemaBuilder {
    */
 
   public MapBuilder addMap(String pathName) {
-    return new MapBuilder(this, pathName);
+    return new MapBuilder(this, pathName, DataMode.REQUIRED);
+  }
+
+  public MapBuilder addMapArray(String pathName) {
+    return new MapBuilder(this, pathName, DataMode.REPEATED);
   }
 
   public SchemaBuilder withSVMode(SelectionVectorMode svMode) {
@@ -218,14 +233,18 @@ public class SchemaBuilder {
   }
 
   public BatchSchema build() {
-    return new BatchSchema(svMode, columns);
+    return schema.toBatchSchema(svMode);
   }
 
-  void finishMap(MaterializedField map) {
-    columns.add(map);
+  void finishMap(MapColumnMetadata map) {
+    schema.add(map);
   }
 
   public SchemaBuilder buildMap() {
     throw new IllegalStateException("Cannot build map for a top-level schema");
   }
+
+  public TupleMetadata buildSchema() {
+    return schema;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/file/JsonFileBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/file/JsonFileBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/file/JsonFileBuilder.java
index ff93bf0..2076b16 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/file/JsonFileBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/file/JsonFileBuilder.java
@@ -23,9 +23,10 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.vector.accessor.ColumnAccessor;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ValueType;
 import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetReader;
 
 import java.io.BufferedOutputStream;
 import java.io.File;
@@ -44,13 +45,14 @@ public class JsonFileBuilder
   public static final String DEFAULT_DECIMAL_FORMATTER = "%s";
   public static final String DEFAULT_PERIOD_FORMATTER = "%s";
 
+  @SuppressWarnings("unchecked")
   public static final Map<String, String> DEFAULT_FORMATTERS = new ImmutableMap.Builder()
-    .put(ColumnAccessor.ValueType.DOUBLE, DEFAULT_DOUBLE_FORMATTER)
-    .put(ColumnAccessor.ValueType.INTEGER, DEFAULT_INTEGER_FORMATTER)
-    .put(ColumnAccessor.ValueType.LONG, DEFAULT_LONG_FORMATTER)
-    .put(ColumnAccessor.ValueType.STRING, DEFAULT_STRING_FORMATTER)
-    .put(ColumnAccessor.ValueType.DECIMAL, DEFAULT_DECIMAL_FORMATTER)
-    .put(ColumnAccessor.ValueType.PERIOD, DEFAULT_PERIOD_FORMATTER)
+    .put(ValueType.DOUBLE, DEFAULT_DOUBLE_FORMATTER)
+    .put(ValueType.INTEGER, DEFAULT_INTEGER_FORMATTER)
+    .put(ValueType.LONG, DEFAULT_LONG_FORMATTER)
+    .put(ValueType.STRING, DEFAULT_STRING_FORMATTER)
+    .put(ValueType.DECIMAL, DEFAULT_DECIMAL_FORMATTER)
+    .put(ValueType.PERIOD, DEFAULT_PERIOD_FORMATTER)
     .build();
 
   private final RowSet rowSet;
@@ -66,8 +68,7 @@ public class JsonFileBuilder
     Preconditions.checkNotNull(columnFormatter);
 
     Iterator<MaterializedField> fields = rowSet
-      .schema()
-      .batch()
+      .batchSchema()
       .iterator();
 
     boolean hasColumn = false;
@@ -90,14 +91,12 @@ public class JsonFileBuilder
     tableFile.getParentFile().mkdirs();
 
     try (BufferedOutputStream os = new BufferedOutputStream(new FileOutputStream(tableFile))) {
-      final RowSet.RowSetReader reader = rowSet.reader();
+      final RowSetReader reader = rowSet.reader();
       final int numCols = rowSet
-        .schema()
-        .batch()
+        .batchSchema()
         .getFieldCount();
       final Iterator<MaterializedField> fieldIterator = rowSet
-        .schema()
-        .batch()
+        .batchSchema()
         .iterator();
       final List<String> columnNames = Lists.newArrayList();
       final List<String> columnFormatters = Lists.newArrayList();
@@ -105,8 +104,8 @@ public class JsonFileBuilder
       // Build formatters from first row.
       while (fieldIterator.hasNext()) {
         final String columnName = fieldIterator.next().getName();
-        final ColumnReader columnReader = reader.column(columnName);
-        final ColumnAccessor.ValueType valueType = columnReader.valueType();
+        final ScalarReader columnReader = reader.scalar(columnName);
+        final ValueType valueType = columnReader.valueType();
         final String columnFormatter;
 
         if (customFormatters.containsKey(columnName)) {
@@ -135,7 +134,7 @@ public class JsonFileBuilder
           sb.append(separator);
 
           final String columnName = columnNames.get(columnIndex);
-          final ColumnReader columnReader = reader.column(columnIndex);
+          final ScalarReader columnReader = reader.scalar(columnIndex);
           final String columnFormatter = columnFormatters.get(columnIndex);
           final Object columnObject = columnReader.getObject();
           final String columnString = String.format(columnFormatter, columnObject);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
new file mode 100644
index 0000000..db33b30
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/DummyWriterTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.assertNull;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
+import org.apache.drill.exec.vector.accessor.writer.ColumnWriterFactory;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+public class DummyWriterTest extends SubOperatorTest {
+
+  /**
+   * Test only, bare-bones tuple writer used to gather the dummy
+   * column writers.
+   */
+
+  public class RootWriterFixture extends AbstractTupleWriter {
+
+    protected RootWriterFixture(TupleMetadata schema,
+        List<AbstractObjectWriter> writers) {
+      super(schema, writers);
+    }
+  }
+
+  /**
+   * Test dummy column writers for scalars and arrays of
+   * scalars.
+   */
+
+  @Test
+  public void testDummyScalar() {
+
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addArray("b", MinorType.VARCHAR)
+        .buildSchema();
+    List<AbstractObjectWriter> writers = new ArrayList<>();
+
+    // We provide no vector. Factory should build us "dummy" writers.
+
+    writers.add(ColumnWriterFactory.buildColumnWriter(schema.metadata("a"), null));
+    writers.add(ColumnWriterFactory.buildColumnWriter(schema.metadata("b"), null));
+    AbstractTupleWriter rootWriter = new RootWriterFixture(schema, writers);
+
+    // Events are ignored.
+
+    rootWriter.startWrite();
+    rootWriter.startRow();
+
+    // At present, dummy writers report no type (because they don't have one.)
+
+    assertNull(rootWriter.scalar(0).valueType());
+
+    // First column. Set int value.
+
+    rootWriter.scalar(0).setInt(10);
+
+    // Dummy writer does not do type checking. Write "wrong" type.
+    // Should be allowed.
+
+    rootWriter.scalar("a").setString("foo");
+
+    // Column is required, but writer does no checking. Can set
+    // a null value.
+
+    rootWriter.column(0).scalar().setNull();
+
+    // Second column: is an array.
+
+    rootWriter.array(1).scalar().setString("bar");
+    rootWriter.array(1).scalar().setString("mumble");
+
+    // Again, type is not checked.
+
+    rootWriter.array("b").scalar().setInt(200);
+
+    // More ignored events.
+
+    rootWriter.restartRow();
+    rootWriter.saveRow();
+    rootWriter.endWrite();
+  }
+
+  /**
+   * Test a dummy map or map array. A (non-enforced) rule is that such maps
+   * contain only dummy writers. The writers act like "real" writers.
+   */
+
+  @Test
+  public void testDummyMap() {
+
+    TupleMetadata schema = new SchemaBuilder()
+        .addMap("m1")
+          .add("a", MinorType.INT)
+          .addArray("b", MinorType.VARCHAR)
+          .buildMap()
+        .addMapArray("m2")
+          .add("c", MinorType.INT)
+          .buildMap()
+        .buildSchema();
+    List<AbstractObjectWriter> writers = new ArrayList<>();
+
+    {
+      schema.metadata("m1").setProjected(false);
+      TupleMetadata mapSchema = schema.metadata("m1").mapSchema();
+      List<AbstractObjectWriter> members = new ArrayList<>();
+      members.add(ColumnWriterFactory.buildColumnWriter(mapSchema.metadata("a"), null));
+      members.add(ColumnWriterFactory.buildColumnWriter(mapSchema.metadata("b"), null));
+      writers.add(ColumnWriterFactory.buildMapWriter(schema.metadata("m1"), null, members));
+    }
+
+    {
+      schema.metadata("m2").setProjected(false);
+      TupleMetadata mapSchema = schema.metadata("m2").mapSchema();
+      List<AbstractObjectWriter> members = new ArrayList<>();
+      members.add(ColumnWriterFactory.buildColumnWriter(mapSchema.metadata("c"), null));
+      writers.add(ColumnWriterFactory.buildMapWriter(schema.metadata("m2"), null, members));
+    }
+
+    AbstractTupleWriter rootWriter = new RootWriterFixture(schema, writers);
+
+    // Events are ignored.
+
+    rootWriter.startWrite();
+    rootWriter.startRow();
+
+    // Dummy columns seem real.
+
+    rootWriter.tuple("m1").scalar("a").setInt(20);
+    rootWriter.tuple(0).array("b").scalar().setString("foo");
+
+    // Dummy array map seems real.
+
+    rootWriter.array("m2").tuple().scalar("c").setInt(30);
+    rootWriter.array("m2").save();
+    rootWriter.array(1).tuple().scalar(0).setInt(40);
+    rootWriter.array(1).save();
+
+    // More ignored events.
+
+    rootWriter.restartRow();
+    rootWriter.saveRow();
+    rootWriter.endWrite();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
new file mode 100644
index 0000000..10e9032
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
@@ -0,0 +1,296 @@
+/*
+ * 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.test.rowSet.test;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.RepeatedIntVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors.IntColumnWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.ArrayObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.NullableScalarWriter;
+import org.apache.drill.exec.vector.accessor.writer.ScalarArrayWriter;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * Tests the performance of the writers compared to using the value
+ * vector mutators directly. In order to achieve apples-to-apples
+ * comparison, the tests work directly with individual columns in
+ * the writer case; the row writer level is omitted as the row writer
+ * simulates the reader logic previously used to write to vectors.
+ * <p>
+ * Current results:
+ * <ul>
+ * <li>Writer is 42% faster than a required mutator.</li>
+ * <li>Writer is 73% faster than a nullable mutator.</li>
+ * <li>Writer is 407% faster than a repeated mutator.</li>
+ * </ul>
+ * Since performance is critical for this component (this is the
+ * ultimate "inner loop", please run these tests periodically to
+ * ensure that performance does not drop; it is very easy to add
+ * a bit of code here or there that greatly impacts performance.
+ * <p>
+ * This is not a JUnit test. Rather, it is a stand-alone program
+ * which must be run explicitly. One handy way is to run it from
+ * your IDE. If using Eclipse, monitor the system to wait for Eclipse
+ * to finish its background processing before launching.
+ */
+
+public class PerformanceTool {
+
+  public static final int ROW_COUNT = 16 * 1024 * 1024 / 4;
+  public static final int ITERATIONS = 300;
+
+  public static abstract class PerfTester {
+    final TupleMetadata rowSchema;
+    final MaterializedField field;
+    final OperatorFixture fixture;
+    final String label;
+    final Stopwatch timer = Stopwatch.createUnstarted();
+
+    public PerfTester(OperatorFixture fixture, DataMode mode, String label) {
+      this.fixture = fixture;
+      this.label = label;
+      field = SchemaBuilder.columnSchema("a", MinorType.INT, mode);
+      rowSchema = new SchemaBuilder()
+                  .add(field)
+                  .buildSchema();
+    }
+
+    public void runTest() {
+      for (int i = 0; i < ITERATIONS; i++) {
+        doTest();
+      }
+      System.out.println(label + ": " + timer.elapsed(TimeUnit.MILLISECONDS));
+    }
+
+    public abstract void doTest();
+  }
+
+  public static class RequiredVectorTester extends PerfTester {
+
+    public RequiredVectorTester(OperatorFixture fixture) {
+      super(fixture, DataMode.REQUIRED, "Required vector");
+    }
+
+    @Override
+    public void doTest() {
+      try (IntVector vector = new IntVector(field, fixture.allocator());) {
+        vector.allocateNew(4096);
+        IntVector.Mutator mutator = vector.getMutator();
+        timer.start();
+        for (int i = 0; i < ROW_COUNT; i++) {
+          mutator.setSafe(i, 1234);
+        }
+        timer.stop();
+      }
+    }
+  }
+
+  public static class NullableVectorTester extends PerfTester {
+
+    public NullableVectorTester(OperatorFixture fixture) {
+      super(fixture, DataMode.OPTIONAL, "Nullable vector");
+    }
+
+    @Override
+    public void doTest() {
+      try (NullableIntVector vector = new NullableIntVector(field, fixture.allocator());) {
+        vector.allocateNew(4096);
+        NullableIntVector.Mutator mutator = vector.getMutator();
+        timer.start();
+        for (int i = 0; i < ROW_COUNT; i++) {
+          mutator.setSafe(i, 1234);
+        }
+        timer.stop();
+      }
+    }
+  }
+
+  public static class RepeatedVectorTester extends PerfTester {
+
+    public RepeatedVectorTester(OperatorFixture fixture) {
+      super(fixture, DataMode.REQUIRED, "Repeated vector");
+    }
+
+    @Override
+    public void doTest() {
+      try (RepeatedIntVector vector = new RepeatedIntVector(field, fixture.allocator());) {
+        vector.allocateNew(ROW_COUNT, 5 * ROW_COUNT);
+        RepeatedIntVector.Mutator mutator = vector.getMutator();
+        timer.start();
+        for (int i = 0; i < ROW_COUNT / 5; i++) {
+          mutator.startNewValue(i);
+          mutator.addSafe(i, 12341);
+          mutator.addSafe(i, 12342);
+          mutator.addSafe(i, 12343);
+          mutator.addSafe(i, 12344);
+          mutator.addSafe(i, 12345);
+        }
+        timer.stop();
+      }
+    }
+  }
+
+  private static class TestWriterIndex implements ColumnWriterIndex {
+
+    public int index;
+
+    @Override
+    public int vectorIndex() { return index; }
+
+    @Override
+    public final void nextElement() { index++; }
+
+    @Override
+    public void rollover() { }
+
+    @Override
+    public int rowStartIndex() { return index; }
+
+    @Override
+    public ColumnWriterIndex outerIndex() { return null; }
+  }
+
+  public static class RequiredWriterTester extends PerfTester {
+
+    public RequiredWriterTester(OperatorFixture fixture) {
+      super(fixture, DataMode.REQUIRED, "Required writer");
+    }
+
+    @Override
+    public void doTest() {
+      try (IntVector vector = new IntVector(rowSchema.column(0), fixture.allocator());) {
+        vector.allocateNew(ROW_COUNT);
+        IntColumnWriter colWriter = new IntColumnWriter(vector);
+        TestWriterIndex index = new TestWriterIndex();
+        colWriter.bindIndex(index);
+        colWriter.startWrite();
+        timer.start();
+        while (index.index < ROW_COUNT) {
+          colWriter.setInt(1234);
+        }
+        timer.stop();
+        colWriter.endWrite();
+      }
+    }
+  }
+
+  public static class NullableWriterTester extends PerfTester {
+
+    public NullableWriterTester(OperatorFixture fixture) {
+      super(fixture, DataMode.OPTIONAL, "Nullable writer");
+    }
+
+    @Override
+    public void doTest() {
+      try (NullableIntVector vector = new NullableIntVector(rowSchema.column(0), fixture.allocator());) {
+        vector.allocateNew(ROW_COUNT);
+        NullableScalarWriter colWriter = new NullableScalarWriter(
+            vector, new IntColumnWriter(vector.getValuesVector()));
+        TestWriterIndex index = new TestWriterIndex();
+        colWriter.bindIndex(index);
+        colWriter.startWrite();
+        timer.start();
+        while (index.index < ROW_COUNT) {
+          colWriter.setInt(1234);
+        }
+        timer.stop();
+        colWriter.endWrite();
+      }
+    }
+  }
+
+  public static class ArrayWriterTester extends PerfTester {
+
+    public ArrayWriterTester(OperatorFixture fixture) {
+      super(fixture, DataMode.REQUIRED, "Array writer");
+    }
+
+    @Override
+    public void doTest() {
+      try (RepeatedIntVector vector = new RepeatedIntVector(rowSchema.column(0), fixture.allocator());) {
+        vector.allocateNew(ROW_COUNT, 5 * ROW_COUNT);
+        IntColumnWriter colWriter = new IntColumnWriter(vector.getDataVector());
+        ColumnMetadata colSchema = TupleSchema.fromField(vector.getField());
+        ArrayObjectWriter arrayWriter = ScalarArrayWriter.build(colSchema, vector, colWriter);
+        TestWriterIndex index = new TestWriterIndex();
+        arrayWriter.events().bindIndex(index);
+        arrayWriter.events().startWrite();
+        timer.start();
+        for ( ; index.index < ROW_COUNT / 5; index.index++) {
+          arrayWriter.events().startRow();
+          colWriter.setInt(12341);
+          colWriter.setInt(12342);
+          colWriter.setInt(12343);
+          colWriter.setInt(12344);
+          colWriter.setInt(12345);
+          arrayWriter.events().endArrayValue();
+        }
+        timer.stop();
+        arrayWriter.events().endWrite();
+      }
+    }
+  }
+
+  public static void main(String args[]) {
+    try (OperatorFixture fixture = OperatorFixture.standardFixture();) {
+      for (int i = 0; i < 2; i++) {
+        System.out.println((i==0) ? "Warmup" : "Test run");
+        new RequiredVectorTester(fixture).runTest();
+        new RequiredWriterTester(fixture).runTest();
+        new NullableVectorTester(fixture).runTest();
+        new NullableWriterTester(fixture).runTest();
+        new RepeatedVectorTester(fixture).runTest();
+        new ArrayWriterTester(fixture).runTest();
+      }
+    } catch (Exception e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+  }
+
+  @SuppressWarnings("unused")
+  private static void testWriter2(TupleMetadata rowSchema,
+      OperatorFixture fixture, Stopwatch timer) {
+    ExtendableRowSet rs = fixture.rowSet(rowSchema);
+    RowSetWriter writer = rs.writer(4096);
+    ScalarWriter colWriter = writer.scalar(0);
+    timer.start();
+    for (int i = 0; i < ROW_COUNT; i++) {
+      colWriter.setInt(i);
+      writer.save();
+    }
+    timer.stop();
+    writer.done().clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
index af35cdf..4db4d09 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/RowSetTest.java
@@ -19,362 +19,508 @@ package org.apache.drill.test.rowSet.test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
 
-import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.TupleAccessor.TupleSchema;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
 import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
-import org.apache.drill.test.rowSet.RowSetSchema;
-import org.apache.drill.test.rowSet.RowSetSchema.FlattenedSchema;
-import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.SchemaBuilder;
 import org.junit.Test;
 
-import com.google.common.base.Splitter;
+/**
+ * Test row sets. Since row sets are a thin wrapper around vectors,
+ * readers and writers, this is also a test of those constructs.
+ * <p>
+ * Tests basic protocol of the writers: <pre><code>
+ * row : tuple
+ * tuple : column *
+ * column : scalar obj | array obj | tuple obj
+ * scalar obj : scalar
+ * array obj : array writer
+ * array writer : element
+ * element : column
+ * tuple obj : tuple</code></pre>
+ */
 
 public class RowSetTest extends SubOperatorTest {
 
   /**
-   * Test a simple physical schema with no maps.
+   * Test the simplest constructs: a row with top-level scalar
+   * columns.
+   * <p>
+   * The focus here is the structure of the readers and writers, along
+   * with the row set loader and verifier that use those constructs.
+   * That is, while this test uses the int vector, this test is not
+   * focused on that vector.
    */
 
-//  @Test
-//  public void testSchema() {
-//    BatchSchema batchSchema = new SchemaBuilder()
-//        .add("c", MinorType.INT)
-//        .add("a", MinorType.INT, DataMode.REPEATED)
-//        .addNullable("b", MinorType.VARCHAR)
-//        .build();
-//
-//    assertEquals("c", batchSchema.getColumn(0).getName());
-//    assertEquals("a", batchSchema.getColumn(1).getName());
-//    assertEquals("b", batchSchema.getColumn(2).getName());
-//
-//    RowSetSchema schema = new RowSetSchema(batchSchema);
-//    TupleSchema access = schema.hierarchicalAccess();
-//    assertEquals(3, access.count());
-//
-//    crossCheck(access, 0, "c", MinorType.INT);
-//    assertEquals(DataMode.REQUIRED, access.column(0).getDataMode());
-//    assertEquals(DataMode.REQUIRED, access.column(0).getType().getMode());
-//    assertTrue(! access.column(0).isNullable());
-//
-//    crossCheck(access, 1, "a", MinorType.INT);
-//    assertEquals(DataMode.REPEATED, access.column(1).getDataMode());
-//    assertEquals(DataMode.REPEATED, access.column(1).getType().getMode());
-//    assertTrue(! access.column(1).isNullable());
-//
-//    crossCheck(access, 2, "b", MinorType.VARCHAR);
-//    assertEquals(MinorType.VARCHAR, access.column(2).getType().getMinorType());
-//    assertEquals(DataMode.OPTIONAL, access.column(2).getDataMode());
-//    assertEquals(DataMode.OPTIONAL, access.column(2).getType().getMode());
-//    assertTrue(access.column(2).isNullable());
-//
-//    // No maps: physical schema is the same as access schema.
-//
-//    PhysicalSchema physical = schema.physical();
-//    assertEquals(3, physical.count());
-//    assertEquals("c", physical.column(0).field().getName());
-//    assertEquals("a", physical.column(1).field().getName());
-//    assertEquals("b", physical.column(2).field().getName());
-//  }
+  @Test
+  public void testScalarStructure() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .buildSchema();
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer();
 
-  /**
-   * Validate that the actual column metadata is as expected by
-   * cross-checking: validate that the column at the index and
-   * the column at the column name are both correct.
-   *
-   * @param schema the schema for the row set
-   * @param index column index
-   * @param fullName expected column name
-   * @param type expected type
-   */
+    // Required Int
+    // Verify the invariants of the "full" and "simple" access paths
 
-//  public void crossCheck(TupleSchema schema, int index, String fullName, MinorType type) {
-//    String name = null;
-//    for (String part : Splitter.on(".").split(fullName)) {
-//      name = part;
-//    }
-//    assertEquals(name, schema.column(index).getName());
-//    assertEquals(index, schema.columnIndex(fullName));
-//    assertSame(schema.column(index), schema.column(fullName));
-//    assertEquals(type, schema.column(index).getType().getMinorType());
-//  }
+    assertEquals(ObjectType.SCALAR, writer.column("a").type());
+    assertSame(writer.column("a"), writer.column(0));
+    assertSame(writer.scalar("a"), writer.scalar(0));
+    assertSame(writer.column("a").scalar(), writer.scalar("a"));
+    assertSame(writer.column(0).scalar(), writer.scalar(0));
+    assertEquals(ValueType.INTEGER, writer.scalar(0).valueType());
 
-  /**
-   * Verify that a nested map schema works as expected.
-   */
+    // Sanity checks
 
-//  @Test
-//  public void testMapSchema() {
-//    BatchSchema batchSchema = new SchemaBuilder()
-//        .add("c", MinorType.INT)
-//        .addMap("a")
-//          .addNullable("b", MinorType.VARCHAR)
-//          .add("d", MinorType.INT)
-//          .addMap("e")
-//            .add("f", MinorType.VARCHAR)
-//            .buildMap()
-//          .add("g", MinorType.INT)
-//          .buildMap()
-//        .add("h", MinorType.BIGINT)
-//        .build();
-//
-//    RowSetSchema schema = new RowSetSchema(batchSchema);
-//
-//    // Access schema: flattened with maps removed
-//
-//    FlattenedSchema access = schema.flatAccess();
-//    assertEquals(6, access.count());
-//    crossCheck(access, 0, "c", MinorType.INT);
-//    crossCheck(access, 1, "a.b", MinorType.VARCHAR);
-//    crossCheck(access, 2, "a.d", MinorType.INT);
-//    crossCheck(access, 3, "a.e.f", MinorType.VARCHAR);
-//    crossCheck(access, 4, "a.g", MinorType.INT);
-//    crossCheck(access, 5, "h", MinorType.BIGINT);
-//
-//    // Should have two maps.
-//
-//    assertEquals(2, access.mapCount());
-//    assertEquals("a", access.map(0).getName());
-//    assertEquals("e", access.map(1).getName());
-//    assertEquals(0, access.mapIndex("a"));
-//    assertEquals(1, access.mapIndex("a.e"));
-//
-//    // Verify physical schema: should mirror the schema created above.
-//
-//    PhysicalSchema physical = schema.physical();
-//    assertEquals(3, physical.count());
-//    assertEquals("c", physical.column(0).field().getName());
-//    assertEquals("c", physical.column(0).fullName());
-//    assertFalse(physical.column(0).isMap());
-//    assertNull(physical.column(0).mapSchema());
-//
-//    assertEquals("a", physical.column(1).field().getName());
-//    assertEquals("a", physical.column(1).fullName());
-//    assertTrue(physical.column(1).isMap());
-//    assertNotNull(physical.column(1).mapSchema());
-//
-//    assertEquals("h", physical.column(2).field().getName());
-//    assertEquals("h", physical.column(2).fullName());
-//    assertFalse(physical.column(2).isMap());
-//    assertNull(physical.column(2).mapSchema());
-//
-//    PhysicalSchema aSchema = physical.column(1).mapSchema();
-//    assertEquals(4, aSchema.count());
-//    assertEquals("b", aSchema.column(0).field().getName());
-//    assertEquals("a.b", aSchema.column(0).fullName());
-//    assertEquals("d", aSchema.column(1).field().getName());
-//    assertEquals("e", aSchema.column(2).field().getName());
-//    assertEquals("g", aSchema.column(3).field().getName());
-//
-//    PhysicalSchema eSchema = aSchema.column(2).mapSchema();
-//    assertEquals(1, eSchema.count());
-//    assertEquals("f", eSchema.column(0).field().getName());
-//    assertEquals("a.e.f", eSchema.column(0).fullName());
-//  }
+    try {
+      writer.column(0).array();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+    try {
+      writer.column(0).tuple();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
 
-  /**
-   * Verify that simple scalar (non-repeated) column readers
-   * and writers work as expected. This is for tiny ints.
-   */
+    // Test the various ways to get at the scalar writer.
 
-  @Test
-  public void testTinyIntRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.TINYINT)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0)
-        .add(Byte.MAX_VALUE)
-        .add(Byte.MIN_VALUE)
-        .build();
-    assertEquals(3, rs.rowCount());
-    RowSetReader reader = rs.reader();
+    writer.column("a").scalar().setInt(10);
+    writer.save();
+    writer.scalar("a").setInt(20);
+    writer.save();
+    writer.column(0).scalar().setInt(30);
+    writer.save();
+    writer.scalar(0).setInt(40);
+    writer.save();
+
+    // Finish the row set and get a reader.
+
+    SingleRowSet actual = writer.done();
+    RowSetReader reader = actual.reader();
+
+    // Verify invariants
+
+    assertEquals(ObjectType.SCALAR, reader.column(0).type());
+    assertSame(reader.column("a"), reader.column(0));
+    assertSame(reader.scalar("a"), reader.scalar(0));
+    assertSame(reader.column("a").scalar(), reader.scalar("a"));
+    assertSame(reader.column(0).scalar(), reader.scalar(0));
+    assertEquals(ValueType.INTEGER, reader.scalar(0).valueType());
+
+    // Test various accessors: full and simple
+
+    assertTrue(reader.next());
+    assertEquals(10, reader.column("a").scalar().getInt());
     assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getInt());
+    assertEquals(20, reader.scalar("a").getInt());
     assertTrue(reader.next());
-    assertEquals(Byte.MAX_VALUE, reader.column(0).getInt());
-    assertEquals((int) Byte.MAX_VALUE, reader.column(0).getObject());
+    assertEquals(30, reader.column(0).scalar().getInt());
     assertTrue(reader.next());
-    assertEquals(Byte.MIN_VALUE, reader.column(0).getInt());
+    assertEquals(40, reader.scalar(0).getInt());
     assertFalse(reader.next());
-    rs.clear();
-  }
 
-  @Test
-  public void testSmallIntRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.SMALLINT)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0)
-        .add(Short.MAX_VALUE)
-        .add(Short.MIN_VALUE)
+    // Test the above again via the writer and reader
+    // utility classes.
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10)
+        .addRow(20)
+        .addRow(30)
+        .addRow(40)
         .build();
-    RowSetReader reader = rs.reader();
-    assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getInt());
-    assertTrue(reader.next());
-    assertEquals(Short.MAX_VALUE, reader.column(0).getInt());
-    assertEquals((int) Short.MAX_VALUE, reader.column(0).getObject());
-    assertTrue(reader.next());
-    assertEquals(Short.MIN_VALUE, reader.column(0).getInt());
-    assertFalse(reader.next());
-    rs.clear();
+    new RowSetComparison(expected).verifyAndClearAll(actual);
   }
 
+  /**
+   * Test a record with a top level array. The focus here is on the
+   * scalar array structure.
+   *
+   * @throws VectorOverflowException should never occur
+   */
+
   @Test
-  public void testIntRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.INT)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0)
-        .add(Integer.MAX_VALUE)
-        .add(Integer.MIN_VALUE)
-        .build();
-    RowSetReader reader = rs.reader();
+  public void testScalarArrayStructure() {
+    TupleMetadata schema = new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .buildSchema();
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer();
+
+    // Repeated Int
+    // Verify the invariants of the "full" and "simple" access paths
+
+    assertEquals(ObjectType.ARRAY, writer.column("a").type());
+
+    assertSame(writer.column("a"), writer.column(0));
+    assertSame(writer.array("a"), writer.array(0));
+    assertSame(writer.column("a").array(), writer.array("a"));
+    assertSame(writer.column(0).array(), writer.array(0));
+
+    assertEquals(ObjectType.SCALAR, writer.column("a").array().entry().type());
+    assertEquals(ObjectType.SCALAR, writer.column("a").array().entryType());
+    assertSame(writer.array(0).entry().scalar(), writer.array(0).scalar());
+    assertEquals(ValueType.INTEGER, writer.array(0).scalar().valueType());
+
+    // Sanity checks
+
+    try {
+      writer.column(0).scalar();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+    try {
+      writer.column(0).tuple();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+
+    // Write some data
+
+    ScalarWriter intWriter = writer.array("a").scalar();
+    intWriter.setInt(10);
+    intWriter.setInt(11);
+    writer.save();
+    intWriter.setInt(20);
+    intWriter.setInt(21);
+    intWriter.setInt(22);
+    writer.save();
+    intWriter.setInt(30);
+    writer.save();
+    intWriter.setInt(40);
+    intWriter.setInt(41);
+    writer.save();
+
+    // Finish the row set and get a reader.
+
+    SingleRowSet actual = writer.done();
+    RowSetReader reader = actual.reader();
+
+    // Verify the invariants of the "full" and "simple" access paths
+
+    assertEquals(ObjectType.ARRAY, writer.column("a").type());
+
+    assertSame(reader.column("a"), reader.column(0));
+    assertSame(reader.array("a"), reader.array(0));
+    assertSame(reader.column("a").array(), reader.array("a"));
+    assertSame(reader.column(0).array(), reader.array(0));
+
+    assertEquals(ObjectType.SCALAR, reader.column("a").array().entryType());
+    assertEquals(ValueType.INTEGER, reader.array(0).elements().valueType());
+
+    // Read and verify the rows
+
+    ScalarElementReader intReader = reader.array(0).elements();
+    assertTrue(reader.next());
+    assertEquals(2, intReader.size());
+    assertEquals(10, intReader.getInt(0));
+    assertEquals(11, intReader.getInt(1));
     assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getInt());
+    assertEquals(3, intReader.size());
+    assertEquals(20, intReader.getInt(0));
+    assertEquals(21, intReader.getInt(1));
+    assertEquals(22, intReader.getInt(2));
     assertTrue(reader.next());
-    assertEquals(Integer.MAX_VALUE, reader.column(0).getInt());
-    assertEquals(Integer.MAX_VALUE, reader.column(0).getObject());
+    assertEquals(1, intReader.size());
+    assertEquals(30, intReader.getInt(0));
     assertTrue(reader.next());
-    assertEquals(Integer.MIN_VALUE, reader.column(0).getInt());
+    assertEquals(2, intReader.size());
+    assertEquals(40, intReader.getInt(0));
+    assertEquals(41, intReader.getInt(1));
     assertFalse(reader.next());
-    rs.clear();
-  }
 
-  @Test
-  public void testLongRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.BIGINT)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0L)
-        .add(Long.MAX_VALUE)
-        .add(Long.MIN_VALUE)
+    // Test the above again via the writer and reader
+    // utility classes.
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addSingleCol(new int[] {10, 11})
+        .addSingleCol(new int[] {20, 21, 22})
+        .addSingleCol(new int[] {30})
+        .addSingleCol(new int[] {40, 41})
         .build();
-    RowSetReader reader = rs.reader();
-    assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getLong());
-    assertTrue(reader.next());
-    assertEquals(Long.MAX_VALUE, reader.column(0).getLong());
-    assertEquals(Long.MAX_VALUE, reader.column(0).getObject());
-    assertTrue(reader.next());
-    assertEquals(Long.MIN_VALUE, reader.column(0).getLong());
-    assertFalse(reader.next());
-    rs.clear();
+    new RowSetComparison(expected)
+      .verifyAndClearAll(actual);
   }
 
+  /**
+   * Test a simple map structure at the top level of a row.
+   *
+   * @throws VectorOverflowException should never occur
+   */
+
   @Test
-  public void testFloatRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.FLOAT4)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0F)
-        .add(Float.MAX_VALUE)
-        .add(Float.MIN_VALUE)
-        .build();
-    RowSetReader reader = rs.reader();
+  public void testMapStructure() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMap("m")
+          .addArray("b", MinorType.INT)
+          .buildMap()
+        .buildSchema();
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer();
+
+    // Map and Int
+    // Test Invariants
+
+    assertEquals(ObjectType.SCALAR, writer.column("a").type());
+    assertEquals(ObjectType.SCALAR, writer.column(0).type());
+    assertEquals(ObjectType.TUPLE, writer.column("m").type());
+    assertEquals(ObjectType.TUPLE, writer.column(1).type());
+    assertSame(writer.column(1).tuple(), writer.tuple(1));
+
+    TupleWriter mapWriter = writer.column(1).tuple();
+    assertEquals(ObjectType.SCALAR, mapWriter.column("b").array().entry().type());
+    assertEquals(ObjectType.SCALAR, mapWriter.column("b").array().entryType());
+
+    ScalarWriter aWriter = writer.column("a").scalar();
+    ScalarWriter bWriter = writer.column("m").tuple().column("b").array().entry().scalar();
+    assertSame(bWriter, writer.tuple(1).array(0).scalar());
+    assertEquals(ValueType.INTEGER, bWriter.valueType());
+
+    // Sanity checks
+
+    try {
+      writer.column(1).scalar();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+    try {
+      writer.column(1).array();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+
+    // Write data
+
+    aWriter.setInt(10);
+    bWriter.setInt(11);
+    bWriter.setInt(12);
+    writer.save();
+    aWriter.setInt(20);
+    bWriter.setInt(21);
+    bWriter.setInt(22);
+    writer.save();
+    aWriter.setInt(30);
+    bWriter.setInt(31);
+    bWriter.setInt(32);
+    writer.save();
+
+    // Finish the row set and get a reader.
+
+    SingleRowSet actual = writer.done();
+    RowSetReader reader = actual.reader();
+
+    assertEquals(ObjectType.SCALAR, reader.column("a").type());
+    assertEquals(ObjectType.SCALAR, reader.column(0).type());
+    assertEquals(ObjectType.TUPLE, reader.column("m").type());
+    assertEquals(ObjectType.TUPLE, reader.column(1).type());
+    assertSame(reader.column(1).tuple(), reader.tuple(1));
+
+    ScalarReader aReader = reader.column(0).scalar();
+    TupleReader mReader = reader.column(1).tuple();
+    assertEquals(ObjectType.SCALAR, mReader.column("b").array().entryType());
+    ScalarElementReader bReader = mReader.column(0).elements();
+    assertEquals(ValueType.INTEGER, bReader.valueType());
+
     assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getDouble(), 0.000001);
+    assertEquals(10, aReader.getInt());
+    assertEquals(11, bReader.getInt(0));
+    assertEquals(12, bReader.getInt(1));
     assertTrue(reader.next());
-    assertEquals((double) Float.MAX_VALUE, reader.column(0).getDouble(), 0.000001);
-    assertEquals((double) Float.MAX_VALUE, (double) reader.column(0).getObject(), 0.000001);
+    assertEquals(20, aReader.getInt());
+    assertEquals(21, bReader.getInt(0));
+    assertEquals(22, bReader.getInt(1));
     assertTrue(reader.next());
-    assertEquals((double) Float.MIN_VALUE, reader.column(0).getDouble(), 0.000001);
+    assertEquals(30, aReader.getInt());
+    assertEquals(31, bReader.getInt(0));
+    assertEquals(32, bReader.getInt(1));
     assertFalse(reader.next());
-    rs.clear();
-  }
 
-  @Test
-  public void testDoubleRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.FLOAT8)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add(0D)
-        .add(Double.MAX_VALUE)
-        .add(Double.MIN_VALUE)
+    // Verify that the map accessor's value count was set.
+
+    @SuppressWarnings("resource")
+    MapVector mapVector = (MapVector) actual.container().getValueVector(1).getValueVector();
+    assertEquals(actual.rowCount(), mapVector.getAccessor().getValueCount());
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {new int[] {11, 12}})
+        .addRow(20, new Object[] {new int[] {21, 22}})
+        .addRow(30, new Object[] {new int[] {31, 32}})
         .build();
-    RowSetReader reader = rs.reader();
-    assertTrue(reader.next());
-    assertEquals(0, reader.column(0).getDouble(), 0.000001);
-    assertTrue(reader.next());
-    assertEquals(Double.MAX_VALUE, reader.column(0).getDouble(), 0.000001);
-    assertEquals(Double.MAX_VALUE, (double) reader.column(0).getObject(), 0.000001);
-    assertTrue(reader.next());
-    assertEquals(Double.MIN_VALUE, reader.column(0).getDouble(), 0.000001);
-    assertFalse(reader.next());
-    rs.clear();
+    new RowSetComparison(expected)
+      .verifyAndClearAll(actual);
   }
 
   @Test
-  public void testStringRW() {
-    BatchSchema batchSchema = new SchemaBuilder()
-        .add("col", MinorType.VARCHAR)
-        .build();
-    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-        .add("")
-        .add("abcd")
-        .build();
-    RowSetReader reader = rs.reader();
+  public void testRepeatedMapStructure() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m")
+          .add("b", MinorType.INT)
+          .add("c", MinorType.INT)
+          .buildMap()
+        .buildSchema();
+    ExtendableRowSet rowSet = fixture.rowSet(schema);
+    RowSetWriter writer = rowSet.writer();
+
+    // Map and Int
+    // Pick out components and lightly test. (Assumes structure
+    // tested earlier is still valid, so no need to exhaustively
+    // test again.)
+
+    assertEquals(ObjectType.SCALAR, writer.column("a").type());
+    assertEquals(ObjectType.ARRAY, writer.column("m").type());
+
+    ArrayWriter maWriter = writer.column(1).array();
+    assertEquals(ObjectType.TUPLE, maWriter.entryType());
+
+    TupleWriter mapWriter = maWriter.tuple();
+    assertEquals(ObjectType.SCALAR, mapWriter.column("b").type());
+    assertEquals(ObjectType.SCALAR, mapWriter.column("c").type());
+
+    ScalarWriter aWriter = writer.column("a").scalar();
+    ScalarWriter bWriter = mapWriter.scalar("b");
+    ScalarWriter cWriter = mapWriter.scalar("c");
+    assertEquals(ValueType.INTEGER, aWriter.valueType());
+    assertEquals(ValueType.INTEGER, bWriter.valueType());
+    assertEquals(ValueType.INTEGER, cWriter.valueType());
+
+    // Write data
+
+    aWriter.setInt(10);
+    bWriter.setInt(101);
+    cWriter.setInt(102);
+    maWriter.save(); // Advance to next array position
+    bWriter.setInt(111);
+    cWriter.setInt(112);
+    maWriter.save();
+    writer.save();
+
+    aWriter.setInt(20);
+    bWriter.setInt(201);
+    cWriter.setInt(202);
+    maWriter.save();
+    bWriter.setInt(211);
+    cWriter.setInt(212);
+    maWriter.save();
+    writer.save();
+
+    aWriter.setInt(30);
+    bWriter.setInt(301);
+    cWriter.setInt(302);
+    maWriter.save();
+    bWriter.setInt(311);
+    cWriter.setInt(312);
+    maWriter.save();
+    writer.save();
+
+    // Finish the row set and get a reader.
+
+    SingleRowSet actual = writer.done();
+    RowSetReader reader = actual.reader();
+
+    // Verify reader structure
+
+    assertEquals(ObjectType.SCALAR, reader.column("a").type());
+    assertEquals(ObjectType.ARRAY, reader.column("m").type());
+
+    ArrayReader maReader = reader.column(1).array();
+    assertEquals(ObjectType.TUPLE, maReader.entryType());
+
+    TupleReader mapReader = maReader.tuple();
+    assertEquals(ObjectType.SCALAR, mapReader.column("b").type());
+    assertEquals(ObjectType.SCALAR, mapReader.column("c").type());
+
+    ScalarReader aReader = reader.column("a").scalar();
+    ScalarReader bReader = mapReader.scalar("b");
+    ScalarReader cReader = mapReader.scalar("c");
+    assertEquals(ValueType.INTEGER, aReader.valueType());
+    assertEquals(ValueType.INTEGER, bReader.valueType());
+    assertEquals(ValueType.INTEGER, cReader.valueType());
+
+    // Row 1: use index accessors
+
+    assertTrue(reader.next());
+    assertEquals(10, aReader.getInt());
+    TupleReader ixReader = maReader.tuple(0);
+    assertEquals(101, ixReader.scalar(0).getInt());
+    assertEquals(102, ixReader.scalar(1).getInt());
+    ixReader = maReader.tuple(1);
+    assertEquals(111, ixReader.scalar(0).getInt());
+    assertEquals(112, ixReader.scalar(1).getInt());
+
+    // Row 2: use common accessor with explicit positioning,
+    // but access scalars through the map reader.
+
     assertTrue(reader.next());
-    assertEquals("", reader.column(0).getString());
+    assertEquals(20, aReader.getInt());
+    maReader.setPosn(0);
+    assertEquals(201, mapReader.scalar(0).getInt());
+    assertEquals(202, mapReader.scalar(1).getInt());
+    maReader.setPosn(1);
+    assertEquals(211, mapReader.scalar(0).getInt());
+    assertEquals(212, mapReader.scalar(1).getInt());
+
+    // Row 3: use common accessor for scalars
+
     assertTrue(reader.next());
-    assertEquals("abcd", reader.column(0).getString());
-    assertEquals("abcd", reader.column(0).getObject());
+    assertEquals(30, aReader.getInt());
+    maReader.setPosn(0);
+    assertEquals(301, bReader.getInt());
+    assertEquals(302, cReader.getInt());
+    maReader.setPosn(1);
+    assertEquals(311, bReader.getInt());
+    assertEquals(312, cReader.getInt());
+
     assertFalse(reader.next());
-    rs.clear();
-  }
 
-  /**
-   * Test writing to and reading from a row set with nested maps.
-   * Map fields are flattened into a logical schema.
-   */
+    // Verify that the map accessor's value count was set.
 
-//  @Test
-//  public void testMap() {
-//    BatchSchema batchSchema = new SchemaBuilder()
-//        .add("a", MinorType.INT)
-//        .addMap("b")
-//          .add("c", MinorType.INT)
-//          .add("d", MinorType.INT)
-//          .buildMap()
-//        .build();
-//    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
-//        .add(10, 20, 30)
-//        .add(40, 50, 60)
-//        .build();
-//    RowSetReader reader = rs.reader();
-//    assertTrue(reader.next());
-//    assertEquals(10, reader.column(0).getInt());
-//    assertEquals(20, reader.column(1).getInt());
-//    assertEquals(30, reader.column(2).getInt());
-//    assertEquals(10, reader.column("a").getInt());
-//    assertEquals(30, reader.column("b.d").getInt());
-//    assertTrue(reader.next());
-//    assertEquals(40, reader.column(0).getInt());
-//    assertEquals(50, reader.column(1).getInt());
-//    assertEquals(60, reader.column(2).getInt());
-//    assertFalse(reader.next());
-//    rs.clear();
-//  }
+    @SuppressWarnings("resource")
+    RepeatedMapVector mapVector = (RepeatedMapVector) actual.container().getValueVector(1).getValueVector();
+    assertEquals(3, mapVector.getAccessor().getValueCount());
+
+    // Verify the readers and writers again using the testing tools.
+
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {new Object[] {101, 102}, new Object[] {111, 112}})
+        .addRow(20, new Object[] {new Object[] {201, 202}, new Object[] {211, 212}})
+        .addRow(30, new Object[] {new Object[] {301, 302}, new Object[] {311, 312}})
+        .build();
+    new RowSetComparison(expected)
+      .verifyAndClearAll(actual);
+  }
 
   /**
    * Test an array of ints (as an example fixed-width type)
@@ -382,7 +528,7 @@ public class RowSetTest extends SubOperatorTest {
    */
 
   @Test
-  public void TestTopFixedWidthArray() {
+  public void testTopFixedWidthArray() {
     BatchSchema batchSchema = new SchemaBuilder()
         .add("c", MinorType.INT)
         .addArray("a", MinorType.INT)
@@ -390,49 +536,131 @@ public class RowSetTest extends SubOperatorTest {
 
     ExtendableRowSet rs1 = fixture.rowSet(batchSchema);
     RowSetWriter writer = rs1.writer();
-    writer.column(0).setInt(10);
-    ArrayWriter array = writer.column(1).array();
+    writer.scalar(0).setInt(10);
+    ScalarWriter array = writer.array(1).scalar();
     array.setInt(100);
     array.setInt(110);
     writer.save();
-    writer.column(0).setInt(20);
-    array = writer.column(1).array();
+    writer.scalar(0).setInt(20);
     array.setInt(200);
     array.setInt(120);
     array.setInt(220);
     writer.save();
-    writer.column(0).setInt(30);
+    writer.scalar(0).setInt(30);
     writer.save();
-    writer.done();
 
-    RowSetReader reader = rs1.reader();
+    SingleRowSet result = writer.done();
+
+    RowSetReader reader = result.reader();
     assertTrue(reader.next());
-    assertEquals(10, reader.column(0).getInt());
-    ArrayReader arrayReader = reader.column(1).array();
+    assertEquals(10, reader.scalar(0).getInt());
+    ScalarElementReader arrayReader = reader.array(1).elements();
     assertEquals(2, arrayReader.size());
     assertEquals(100, arrayReader.getInt(0));
     assertEquals(110, arrayReader.getInt(1));
     assertTrue(reader.next());
-    assertEquals(20, reader.column(0).getInt());
-    arrayReader = reader.column(1).array();
+    assertEquals(20, reader.scalar(0).getInt());
     assertEquals(3, arrayReader.size());
     assertEquals(200, arrayReader.getInt(0));
     assertEquals(120, arrayReader.getInt(1));
     assertEquals(220, arrayReader.getInt(2));
     assertTrue(reader.next());
-    assertEquals(30, reader.column(0).getInt());
-    arrayReader = reader.column(1).array();
+    assertEquals(30, reader.scalar(0).getInt());
     assertEquals(0, arrayReader.size());
     assertFalse(reader.next());
 
     SingleRowSet rs2 = fixture.rowSetBuilder(batchSchema)
-      .add(10, new int[] {100, 110})
-      .add(20, new int[] {200, 120, 220})
-      .add(30, null)
+      .addRow(10, new int[] {100, 110})
+      .addRow(20, new int[] {200, 120, 220})
+      .addRow(30, null)
       .build();
 
     new RowSetComparison(rs1)
       .verifyAndClearAll(rs2);
   }
 
+  /**
+   * Test filling a row set up to the maximum number of rows.
+   * Values are small enough to prevent filling to the
+   * maximum buffer size.
+   */
+
+  @Test
+  public void testRowBounds() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .build();
+
+    ExtendableRowSet rs = fixture.rowSet(batchSchema);
+    RowSetWriter writer = rs.writer();
+    int count = 0;
+    while (! writer.isFull()) {
+      writer.scalar(0).setInt(count++);
+      writer.save();
+    }
+    writer.done();
+
+    assertEquals(ValueVector.MAX_ROW_COUNT, count);
+    // The writer index points past the writable area.
+    // But, this is fine, the valid() method says we can't
+    // write at this location.
+    assertEquals(ValueVector.MAX_ROW_COUNT, writer.rowIndex());
+    assertEquals(ValueVector.MAX_ROW_COUNT, rs.rowCount());
+    rs.clear();
+  }
+
+  /**
+   * Test filling a row set up to the maximum vector size.
+   * Values in the first column are small enough to prevent filling to the
+   * maximum buffer size, but values in the second column
+   * will reach maximum buffer size before maximum row size.
+   * The result should be the number of rows that fit, with the
+   * partial last row not counting. (A complete application would
+   * reload the partial row into a new row set.)
+   */
+
+  @Test
+  public void testBufferBounds() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .build();
+
+    String varCharValue;
+    try {
+      byte rawValue[] = new byte[512];
+      Arrays.fill(rawValue, (byte) 'X');
+      varCharValue = new String(rawValue, "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalStateException(e);
+    }
+
+    ExtendableRowSet rs = fixture.rowSet(batchSchema);
+    RowSetWriter writer = rs.writer();
+    int count = 0;
+    try {
+
+      // Test overflow. This is not a typical use case: don't want to
+      // hit overflow without overflow handling. In this case, we throw
+      // away the last row because the row set abstraction does not
+      // implement vector overflow other than throwing an exception.
+
+      for (;;) {
+        writer.scalar(0).setInt(count);
+        writer.scalar(1).setString(varCharValue);
+
+        // Won't get here on overflow.
+        writer.save();
+        count++;
+      }
+    } catch (IndexOutOfBoundsException e) {
+      assertTrue(e.getMessage().contains("Overflow"));
+    }
+    writer.done();
+
+    assertTrue(count < ValueVector.MAX_ROW_COUNT);
+    assertEquals(count, writer.rowIndex());
+    assertEquals(count, rs.rowCount());
+    rs.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
new file mode 100644
index 0000000..147b713
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
@@ -0,0 +1,241 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+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.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test the "fill empties" logic for all types for all modes.
+ * This test exploits the dynamic typing ability of the
+ * accessors. Creating an object per value is too slow for
+ * production code, but very handy for tests such as this.
+ * <p>
+ * Note that this test also has the handy side-effect of testing
+ * null handling in the accessor classes.
+ */
+
+public class TestFillEmpties extends SubOperatorTest {
+
+  public static final int ROW_COUNT = 1000;
+
+  /**
+   * Test "fill empties" for required types. Here, the fill value
+   * is more of a convention: 0 (fixed-width) or an empty
+   * entry (variable width.) Some fill value is required to avoid
+   * the alternatives which are either 1) leave the value as
+   * garbage, or 2) raise an exception about the missing value.
+   */
+
+  @Test
+  public void testFillEmptiesRequired() {
+    testFillEmpties(DataMode.REQUIRED);
+  }
+
+  /**
+   * Test "fill empties" for nullable types which are the most
+   * "natural" type for omitted values.
+   * Nullable vectors fill empties with nulls.
+   */
+
+  @Test
+  public void testFillEmptiesNullable() {
+    testFillEmpties(DataMode.OPTIONAL);
+  }
+
+  /**
+   * Test "fill empties" for repeated types.
+   * Drill defines a null (omitted) array as the same thing as
+   * a zero-length array.
+   */
+
+  @Test
+  public void testFillEmptiesRepeated() {
+    testFillEmpties(DataMode.REPEATED);
+  }
+
+  private void testFillEmpties(DataMode mode) {
+    for (MinorType type : MinorType.values()) {
+      switch (type) {
+      case DECIMAL28DENSE:
+      case DECIMAL38DENSE:
+        // Not yet supported
+        break;
+      case GENERIC_OBJECT:
+      case LATE:
+      case LIST:
+      case MAP:
+      case NULL:
+      case UNION:
+        // Writer N/A
+        break;
+      case BIT:
+      case FIXEDBINARY:
+      case FIXEDCHAR:
+      case FIXED16CHAR:
+      case MONEY:
+      case TIMESTAMPTZ:
+      case TIMETZ:
+        // Not supported in Drill
+        break;
+      case DECIMAL18:
+      case DECIMAL28SPARSE:
+      case DECIMAL9:
+      case DECIMAL38SPARSE:
+        doFillEmptiesTest(type, mode, 9, 2);
+        break;
+      default:
+        doFillEmptiesTest(type, mode);
+      }
+    }
+  }
+
+  private void doFillEmptiesTest(MinorType type, DataMode mode, int prec, int scale) {
+    MajorType majorType = MajorType.newBuilder()
+        .setMinorType(type)
+        .setMode(mode)
+        .setPrecision(prec)
+        .setScale(scale)
+        .build();
+    doFillEmptiesTest(majorType);
+  }
+
+  private void doFillEmptiesTest(MinorType type, DataMode mode) {
+    MajorType majorType = MajorType.newBuilder()
+        .setMinorType(type)
+        .setMode(mode)
+        .build();
+    doFillEmptiesTest(majorType);
+  }
+
+  private void doFillEmptiesTest(MajorType majorType) {
+    if (majorType.getMode() == DataMode.REPEATED) {
+      dofillEmptiesRepeated(majorType);
+    } else {
+      doFillEmptiesScalar(majorType);
+    }
+  }
+
+  private void doFillEmptiesScalar(MajorType majorType) {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", majorType)
+        .buildSchema();
+    ExtendableRowSet rs = fixture.rowSet(schema);
+    RowSetWriter writer = rs.writer();
+    ScalarWriter colWriter = writer.scalar(0);
+    ValueType valueType = colWriter.valueType();
+    boolean nullable = majorType.getMode() == DataMode.OPTIONAL;
+    for (int i = 0; i < ROW_COUNT; i++) {
+      if (i % 5 == 0) {
+        colWriter.setObject(RowSetUtilities.testDataFromInt(valueType, majorType, i));
+      }
+      writer.save();
+    }
+    SingleRowSet result = writer.done();
+    RowSetReader reader = result.reader();
+    ScalarReader colReader = reader.scalar(0);
+    MinorType type = majorType.getMinorType();
+    boolean isVariable = (type == MinorType.VARCHAR ||
+                          type == MinorType.VAR16CHAR ||
+                          type == MinorType.VARBINARY);
+    for (int i = 0; i < ROW_COUNT; i++) {
+      assertTrue(reader.next());
+      if (i % 5 != 0) {
+        if (nullable) {
+          // Nullable types fill with nulls.
+
+          assertTrue(colReader.isNull());
+          continue;
+        }
+        if (isVariable) {
+          // Variable width types fill with a zero-length value.
+
+          assertEquals(0, colReader.getBytes().length);
+          continue;
+        }
+      }
+
+      // All other types fill with zero-bytes, interpreted as some form
+      // of zero for each type.
+
+      Object actual = colReader.getObject();
+      Object expected = RowSetUtilities.testDataFromInt(valueType, majorType,
+          i % 5 == 0 ? i : 0);
+      RowSetUtilities.assertEqualValues(
+          majorType.toString().replace('\n', ' ') + "[" + i + "]",
+          valueType, expected, actual);
+    }
+    result.clear();
+  }
+
+  private void dofillEmptiesRepeated(MajorType majorType) {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", majorType)
+        .buildSchema();
+    ExtendableRowSet rs = fixture.rowSet(schema);
+    RowSetWriter writer = rs.writer();
+    ScalarWriter colWriter = writer.array(0).scalar();
+    ValueType valueType = colWriter.valueType();
+    for (int i = 0; i < ROW_COUNT; i++) {
+      if (i % 5 == 0) {
+        // Write two values so we can exercise a bit of the array logic.
+
+        colWriter.setObject(RowSetUtilities.testDataFromInt(valueType, majorType, i));
+        colWriter.setObject(RowSetUtilities.testDataFromInt(valueType, majorType, i+1));
+      }
+      writer.save();
+    }
+    SingleRowSet result = writer.done();
+    RowSetReader reader = result.reader();
+    ScalarElementReader colReader = reader.array(0).elements();
+    for (int i = 0; i < ROW_COUNT; i++) {
+      assertTrue(reader.next());
+      if (i % 5 != 0) {
+        // Empty arrays are defined to be the same as a zero-length array.
+
+        assertEquals(0, colReader.size());
+      } else {
+        for (int j = 0; j < 2; j++) {
+          Object actual = colReader.getObject(j);
+          Object expected = RowSetUtilities.testDataFromInt(valueType, majorType, i + j);
+          RowSetUtilities.assertEqualValues(
+              majorType.toString().replace('\n', ' ') + "[" + i + "][" + j + "]",
+              valueType, expected, actual);
+        }
+      }
+    }
+    result.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFixedWidthWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFixedWidthWriter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFixedWidthWriter.java
new file mode 100644
index 0000000..a27fdf4
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFixedWidthWriter.java
@@ -0,0 +1,444 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors.IntColumnWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test the int writer as a typical example of a fixed-width
+ * writer. Exercises normal writing, writing after a (simulated)
+ * overflow, and filling in empty values.
+ */
+
+public class TestFixedWidthWriter extends SubOperatorTest {
+
+  public static class TestIndex implements ColumnWriterIndex {
+
+    public int index;
+
+    @Override
+    public int vectorIndex() { return index; }
+
+    @Override
+    public void nextElement() { }
+
+    @Override
+    public void rollover() { }
+
+    @Override
+    public int rowStartIndex() { return index; }
+
+    @Override
+    public ColumnWriterIndex outerIndex() { return null; }
+  }
+
+  /**
+   * Basic test to write a contiguous set of values, enough to cause
+   * the vector to double in size twice, then read back the values.
+   */
+
+  @Test
+  public void testWrite() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+
+      writer.startWrite();
+
+      // Write integers.
+      // Write enough that the vector is resized.
+
+      long origAddr = vector.getBuffer().addr();
+      for (int i = 0; i < 3000; i++) {
+        index.index = i;
+        writer.setInt(i * 10);
+      }
+      writer.endWrite();
+
+      // Should have been reallocated.
+
+      assertNotEquals(origAddr, vector.getBuffer().addr());
+
+      // Verify values
+
+      for (int i = 0; i < 3000; i++) {
+        assertEquals(i * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  @Test
+  public void testRestartRow() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write rows, rewriting every other row.
+
+      writer.startRow();
+      index.index = 0;
+      for (int i = 0; i < 50; i++) {
+        writer.setInt(i);
+        if (i % 2 == 0) {
+          writer.saveRow();
+          writer.startRow();
+          index.index++;
+        } else {
+          writer.restartRow();
+        }
+      }
+      writer.endWrite();
+
+      // Verify values
+
+      for (int i = 0; i < 25; i++) {
+        assertEquals(2 * i, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Required, fixed-width vectors are back-filling with 0 to fill in missing
+   * values. While using zero is not strictly SQL compliant, it is better
+   * than failing. (The SQL solution would be to fill with nulls, but a
+   * required vector does not support nulls...)
+   */
+
+  @Test
+  public void testFillEmpties() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write values, skipping four out of five positions,
+      // forcing backfill.
+      // The number of values is odd, forcing the writer to
+      // back-fill at the end as well as between values.
+      // Keep the number of values below the allocation so
+      // that we know all values were initially garbage-filled.
+
+      for (int i = 0; i < 501; i += 5) {
+        index.index = i;
+        writer.startRow();
+        writer.setInt(i);
+        writer.saveRow();
+      }
+      // At end, vector index defined to point one past the
+      // last row. That is, the vector index gives the row count.
+
+      index.index = 504;
+      writer.endWrite();
+
+      // Verify values
+
+      for (int i = 0; i < 504; i++) {
+        assertEquals("Mismatch on " + i,
+            (i%5) == 0 ? i : 0,
+            vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * The rollover method is used during vector overflow.
+   */
+
+  @Test
+  public void testRollover() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of ten values.
+
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setInt(i);
+        writer.saveRow();
+      }
+
+      // Overflow occurs after writing the 11th row
+
+      index.index = 10;
+      writer.startRow();
+      writer.setInt(10);
+
+      // Overflow occurs
+
+      writer.preRollover();
+
+      // Simulate rollover
+
+      for (int i = 0; i < 15; i++) {
+        vector.getMutator().set(i, 0xdeadbeef);
+      }
+      vector.getMutator().set(0, 10);
+
+      writer.postRollover();
+      index.index = 0;
+      writer.saveRow();
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setInt(10 + i);
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 5; i++) {
+        assertEquals(10 + i, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Simulate the case in which the tail end of an overflow
+   * batch has empties. <tt>preRollover()</tt> should back-fill
+   * them with the next offset prior to rollover.
+   */
+
+  @Test
+  public void testRolloverWithEmpties() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of 15 values,
+      // of which 5 are empty.
+
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setInt(i);
+        writer.saveRow();
+      }
+
+      for (int i = 10; i < 15; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Overflow occurs before writing the 16th row
+
+      index.index = 15;
+      writer.startRow();
+
+      // Overflow occurs. This should fill empty offsets.
+
+      writer.preRollover();
+
+      // Verify the first "batch" results
+
+      for (int i = 0; i < 10; i++) {
+        assertEquals(i, vector.getAccessor().get(i));
+      }
+      for (int i = 10; i < 15; i++) {
+        assertEquals(0, vector.getAccessor().get(i));
+      }
+
+      // Simulate rollover
+
+      for (int i = 0; i < 20; i++) {
+        vector.getMutator().set(i, 0xdeadbeef);
+      }
+
+      writer.postRollover();
+      index.index = 0;
+      writer.saveRow();
+
+      // Skip more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 5; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setInt(i + 20);
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 5; i++) {
+        assertEquals(0, vector.getAccessor().get(i));
+      }
+      for (int i = 5; i < 10; i++) {
+        assertEquals(i + 20, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Test the case in which a scalar vector is used in conjunction
+   * with a nullable bits vector. The nullable vector will call the
+   * <tt>skipNulls()</tt> method to avoid writing values for null
+   * entries. (Without the call, the scalar writer will fill the
+   * empty values with zeros.)
+   */
+
+  @Test
+  public void testSkipNulls() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write values, skipping four out of five positions,
+      // skipping nulls.
+      // The loop will cause the vector to double in size.
+      // The number of values is odd, forcing the writer to
+      // skip nulls at the end as well as between values.
+
+      long origAddr = vector.getBuffer().addr();
+      for (int i = 0; i < 3000; i += 5) {
+        index.index = i;
+        writer.startRow();
+        writer.skipNulls();
+        writer.setInt(i);
+        writer.saveRow();
+      }
+      index.index = 3003;
+      writer.startRow();
+      writer.skipNulls();
+      writer.saveRow();
+      writer.endWrite();
+
+      // Should have been reallocated.
+
+      assertNotEquals(origAddr, vector.getBuffer().addr());
+
+      // Verify values. First 1000 were filled with known
+      // garbage values.
+
+      for (int i = 0; i < 1000; i++) {
+        assertEquals("Mismatch at " + i,
+            (i%5) == 0 ? i : 0xdeadbeef,
+            vector.getAccessor().get(i));
+      }
+
+      // Next values are filled with unknown values:
+      // whatever was left in the buffer allocated by Netty.
+
+      for (int i = 1005; i < 3000; i+= 5) {
+        assertEquals(i, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Test resize monitoring. Add a listener to an int writer,
+   * capture each resize, and refuse a resize when the number
+   * of ints exceeds 8K values. This will trigger an overflow,
+   * which will throw an exception which we then check for.
+   */
+
+  @Test
+  public void testSizeLimit() {
+    try (IntVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      IntColumnWriter writer = makeWriter(vector, index);
+      writer.bindListener(new ColumnWriterListener() {
+        int totalAlloc = 4096;
+
+        @Override
+        public void overflowed(ScalarWriter writer) {
+          throw new IllegalStateException("overflow called");
+        }
+
+        @Override
+        public boolean canExpand(ScalarWriter writer, int delta) {
+//          System.out.println("Delta: " + delta);
+          totalAlloc += delta;
+          return totalAlloc < 16_384 * 4;
+        }
+      });
+      writer.startWrite();
+      try {
+        for (int i = 0; ; i++ ) {
+          index.index = i;
+          writer.startRow();
+          writer.setInt(i);
+          writer.saveRow();
+        }
+      }
+      catch(IllegalStateException e) {
+        assertTrue(e.getMessage().contains("overflow called"));
+      }
+
+      // Should have failed on 8192, which doubled vector
+      // to 16K, which was rejected.
+
+      assertEquals(8192, index.index);
+    }
+  }
+
+  private IntVector allocVector(int size) {
+    MaterializedField field =
+        SchemaBuilder.columnSchema("x", MinorType.INT, DataMode.REQUIRED);
+    IntVector vector = new IntVector(field, fixture.allocator());
+    vector.allocateNew(size);
+
+    // Party on the bytes of the vector so we start dirty
+
+    for (int i = 0; i < size; i++) {
+      vector.getMutator().set(i, 0xdeadbeef);
+    }
+    return vector;
+  }
+
+  private IntColumnWriter makeWriter(IntVector vector, TestIndex index) {
+    IntColumnWriter writer = new IntColumnWriter(vector);
+    writer.bindIndex(index);
+
+    assertEquals(ValueType.INTEGER, writer.valueType());
+    return writer;
+  }
+}


[03/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index f51c1a9..c90a734 100644
--- 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
@@ -24,8 +24,40 @@
  * 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.
+ *
+ * <h4>Vector Overflow Handling</h4>
+ *
+ * The writers provide integrated support for detecting and handling vector
+ * overflow. Overflow occurs when a value exceeds some maximum, such as the
+ * 16MB block size in Netty. Overflow handling consists of replacing the
+ * "full" vector with a new, empty vector as part of a new batch. Overflow
+ * handing code must copy partially written values from the "overflow" row
+ * to the new vectors. The classes here do not provide overflow handling,
+ * rather they provide the framework on top of which overflow handling can be
+ * built by a higher level of abstraction.
+ *
+ * <h4>JSON-Like Model</h4>
+ *
+ * The object reader and writer provide a generic, JSON-like interface
+ * to allow any valid combination of readers or writers (generically
+ * accessors):<pre><code>
+ * row : tuple
+ * tuple : (name column) *
+ * column : scalar obj | array obj | tuple obj
+ * scalar obj : scalar accessor
+ * array obj : array accessor
+ * array accessor : element accessor
+ * tuple obj : tuple</code></pre>
  * <p>
- * Drill provides a set of column readers and writers. Compared to those, this
+ * As seen above, the accessor tree starts with a tuple (a row in the form of
+ * a class provided by the consumer.) Each column in the tuple is represented
+ * by an object accesor. That object accessor contains a scalar, tuple or array
+ * accessor. This models Drill's JSON structure: a row can have a list of lists
+ * of tuples that contains lists of ints, say.
+ *
+ * <h4>Comparison with Previous Vector Readers and Writers</h4>
+ *
+ * Drill provides a set of vector readers and writers. Compared to those, this
  * set:
  * <ul>
  * <li>Works with all Drill data types. The other set works only with repeated
@@ -36,23 +68,24 @@
  * 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
+ * The key difference is that this set is designed for both developer ease-of-use
+ * and performance. Developer eas-of-use is a
+ * primary requirement for unit tests. Performance is critical for production
+ * code. 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
+ *
+ * <h4>Overview of the Code Structure</h4>
+ *
+ * {@link ScalarReader} 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>ObjectWriter, ObjectReader</dt>
+ * <dd>Drill follows a JSON data model. A row is a tuple (AKA structure). Each
+ * column is a scalar, a map (AKA tuple, structure) or an array (AKA a repeated
+ * value.)</dd>
  * <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.)
@@ -62,12 +95,8 @@
  * 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
+ * <dt>ScalarWriter, ScalarReader</dt>
+ * <dd>A uniform interface for 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
@@ -98,11 +127,11 @@
  * <dd>The generated accessors: one for each combination of write/read, data
  * (minor) type and cardinality (data model).
  * <dd>
- * <dt>RowIndex</dt>
+ * <dt>ColumnReaderIndex, ColumnWriterIndex</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.)
+ * Instead, the implementation sets the row position (with, say an iterator.)
  * Then, all columns implicitly return values for the current row.
  * <p>
  * Different implementations of the row index handle the case of no selection
@@ -122,6 +151,16 @@
  * The column index picks out the x coordinate (horizontal position along the
  * columns.)</dt>
  * </dl>
+ * <h4>Column Writer Optimizations</h4>
+ * The writer classes here started as a simple abstraction on top of the existing
+ * vector mutators. The classes were then recruited for use in a new writer
+ * abstraction for Drill's record readers. At that point, performance became
+ * critical. The key to performance is to bypass the vector and the mutator and
+ * instead work with the Netty direct memory functions. This seems a risky
+ * approach until we realize that the writers form a very clear interface:
+ * the same interface supported the original mutator-based implementation and
+ * the revised Netty-based implementation. The benefit, however, is stark;
+ * the direct-to-Netty version is up to 4x faster (for repeated types).
  */
 
 package org.apache.drill.exec.vector.accessor;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractArrayReader.java
new file mode 100644
index 0000000..7fb0c9d
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractArrayReader.java
@@ -0,0 +1,188 @@
+/*
+ * 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.reader;
+
+import org.apache.drill.exec.vector.UInt4Vector.Accessor;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * 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 implements ArrayReader {
+
+  /**
+   * Object representation of an array reader.
+   */
+
+  public static class ArrayObjectReader extends AbstractObjectReader {
+
+    private AbstractArrayReader arrayReader;
+
+    public ArrayObjectReader(AbstractArrayReader arrayReader) {
+      this.arrayReader = arrayReader;
+    }
+
+    @Override
+    public void bindIndex(ColumnReaderIndex index) {
+      arrayReader.bindIndex(index);
+    }
+
+    @Override
+    public ObjectType type() {
+      return ObjectType.ARRAY;
+    }
+
+    @Override
+    public ArrayReader array() {
+      return arrayReader;
+    }
+
+    @Override
+    public ScalarElementReader elements() {
+      return arrayReader.elements();
+    }
+
+    @Override
+    public Object getObject() {
+      return arrayReader.getObject();
+    }
+
+    @Override
+    public String getAsString() {
+      return arrayReader.getAsString();
+    }
+
+    @Override
+    public void reposition() {
+      arrayReader.reposition();
+    }
+  }
+
+  public static class BaseElementIndex {
+    private final ColumnReaderIndex base;
+    protected int startOffset;
+    protected int length;
+
+    public BaseElementIndex(ColumnReaderIndex base) {
+      this.base = base;
+    }
+
+    public int batchIndex() {
+      return base.batchIndex();
+    }
+
+    public void reset(int startOffset, int length) {
+      assert length >= 0;
+      assert startOffset >= 0;
+      this.startOffset = startOffset;
+      this.length = length;
+    }
+
+    public int size() { return length; }
+
+    public int elementIndex(int index) {
+      if (index < 0 || length <= index) {
+        throw new IndexOutOfBoundsException("Index = " + index + ", length = " + length);
+      }
+      return startOffset + index;
+    }
+  }
+
+  private final Accessor accessor;
+  private final VectorAccessor vectorAccessor;
+  protected ColumnReaderIndex baseIndex;
+  protected BaseElementIndex elementIndex;
+
+  public AbstractArrayReader(RepeatedValueVector vector) {
+    accessor = vector.getOffsetVector().getAccessor();
+    vectorAccessor = null;
+  }
+
+  public AbstractArrayReader(VectorAccessor vectorAccessor) {
+    accessor = null;
+    this.vectorAccessor = vectorAccessor;
+  }
+
+  public void bindIndex(ColumnReaderIndex index) {
+    baseIndex = index;
+    if (vectorAccessor != null) {
+      vectorAccessor.bind(index);
+    }
+  }
+
+  private Accessor accessor() {
+    if (accessor != null) {
+      return accessor;
+    }
+    return ((RepeatedValueVector) (vectorAccessor.vector())).getOffsetVector().getAccessor();
+  }
+
+  public void reposition() {
+    final int index = baseIndex.vectorIndex();
+    final Accessor curAccesssor = accessor();
+    final int startPosn = curAccesssor.get(index);
+    elementIndex.reset(startPosn, curAccesssor.get(index + 1) - startPosn);
+  }
+
+  @Override
+  public int size() { return elementIndex.size(); }
+
+  @Override
+  public ScalarElementReader elements() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ObjectReader entry(int index) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleReader tuple(int index) {
+    return entry(index).tuple();
+  }
+
+  @Override
+  public ArrayReader array(int index) {
+    return entry(index).array();
+  }
+
+  @Override
+  public ObjectReader entry() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleReader tuple() {
+    return entry().tuple();
+  }
+
+  @Override
+  public ArrayReader array() {
+    return entry().array();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractObjectReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractObjectReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractObjectReader.java
new file mode 100644
index 0000000..59a066e
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractObjectReader.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.reader;
+
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+
+public abstract class AbstractObjectReader implements ObjectReader {
+
+  public abstract void bindIndex(ColumnReaderIndex index);
+
+  public void reposition() { }
+
+  @Override
+  public ScalarReader scalar() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleReader tuple() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ArrayReader array() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ScalarElementReader elements() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
new file mode 100644
index 0000000..afa0cb7
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/AbstractTupleReader.java
@@ -0,0 +1,189 @@
+/*
+ * 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.reader;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+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 abstract class AbstractTupleReader implements TupleReader {
+
+  public static class TupleObjectReader extends AbstractObjectReader {
+
+    private AbstractTupleReader tupleReader;
+
+    public TupleObjectReader(AbstractTupleReader tupleReader) {
+      this.tupleReader = tupleReader;
+    }
+
+    @Override
+    public void bindIndex(ColumnReaderIndex index) {
+      tupleReader.bindIndex(index);
+    }
+
+    @Override
+    public ObjectType type() {
+      return ObjectType.TUPLE;
+    }
+
+    @Override
+    public TupleReader tuple() {
+      return tupleReader;
+    }
+
+    @Override
+    public Object getObject() {
+      return tupleReader.getObject();
+    }
+
+    @Override
+    public String getAsString() {
+      return tupleReader.getAsString();
+    }
+
+    @Override
+    public void reposition() {
+      tupleReader.reposition();
+    }
+  }
+
+  protected final TupleMetadata schema;
+  private final AbstractObjectReader readers[];
+
+  protected AbstractTupleReader(TupleMetadata schema, AbstractObjectReader readers[]) {
+    this.schema = schema;
+    this.readers = readers;
+  }
+
+  public void bindIndex(ColumnReaderIndex index) {
+    for (int i = 0; i < readers.length; i++) {
+      readers[i].bindIndex(index);
+    }
+  }
+
+  @Override
+  public TupleMetadata schema() { return schema; }
+
+  @Override
+  public int columnCount() { return schema().size(); }
+
+  @Override
+  public ObjectReader column(int colIndex) {
+    return readers[colIndex];
+  }
+
+  @Override
+  public ObjectReader column(String colName) {
+    int index = schema.index(colName);
+    if (index == -1) {
+      return null; }
+    return readers[index];
+  }
+
+  @Override
+  public ScalarReader scalar(int colIndex) {
+    return column(colIndex).scalar();
+  }
+
+  @Override
+  public ScalarReader scalar(String colName) {
+    return column(colName).scalar();
+  }
+
+  @Override
+  public TupleReader tuple(int colIndex) {
+    return column(colIndex).tuple();
+  }
+
+  @Override
+  public TupleReader tuple(String colName) {
+    return column(colName).tuple();
+  }
+
+  @Override
+  public ArrayReader array(int colIndex) {
+    return column(colIndex).array();
+  }
+
+  @Override
+  public ArrayReader array(String colName) {
+    return column(colName).array();
+  }
+
+  @Override
+  public ObjectType type(int colIndex) {
+    return column(colIndex).type();
+  }
+
+  @Override
+  public ObjectType type(String colName) {
+    return column(colName).type();
+  }
+
+  @Override
+  public ScalarElementReader elements(int colIndex) {
+    return column(colIndex).elements();
+  }
+
+  @Override
+  public ScalarElementReader elements(String colName) {
+    return column(colName).elements();
+  }
+
+  public void reposition() {
+    for (int i = 0; i < columnCount(); i++) {
+      readers[i].reposition();
+    }
+  }
+
+  @Override
+  public Object getObject() {
+    List<Object> elements = new ArrayList<>();
+    for (int i = 0; i < columnCount(); i++) {
+      elements.add(readers[i].getObject());
+    }
+    return elements;
+  }
+
+  @Override
+  public String getAsString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("(");
+    for (int i = 0; i < columnCount(); i++) {
+      if (i > 0) {
+        buf.append( ", " );
+      }
+      buf.append(readers[i].getAsString());
+    }
+    buf.append(")");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseElementReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseElementReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseElementReader.java
new file mode 100644
index 0000000..f32c101
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseElementReader.java
@@ -0,0 +1,187 @@
+/*
+ * 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.reader;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.impl.AccessorUtilities;
+import org.joda.time.Period;
+
+public abstract class BaseElementReader implements ScalarElementReader {
+
+  public static class ScalarElementObjectReader extends AbstractObjectReader {
+
+    private BaseElementReader elementReader;
+
+    public ScalarElementObjectReader(BaseElementReader elementReader) {
+      this.elementReader = elementReader;
+    }
+
+    @Override
+    public void bindIndex(ColumnReaderIndex index) {
+      elementReader.bindIndex((ElementReaderIndex) index);
+    }
+
+    @Override
+    public ObjectType type() {
+      return ObjectType.SCALAR;
+    }
+
+    @Override
+    public ScalarElementReader elements() {
+      return elementReader;
+    }
+
+    @Override
+    public Object getObject() {
+      // Simple: return elements as an object list.
+      // If really needed, could return as a typed array, but that
+      // is a bit of a hassle.
+
+      List<Object> elements = new ArrayList<>();
+      for (int i = 0; i < elementReader.size(); i++) {
+        elements.add(elementReader.getObject(i));
+      }
+      return elements;
+    }
+
+    @Override
+    public String getAsString() {
+      StringBuilder buf = new StringBuilder();
+      buf.append("[");
+      for (int i = 0; i < elementReader.size(); i++) {
+        if (i > 0) {
+          buf.append( ", " );
+        }
+        buf.append(elementReader.getAsString(i));
+      }
+      buf.append("]");
+      return buf.toString();
+    }
+  }
+
+  protected ElementReaderIndex vectorIndex;
+  protected VectorAccessor vectorAccessor;
+
+  public abstract void bindVector(ValueVector vector);
+
+  public void bindVector(MajorType majorType, VectorAccessor va) {
+    vectorAccessor = va;
+  }
+
+  protected void bindIndex(ElementReaderIndex rowIndex) {
+    this.vectorIndex = rowIndex;
+  }
+
+  @Override
+  public int size() { return vectorIndex.size(); }
+
+  @Override
+  public Object getObject(int index) {
+    if (isNull(index)) {
+      return "null";
+    }
+    switch (valueType()) {
+    case BYTES:
+      return getBytes(index);
+    case DECIMAL:
+      return getDecimal(index);
+    case DOUBLE:
+      return getDouble(index);
+    case INTEGER:
+      return getInt(index);
+    case LONG:
+      return getLong(index);
+    case PERIOD:
+      return getPeriod(index);
+    case STRING:
+      return getString(index);
+    default:
+      throw new IllegalStateException("Unexpected type: " + valueType());
+    }
+  }
+
+  @Override
+  public String getAsString(int index) {
+    switch (valueType()) {
+    case BYTES:
+      return AccessorUtilities.bytesToString(getBytes(index));
+    case DOUBLE:
+      return Double.toString(getDouble(index));
+    case INTEGER:
+      return Integer.toString(getInt(index));
+    case LONG:
+      return Long.toString(getLong(index));
+    case STRING:
+      return "\"" + getString(index) + "\"";
+    case DECIMAL:
+      return getDecimal(index).toPlainString();
+    case PERIOD:
+      return getPeriod(index).normalizedStandard().toString();
+    default:
+      throw new IllegalArgumentException("Unsupported type " + valueType());
+    }
+  }
+
+  @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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseScalarReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseScalarReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseScalarReader.java
new file mode 100644
index 0000000..fb9a711
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/BaseScalarReader.java
@@ -0,0 +1,189 @@
+/*
+ * 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.reader;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.impl.AccessorUtilities;
+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 BaseScalarReader implements ScalarReader {
+
+  public static class ScalarObjectReader extends AbstractObjectReader {
+
+    private BaseScalarReader scalarReader;
+
+    public ScalarObjectReader(BaseScalarReader scalarReader) {
+      this.scalarReader = scalarReader;
+    }
+
+    @Override
+    public void bindIndex(ColumnReaderIndex index) {
+      scalarReader.bindIndex(index);
+    }
+
+    @Override
+    public ObjectType type() {
+      return ObjectType.SCALAR;
+    }
+
+    @Override
+    public ScalarReader scalar() {
+      return scalarReader;
+    }
+
+    @Override
+    public Object getObject() {
+      return scalarReader.getObject();
+    }
+
+    @Override
+    public String getAsString() {
+      return scalarReader.getAsString();
+    }
+  }
+
+  protected ColumnReaderIndex vectorIndex;
+  protected VectorAccessor vectorAccessor;
+
+  public static ScalarObjectReader build(ValueVector vector, BaseScalarReader reader) {
+    reader.bindVector(vector);
+    return new ScalarObjectReader(reader);
+  }
+
+  public static AbstractObjectReader build(MajorType majorType, VectorAccessor va,
+                                           BaseScalarReader reader) {
+    reader.bindVector(majorType, va);
+    return new ScalarObjectReader(reader);
+  }
+
+  public abstract void bindVector(ValueVector vector);
+
+  protected void bindIndex(ColumnReaderIndex rowIndex) {
+    this.vectorIndex = rowIndex;
+    if (vectorAccessor != null) {
+      vectorAccessor.bind(rowIndex);
+    }
+  }
+
+  public void bindVector(MajorType majorType, VectorAccessor va) {
+    vectorAccessor = va;
+  }
+
+  @Override
+  public Object getObject() {
+    if (isNull()) {
+      return null;
+    }
+    switch (valueType()) {
+    case BYTES:
+      return getBytes();
+    case DECIMAL:
+      return getDecimal();
+    case DOUBLE:
+      return getDouble();
+    case INTEGER:
+      return getInt();
+    case LONG:
+      return getLong();
+    case PERIOD:
+      return getPeriod();
+    case STRING:
+      return getString();
+    default:
+      throw new IllegalStateException("Unexpected type: " + valueType());
+    }
+  }
+
+  @Override
+  public String getAsString() {
+    if (isNull()) {
+      return "null";
+    }
+    switch (valueType()) {
+    case BYTES:
+      return AccessorUtilities.bytesToString(getBytes());
+    case DOUBLE:
+      return Double.toString(getDouble());
+    case INTEGER:
+      return Integer.toString(getInt());
+    case LONG:
+      return Long.toString(getLong());
+    case STRING:
+      return "\"" + getString() + "\"";
+    case DECIMAL:
+      return getDecimal().toPlainString();
+    case PERIOD:
+      return getPeriod().normalizedStandard().toString();
+    default:
+      throw new IllegalArgumentException("Unsupported type " + valueType());
+    }
+  }
+
+  @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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ColumnReaderFactory.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ColumnReaderFactory.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ColumnReaderFactory.java
new file mode 100644
index 0000000..0bcb6e2
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ColumnReaderFactory.java
@@ -0,0 +1,109 @@
+/*
+ * 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.reader;
+
+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.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * Gather generated reader classes into a set of class tables to allow rapid
+ * run-time creation of readers. Builds the reader and its object reader
+ * wrapper which binds the vector to the reader.
+ */
+
+@SuppressWarnings("unchecked")
+public class ColumnReaderFactory {
+
+  private static final int typeCount = MinorType.values().length;
+  private static final Class<? extends BaseScalarReader> requiredReaders[] = new Class[typeCount];
+  private static final Class<? extends BaseScalarReader> nullableReaders[] = new Class[typeCount];
+  private static final Class<? extends BaseElementReader> elementReaders[] = new Class[typeCount];
+
+  static {
+    ColumnAccessors.defineRequiredReaders(requiredReaders);
+    ColumnAccessors.defineNullableReaders(nullableReaders);
+    ColumnAccessors.defineArrayReaders(elementReaders);
+  }
+
+  public static AbstractObjectReader buildColumnReader(ValueVector vector) {
+    MajorType major = vector.getField().getType();
+    MinorType type = major.getMinorType();
+    DataMode mode = major.getMode();
+
+    switch (type) {
+    case GENERIC_OBJECT:
+    case LATE:
+    case NULL:
+    case LIST:
+    case MAP:
+      throw new UnsupportedOperationException(type.toString());
+    default:
+      switch (mode) {
+      case OPTIONAL:
+        return BaseScalarReader.build(vector, newAccessor(type, nullableReaders));
+      case REQUIRED:
+        return BaseScalarReader.build(vector, newAccessor(type, requiredReaders));
+      case REPEATED:
+        return ScalarArrayReader.build((RepeatedValueVector) vector, newAccessor(type, elementReaders));
+      default:
+        throw new UnsupportedOperationException(mode.toString());
+      }
+    }
+  }
+
+  public static AbstractObjectReader buildColumnReader(MajorType majorType, VectorAccessor va) {
+    MinorType type = majorType.getMinorType();
+    DataMode mode = majorType.getMode();
+
+    switch (type) {
+    case GENERIC_OBJECT:
+    case LATE:
+    case NULL:
+    case LIST:
+    case MAP:
+      throw new UnsupportedOperationException(type.toString());
+    default:
+      switch (mode) {
+      case OPTIONAL:
+        return BaseScalarReader.build(majorType, va, newAccessor(type, nullableReaders));
+      case REQUIRED:
+        return BaseScalarReader.build(majorType, va, newAccessor(type, requiredReaders));
+      case REPEATED:
+        return ScalarArrayReader.build(majorType, va, newAccessor(type, elementReaders));
+      default:
+        throw new UnsupportedOperationException(mode.toString());
+      }
+    }
+  }
+
+  public static <T> T newAccessor(MinorType type, Class<? extends T> accessors[]) {
+    try {
+      Class<? extends T> accessorClass = accessors[type.ordinal()];
+      if (accessorClass == null) {
+        throw new UnsupportedOperationException(type.toString());
+      }
+      return accessorClass.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ElementReaderIndex.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ElementReaderIndex.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ElementReaderIndex.java
new file mode 100644
index 0000000..9985edc
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ElementReaderIndex.java
@@ -0,0 +1,24 @@
+/*
+ * 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.reader;
+
+public interface ElementReaderIndex {
+  int batchIndex();
+  int size();
+  int vectorIndex(int posn);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/FixedWidthElementReaderIndex.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/FixedWidthElementReaderIndex.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/FixedWidthElementReaderIndex.java
new file mode 100644
index 0000000..4f3aeeb
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/FixedWidthElementReaderIndex.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.reader;
+
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.reader.AbstractArrayReader.BaseElementIndex;
+
+/**
+ * Index into the vector of elements for a repeated vector.
+ * Keeps track of the current offset in terms of value positions.
+ */
+
+public class FixedWidthElementReaderIndex extends BaseElementIndex implements ElementReaderIndex {
+
+  public FixedWidthElementReaderIndex(ColumnReaderIndex base) {
+    super(base);
+  }
+
+  @Override
+  public int vectorIndex(int posn) {
+    return elementIndex(posn);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.java
new file mode 100644
index 0000000..66bc067
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/MapReader.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.reader;
+
+import java.util.List;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+
+/**
+ * Reader for a Drill Map type. Maps are actually tuples, just like rows.
+ */
+
+public class MapReader extends AbstractTupleReader {
+
+  protected MapReader(ColumnMetadata schema, AbstractObjectReader readers[]) {
+    super(schema.mapSchema(), readers);
+  }
+
+  public static TupleObjectReader build(ColumnMetadata schema, AbstractObjectReader readers[]) {
+    return new TupleObjectReader(new MapReader(schema, readers));
+  }
+
+  public static AbstractObjectReader build(ColumnMetadata metadata,
+      List<AbstractObjectReader> readers) {
+    AbstractObjectReader readerArray[] = new AbstractObjectReader[readers.size()];
+    return build(metadata, readers.toArray(readerArray));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ObjectArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ObjectArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ObjectArrayReader.java
new file mode 100644
index 0000000..9ed89f1
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ObjectArrayReader.java
@@ -0,0 +1,159 @@
+/*
+ * 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.reader;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * Reader for an array of either tuples or other arrays.
+ */
+
+public class ObjectArrayReader extends AbstractArrayReader {
+
+  /**
+   * Index into the vector of elements for a repeated vector.
+   * Keeps track of the current offset in terms of value positions.
+   * This is a derived index. The base index points to an entry
+   * in the offset vector for the array. This inner index picks
+   * off elements within the range of offsets for that one entry.
+   * For example:<pre><code>
+   * [ ... 100 105 ...]
+   * </code></pre>In the above the value 100 might be at outer
+   * offset 5. The inner array will pick off the five values
+   * 100...104.
+   * <p>
+   * Because arrays allow random access on read, the inner offset
+   * is reset on each access to the array.
+   */
+
+  public static class ObjectElementReaderIndex extends BaseElementIndex implements ColumnReaderIndex {
+
+    private int posn;
+
+    public ObjectElementReaderIndex(ColumnReaderIndex base) {
+      super(base);
+    }
+
+    @Override
+    public int vectorIndex() {
+      return startOffset + posn;
+    }
+
+    public void set(int index) {
+      if (index < 0 ||  length <= index) {
+        throw new IndexOutOfBoundsException("Index = " + index + ", length = " + length);
+      }
+      posn = index;
+    }
+
+    public int posn() { return posn; }
+  }
+
+  /**
+   * Reader for each element.
+   */
+
+  private final AbstractObjectReader elementReader;
+
+  /**
+   * Index used to access elements.
+   */
+
+  private ObjectElementReaderIndex objElementIndex;
+
+  private ObjectArrayReader(RepeatedValueVector vector, AbstractObjectReader elementReader) {
+    super(vector);
+    this.elementReader = elementReader;
+  }
+
+  private ObjectArrayReader(VectorAccessor vectorAccessor, AbstractObjectReader elementReader) {
+    super(vectorAccessor);
+    this.elementReader = elementReader;
+  }
+
+  public static ArrayObjectReader build(RepeatedValueVector vector,
+                                        AbstractObjectReader elementReader) {
+    return new ArrayObjectReader(
+        new ObjectArrayReader(vector, elementReader));
+  }
+
+  public static AbstractObjectReader build(VectorAccessor vectorAccessor,
+                                           AbstractObjectReader elementReader) {
+    return new ArrayObjectReader(
+        new ObjectArrayReader(vectorAccessor, elementReader));
+  }
+
+  @Override
+  public void bindIndex(ColumnReaderIndex index) {
+    super.bindIndex(index);
+    objElementIndex = new ObjectElementReaderIndex(baseIndex);
+    elementIndex = objElementIndex;
+    elementReader.bindIndex(objElementIndex);
+  }
+
+  @Override
+  public ObjectType entryType() {
+    return elementReader.type();
+  }
+
+  @Override
+  public void setPosn(int index) {
+    objElementIndex.set(index);
+    elementReader.reposition();
+  }
+
+  @Override
+  public ObjectReader entry() {
+    return elementReader;
+  }
+
+  @Override
+  public ObjectReader entry(int index) {
+    setPosn(index);
+    return entry();
+  }
+
+  @Override
+  public Object getObject() {
+    List<Object> array = new ArrayList<>();
+    for (int i = 0; i < objElementIndex.size(); i++) {
+      array.add(entry(i).getObject());
+    }
+    return array;
+  }
+
+  @Override
+  public String getAsString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("[");
+    for (int i = 0; i < size(); i++) {
+      if (i > 0) {
+        buf.append( ", " );
+      }
+      buf.append(entry(i).getAsString());
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ScalarArrayReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ScalarArrayReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ScalarArrayReader.java
new file mode 100644
index 0000000..d93e4a5
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/ScalarArrayReader.java
@@ -0,0 +1,102 @@
+/*
+ * 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.reader;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+public class ScalarArrayReader extends AbstractArrayReader {
+
+  private final BaseElementReader elementReader;
+
+  private ScalarArrayReader(RepeatedValueVector vector,
+                           BaseElementReader elementReader) {
+    super(vector);
+    this.elementReader = elementReader;
+  }
+
+  private ScalarArrayReader(VectorAccessor va,
+                            BaseElementReader elementReader) {
+    super(va);
+    this.elementReader = elementReader;
+  }
+
+  public static ArrayObjectReader build(RepeatedValueVector vector,
+                                        BaseElementReader elementReader) {
+    elementReader.bindVector(vector.getDataVector());
+    return new ArrayObjectReader(new ScalarArrayReader(vector, elementReader));
+  }
+
+  public static ArrayObjectReader build(MajorType majorType, VectorAccessor va,
+                                        BaseElementReader elementReader) {
+    elementReader.bindVector(majorType, va);
+    return new ArrayObjectReader(new ScalarArrayReader(va, elementReader));
+  }
+
+  @Override
+  public void bindIndex(ColumnReaderIndex index) {
+    super.bindIndex(index);
+    FixedWidthElementReaderIndex fwElementIndex = new FixedWidthElementReaderIndex(baseIndex);
+    elementIndex = fwElementIndex;
+    elementReader.bindIndex(fwElementIndex);
+  }
+
+  @Override
+  public ObjectType entryType() {
+    return ObjectType.SCALAR;
+  }
+
+  @Override
+  public ScalarElementReader elements() {
+    return elementReader;
+  }
+
+  @Override
+  public void setPosn(int index) {
+    throw new IllegalStateException("setPosn() not supported for scalar arrays");
+  }
+
+  @Override
+  public Object getObject() {
+    List<Object> elements = new ArrayList<>();
+    for (int i = 0; i < size(); i++) {
+      elements.add(elementReader.getObject(i));
+    }
+    return elements;
+  }
+
+  @Override
+  public String getAsString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("[");
+    for (int i = 0; i < size(); i++) {
+      if (i > 0) {
+        buf.append( ", " );
+      }
+      buf.append(elementReader.getAsString(i));
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/VectorAccessor.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/VectorAccessor.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/VectorAccessor.java
new file mode 100644
index 0000000..1cf2a19
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/VectorAccessor.java
@@ -0,0 +1,26 @@
+/*
+ * 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.reader;
+
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
+
+public interface VectorAccessor {
+  void bind(ColumnReaderIndex index);
+  ValueVector vector();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/package-info.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/package-info.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/package-info.java
new file mode 100644
index 0000000..a94d2e8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/reader/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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 the reader hierarchy as explained in the API package.
+ * The only caveat is that a simplification is provided for arrays of
+ * scalar values: rather than a scalar reader for each value, the
+ * {#link ScalarElementReader} class provides access to the entire array
+ * via indexed get methods.
+ */
+
+package org.apache.drill.exec.vector.accessor.reader;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
new file mode 100644
index 0000000..e6e29b4
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractArrayWriter.java
@@ -0,0 +1,348 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * 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.
+ * <p>
+ * This class represents the array as a whole. In practice that means building
+ * the offset vector. The array is associated with an element object that
+ * manages writing to the scalar, array or tuple that is the array element. Note
+ * that this representation makes little use of the methods in the "Repeated"
+ * vector class: instead it works directly with the offset and element vectors.
+ * <p>
+ * An array has a one-to-many relationship with its children. Starting an array
+ * prepares for writing the first element. Each element must be saved by calling
+ * <tt>endValue()</tt>. This is done automatically for scalars (since there is
+ * exactly one value per element), but must be done via the client code for
+ * arrays of arrays or tuples. Valid state transitions:
+ *
+ * <table border=1>
+ * <tr><th>Public API</th><th>Array Event</th><th>Offset Event</th><th>Element Event</th></tr>
+ * <tr><td>startBatch()</td>
+ *     <td>startWrite()</td>
+ *     <td>startWrite()</td>
+ *     <td>startWrite()</td></tr>
+ * <tr><td>start() (new row)</td>
+ *     <td>startRow()</td>
+ *     <td>startRow()</td>
+ *     <td>startRow()</td></tr>
+ * <tr><td>start() (without save)</td>
+ *     <td>restartRow()</td>
+ *     <td>restartRow()</td>
+ *     <td>restartRow()</td></tr>
+ * <tr><td>save() (array)</td>
+ *     <td>saveValue()</td>
+ *     <td>saveValue()</td>
+ *     <td>saveValue()</td></tr>
+ * <tr><td>save() (row)</td>
+ *     <td colspan=3>See subclasses.</td></tr>
+ * <tr><td>harvest()</td>
+ *     <td>endWrite()</td>
+ *     <td>endWrite()</td>
+ *     <td>endWrite()</td></tr>
+ * </table>
+ *
+ * Some items to note:
+ * <ul>
+ * <li>Batch and row events are passed to the element.</li>
+ * <li>Each element is saved via a call to {@link #save()} on the array.
+ *     Without this call, the element value is discarded. This is necessary
+ *     because the array always has an active element: no "startElement"
+ *     method is necessary. This also means that any unsaved element values
+ *     can be discarded simply by omitting a call to <tt>save()</tt>.</li>
+ * <li>Since elements must be saved individually, the call to
+ *     {@link #saveRow()} <i>does not</i> call <tt>saveValue()</tt>. This
+ *     is an important distinction between an array and a tuple.</li>
+ * <li>The offset and element writers are treated equally: the same events
+ *     are passed to both.</li>
+ * </ul>
+ */
+
+public abstract class AbstractArrayWriter implements ArrayWriter, WriterEvents {
+
+  /**
+   * Object representation of an array writer.
+   */
+
+  public static class ArrayObjectWriter extends AbstractObjectWriter {
+
+    private AbstractArrayWriter arrayWriter;
+
+    public ArrayObjectWriter(ColumnMetadata schema, AbstractArrayWriter arrayWriter) {
+      super(schema);
+      this.arrayWriter = arrayWriter;
+    }
+
+    @Override
+    public ObjectType type() { return ObjectType.ARRAY; }
+
+    @Override
+    public void set(Object value) {
+      arrayWriter.setObject(value);
+    }
+
+    @Override
+    public ArrayWriter array() { return arrayWriter; }
+
+    @Override
+    public WriterEvents events() { return arrayWriter; }
+
+    @Override
+    public void bindListener(ColumnWriterListener listener) {
+      arrayWriter.bindListener(listener);
+    }
+
+    @Override
+    public void bindListener(TupleWriterListener listener) {
+      arrayWriter.bindListener(listener);
+    }
+
+    @Override
+    public void dump(HierarchicalFormatter format) {
+      format
+        .startObject(this)
+        .attribute("arrayWriter");
+      arrayWriter.dump(format);
+      format.endObject();
+    }
+  }
+
+  public static abstract class BaseArrayWriter extends AbstractArrayWriter {
+
+    /**
+     * Index into the vector of elements for a repeated vector.
+     * Keeps track of the current offset in terms of value positions.
+     * Forwards overflow events to the base index.
+     */
+
+    public class ArrayElementWriterIndex implements ColumnWriterIndex {
+
+      private int elementIndex;
+
+      public void reset() { elementIndex = 0; }
+
+      @Override
+      public int vectorIndex() { return elementIndex + offsetsWriter.nextOffset(); }
+
+      @Override
+      public int rowStartIndex() { return offsetsWriter.rowStartOffset(); }
+
+      public int arraySize() { return elementIndex; }
+
+      @Override
+      public void nextElement() { }
+
+      public final void next() { elementIndex++; }
+
+      public int valueStartOffset() { return offsetsWriter.nextOffset(); }
+
+      @Override
+      public void rollover() { }
+
+      @Override
+      public ColumnWriterIndex outerIndex() {
+        return outerIndex;
+      }
+
+      @Override
+      public String toString() {
+        return new StringBuilder()
+          .append("[")
+          .append(getClass().getSimpleName())
+          .append(" elementIndex = ")
+          .append(elementIndex)
+          .append("]")
+          .toString();
+      }
+    }
+
+    private final OffsetVectorWriter offsetsWriter;
+    private ColumnWriterIndex outerIndex;
+    protected ArrayElementWriterIndex elementIndex;
+
+    public BaseArrayWriter(UInt4Vector offsetVector, AbstractObjectWriter elementObjWriter) {
+      super(elementObjWriter);
+      offsetsWriter = new OffsetVectorWriter(offsetVector);
+    }
+
+    @Override
+    public void bindIndex(ColumnWriterIndex index) {
+      assert elementIndex != null;
+      outerIndex = index;
+      offsetsWriter.bindIndex(index);
+      elementObjWriter.events().bindIndex(elementIndex);
+    }
+
+    @Override
+    public ColumnWriterIndex writerIndex() { return outerIndex; }
+
+    @Override
+    public int size() { return elementIndex.arraySize(); }
+
+    @Override
+    public void startWrite() {
+      elementIndex.reset();
+      offsetsWriter.startWrite();
+      elementObjWriter.events().startWrite();
+    }
+
+    @Override
+    public void startRow() {
+
+      // Starting an outer value automatically starts the first
+      // element value. If no elements are written, then this
+      // inner start will just be ignored.
+
+      offsetsWriter.startRow();
+      elementIndex.reset();
+      elementObjWriter.events().startRow();
+    }
+
+    @Override
+    public void endArrayValue() {
+      offsetsWriter.setNextOffset(elementIndex.vectorIndex());
+      elementIndex.reset();
+    }
+
+    @Override
+    public void restartRow() {
+      offsetsWriter.restartRow();
+      elementIndex.reset();
+      elementObjWriter.events().restartRow();
+    }
+
+    @Override
+    public void saveRow() {
+      offsetsWriter.saveRow();
+      elementObjWriter.events().saveRow();
+    }
+
+    @Override
+    public void endWrite() {
+      offsetsWriter.endWrite();
+      elementObjWriter.events().endWrite();
+    }
+
+    @Override
+    public void preRollover() {
+      elementObjWriter.events().preRollover();
+      offsetsWriter.preRollover();
+    }
+
+    @Override
+    public void postRollover() {
+      elementObjWriter.events().postRollover();
+
+      // Reset the index after the vectors: the vectors
+      // need the old row start index from the index.
+
+      offsetsWriter.postRollover();
+      elementIndex.rollover();
+    }
+
+    @Override
+    public int lastWriteIndex() { return outerIndex.vectorIndex(); }
+
+    /**
+     * Return the writer for the offset vector for this array. Primarily used
+     * to handle overflow; other clients should not attempt to muck about with
+     * the offset vector directly.
+     *
+     * @return the writer for the offset vector associated with this array
+     */
+
+    @Override
+    public OffsetVectorWriter offsetWriter() { return offsetsWriter; }
+
+    @Override
+    public void bindListener(ColumnWriterListener listener) {
+      elementObjWriter.bindListener(listener);
+    }
+
+    @Override
+    public void bindListener(TupleWriterListener listener) {
+      elementObjWriter.bindListener(listener);
+    }
+
+    @Override
+    public void dump(HierarchicalFormatter format) {
+      format.extend();
+      super.dump(format);
+      format
+        .attribute("elementIndex", elementIndex.vectorIndex())
+        .attribute("offsetsWriter");
+      offsetsWriter.dump(format);
+    }
+  }
+
+  protected final AbstractObjectWriter elementObjWriter;
+
+  public AbstractArrayWriter(AbstractObjectWriter elementObjWriter) {
+    this.elementObjWriter = elementObjWriter;
+  }
+
+  @Override
+  public ObjectType entryType() {
+    return elementObjWriter.type();
+  }
+
+  @Override
+  public ObjectWriter entry() { return elementObjWriter; }
+
+  @Override
+  public ScalarWriter scalar() {
+    return elementObjWriter.scalar();
+  }
+
+  @Override
+  public TupleWriter tuple() {
+    return elementObjWriter.tuple();
+  }
+
+  @Override
+  public ArrayWriter array() {
+    return elementObjWriter.array();
+  }
+
+  public abstract void bindListener(ColumnWriterListener listener);
+  public abstract void bindListener(TupleWriterListener listener);
+  public abstract OffsetVectorWriter offsetWriter();
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("elementObjWriter");
+      elementObjWriter.dump(format);
+    format.endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractFixedWidthWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractFixedWidthWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractFixedWidthWriter.java
new file mode 100644
index 0000000..e49f92c
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractFixedWidthWriter.java
@@ -0,0 +1,258 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Base class for writers for fixed-width vectors. Handles common
+ * tasks, leaving the generated code to handle only type-specific
+ * operations.
+ */
+
+public abstract class AbstractFixedWidthWriter extends BaseScalarWriter {
+
+  public static abstract class BaseFixedWidthWriter extends AbstractFixedWidthWriter {
+
+    /**
+     * Buffer of zeros used to back-fill vector buffers with
+     * zeros.
+     */
+
+    private static final byte ZERO_BUF[] = new byte[256];
+
+    /**
+     * Determine the write index, growing, overflowing and back-filling
+     * the vector as needed.
+     * <p>
+     * This is a bit tricky. This method has side effects, by design.
+     * The current vector buffer, and buffer address, will change in
+     * this method when a vector grows or overflows. So, don't use this
+     * method in inline calls of the form<br><code>
+     * vector.getBuffer().doSomething(writeIndex());</code></br>
+     * The buffer obtained by <tt>getBuffer()</tt> can be different than
+     * the current buffer after <tt>writeIndex()</tt>.
+     *
+     * @return the index at which to write the current value
+     */
+
+    protected final int writeIndex() {
+
+      // "Fast path" for the normal case of no fills, no overflow.
+      // This is the only bounds check we want to do for the entire
+      // set operation.
+
+      // This is performance critical code; every operation counts.
+      // Please be thoughtful when changing the code.
+
+      int writeIndex = vectorIndex.vectorIndex();
+      if (lastWriteIndex + 1 < writeIndex || writeIndex >= capacity) {
+        writeIndex = prepareWrite(writeIndex);
+      }
+
+      // Track the last write location for zero-fill use next time around.
+
+      lastWriteIndex = writeIndex;
+      return writeIndex;
+    }
+
+    protected final int prepareWrite(int writeIndex) {
+
+      // Either empties must be filed or the vector is full.
+
+      writeIndex = resize(writeIndex);
+
+      // Fill empties to the write position.
+
+      fillEmpties(writeIndex);
+      return writeIndex;
+    }
+
+    /**
+     * Fill empties. This is required because the allocated memory is not
+     * zero-filled.
+     */
+
+    @Override
+    protected final void fillEmpties(final int writeIndex) {
+      final int width = width();
+      final int stride = ZERO_BUF.length / width;
+      int dest = lastWriteIndex + 1;
+      while (dest < writeIndex) {
+        int length = writeIndex - dest;
+        length = Math.min(length, stride);
+        drillBuf.unsafeCopyMemory(ZERO_BUF, 0, dest * width, length * width);
+        dest += length;
+      }
+    }
+  }
+
+  /**
+   * The largest position to which the writer has written data. Used to allow
+   * "fill-empties" (AKA "back-fill") of missing values one each value write
+   * and at the end of a batch. Note that this is the position of the last
+   * write, not the next write position. Starts at -1 (no last write).
+   */
+
+  protected int lastWriteIndex;
+
+  @Override
+  public void startWrite() {
+    setBuffer();
+    lastWriteIndex = -1;
+  }
+
+  public abstract int width();
+
+  @Override
+  protected final void setBuffer() {
+    drillBuf = vector().getBuffer();
+    capacity = drillBuf.capacity() / width();
+  }
+
+  protected final void mandatoryResize(final int writeIndex) {
+    if (writeIndex < capacity) {
+      return;
+    }
+
+    // Since some vectors start off as 0 length, set a
+    // minimum size to avoid silly thrashing on early rows.
+
+    final int size = BaseAllocator.nextPowerOfTwo(
+        Math.max((writeIndex + 1) * width(), MIN_BUFFER_SIZE));
+    realloc(size);
+  }
+
+  protected final int resize(final int writeIndex) {
+    if (writeIndex < capacity) {
+      return writeIndex;
+    }
+    final int width = width();
+
+    // Since some vectors start off as 0 length, set a
+    // minimum size to avoid silly thrashing on early rows.
+
+    final int size = BaseAllocator.nextPowerOfTwo(
+        Math.max((writeIndex + 1) * width, MIN_BUFFER_SIZE));
+
+    // Two cases: grow this vector or allocate a new one.
+
+    // Grow the vector -- or overflow if the growth would make the batch
+    // consume too much memory. The idea is that we grow vectors as they
+    // fit the available memory budget, then we fill those vectors until
+    // one of them needs more space. At that point we trigger overflow to
+    // a new set of vectors. Internal fragmentation will result, but this
+    // approach (along with proper initial vector sizing), minimizes that
+    // fragmentation.
+
+    if (size <= ValueVector.MAX_BUFFER_SIZE &&
+        canExpand(size - capacity * width)) {
+
+      // Optimized form of reAlloc() which does not zero memory, does not do
+      // bounds checks (since they were already done above). The write index
+      // and offset remain unchanged.
+
+      realloc(size);
+    } else {
+
+      // Allocate a new vector, or throw an exception if overflow is not
+      // supported. If overflow is supported, the callback will call
+      // endWrite(), which will fill empties, so no need to do that here.
+      // The call to endWrite() will also set the final writer index for the
+      // current vector. Then, bindVector() will be called to provide the new
+      // vector. The write index changes with the new vector.
+
+      overflowed();
+    }
+
+    // Call to resize may cause rollover, so reset write index
+    // afterwards.
+
+    return vectorIndex.vectorIndex();
+  }
+
+  @Override
+  public int lastWriteIndex() { return lastWriteIndex; }
+
+  @Override
+  public void skipNulls() {
+
+    // Pretend we've written up to the previous value.
+    // This will leave null values (as specified by the
+    // caller) uninitialized.
+
+    lastWriteIndex = vectorIndex.vectorIndex() - 1;
+  }
+
+  @Override
+  public void restartRow() {
+    lastWriteIndex = Math.min(lastWriteIndex, vectorIndex.vectorIndex() - 1);
+  }
+
+  @Override
+  public void preRollover() {
+    setValueCount(vectorIndex.rowStartIndex());
+  }
+
+  @Override
+  public void postRollover() {
+    int newIndex = Math.max(lastWriteIndex - vectorIndex.rowStartIndex(), -1);
+    startWrite();
+    lastWriteIndex = newIndex;
+  }
+
+  @Override
+  public void endWrite() {
+    setValueCount(vectorIndex.vectorIndex());
+  }
+
+  protected abstract void fillEmpties(int writeIndex);
+
+  public void setValueCount(int valueCount) {
+
+    // Done this way to avoid another drill buf access in value set path.
+    // Though this calls writeOffset(), which handles vector overflow,
+    // such overflow should never occur because here we are simply
+    // finalizing a position already set. However, the vector size may
+    // grow and the "missing" values may be zero-filled. Note that, in
+    // odd cases, the call to writeOffset() might cause the vector to
+    // resize (as part of filling empties), so grab the buffer AFTER
+    // the call to writeOffset().
+
+    mandatoryResize(valueCount - 1);
+    fillEmpties(valueCount);
+    vector().getBuffer().writerIndex(valueCount * width());
+
+    // Last write index is either the last value we just filled,
+    // or it is the last actual write, if this is an overflow
+    // situation.
+
+    lastWriteIndex = Math.max(lastWriteIndex, valueCount - 1);
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.extend();
+    super.dump(format);
+    format
+      .attribute("lastWriteIndex", lastWriteIndex)
+      .endObject();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
new file mode 100644
index 0000000..a8f1c64
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractObjectWriter.java
@@ -0,0 +1,72 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Abstract base class for the object layer in writers. This class acts
+ * as the glue between a column and the data type of that column, per the
+ * JSON model which Drill uses. This base class provides stubs for most
+ * methods so that type-specific subclasses can simply fill in the bits
+ * needed for that particular class.
+ */
+
+public abstract class AbstractObjectWriter implements ObjectWriter {
+
+  private ColumnMetadata schema;
+
+  public AbstractObjectWriter(ColumnMetadata schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  public ColumnMetadata schema() { return schema; }
+
+  @Override
+  public ScalarWriter scalar() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public TupleWriter tuple() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ArrayWriter array() {
+    throw new UnsupportedOperationException();
+  }
+
+  public abstract WriterEvents events();
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) { }
+
+  @Override
+  public void bindListener(TupleWriterListener listener) { }
+
+  public abstract void dump(HierarchicalFormatter format);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
new file mode 100644
index 0000000..c02e2d9
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractScalarWriter.java
@@ -0,0 +1,126 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+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 AbstractScalarWriter implements ScalarWriter, WriterEvents {
+
+  public static class ScalarObjectWriter extends AbstractObjectWriter {
+
+    private AbstractScalarWriter scalarWriter;
+
+    public ScalarObjectWriter(ColumnMetadata schema, AbstractScalarWriter scalarWriter) {
+      super(schema);
+      this.scalarWriter = scalarWriter;
+    }
+
+    @Override
+    public ObjectType type() { return ObjectType.SCALAR; }
+
+    @Override
+    public void set(Object value) { scalarWriter.setObject(value); }
+
+    @Override
+    public ScalarWriter scalar() { return scalarWriter; }
+
+    @Override
+    public WriterEvents events() { return scalarWriter; }
+
+    @Override
+    public void bindListener(ColumnWriterListener listener) {
+      scalarWriter.bindListener(listener);
+    }
+
+    @Override
+    public void dump(HierarchicalFormatter format) {
+      format
+        .startObject(this)
+        .attribute("scalarWriter");
+      scalarWriter.dump(format);
+      format.endObject();
+    }
+  }
+
+  public abstract BaseDataValueVector vector();
+
+  @Override
+  public void startWrite() { }
+
+  @Override
+  public void startRow() { }
+
+  @Override
+  public void endArrayValue() { }
+
+  @Override
+  public void saveRow() { }
+
+  @Override
+  public void setObject(Object value) {
+    if (value == null) {
+      setNull();
+    } else if (value instanceof Integer) {
+      setInt((Integer) value);
+    } else if (value instanceof Long) {
+      setLong((Long) value);
+    } else if (value instanceof String) {
+      setString((String) value);
+    } else if (value instanceof BigDecimal) {
+      setDecimal((BigDecimal) value);
+    } else if (value instanceof Period) {
+      setPeriod((Period) value);
+    } else if (value instanceof byte[]) {
+      byte[] bytes = (byte[]) value;
+      setBytes(bytes, bytes.length);
+    } else if (value instanceof Byte) {
+      setInt((Byte) value);
+    } else if (value instanceof Short) {
+      setInt((Short) value);
+    } else if (value instanceof Double) {
+      setDouble((Double) value);
+    } else if (value instanceof Float) {
+      setDouble((Float) value);
+    } else {
+      throw new IllegalArgumentException("Unsupported type " +
+                value.getClass().getSimpleName());
+    }
+  }
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attributeIdentity("vector", vector())
+      .attribute("schema", vector().getField())
+      .endObject();
+  }
+}


[09/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
index 030f95a..e1e18dc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractSingleRowSet.java
@@ -17,23 +17,13 @@
  */
 package org.apache.drill.test.rowSet;
 
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.spill.RecordBatchSizer;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
+import org.apache.drill.exec.physical.rowSet.model.single.BaseReaderBuilder;
+import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader;
-import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
-import org.apache.drill.exec.vector.complex.MapVector;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetSchema.FlattenedSchema;
-import org.apache.drill.test.rowSet.RowSetSchema.LogicalColumn;
-import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
 
 /**
  * Base class for row sets backed by a single record batch.
@@ -41,151 +31,27 @@ import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
 
 public abstract class AbstractSingleRowSet extends AbstractRowSet implements SingleRowSet {
 
-  /**
-   * Internal helper class to organize a set of value vectors for use by the
-   * row set class. Subclasses either build vectors from a schema, or map an
-   * existing vector container into the row set structure. The row set
-   * structure is based on a flattened structure; all vectors appear in
-   * a single vector array. Maps are set aside in a separate map list.
-   */
-
-  public abstract static class StructureBuilder {
-    protected final PhysicalSchema schema;
-    protected final BufferAllocator allocator;
-    protected final ValueVector[] valueVectors;
-    protected final MapVector[] mapVectors;
-    protected int vectorIndex;
-    protected int mapIndex;
-
-    public StructureBuilder(BufferAllocator allocator, RowSetSchema schema) {
-      this.allocator = allocator;
-      this.schema = schema.physical();
-      FlattenedSchema flatSchema = schema.flatAccess();
-      valueVectors = new ValueVector[flatSchema.count()];
-      if (flatSchema.mapCount() == 0) {
-        mapVectors = null;
-      } else {
-        mapVectors = new MapVector[flatSchema.mapCount()];
-      }
-    }
-  }
-
-  /**
-   * Create a set of value vectors given a schema, then map them into both
-   * the value container and the row set structure.
-   */
-
-  public static class VectorBuilder extends StructureBuilder {
-
-    public VectorBuilder(BufferAllocator allocator, RowSetSchema schema) {
-      super(allocator, schema);
-    }
-
-    public ValueVector[] buildContainer(VectorContainer container) {
-      for (int i = 0; i < schema.count(); i++) {
-        LogicalColumn colSchema = schema.column(i);
-        @SuppressWarnings("resource")
-        ValueVector v = TypeHelper.getNewVector(colSchema.field, allocator, null);
-        container.add(v);
-        if (colSchema.field.getType().getMinorType() == MinorType.MAP) {
-          MapVector mv = (MapVector) v;
-          mapVectors[mapIndex++] = mv;
-          buildMap(mv, colSchema.mapSchema);
-        } else {
-          valueVectors[vectorIndex++] = v;
-        }
-      }
-      container.buildSchema(SelectionVectorMode.NONE);
-      return valueVectors;
-    }
-
-    private void buildMap(MapVector mapVector, PhysicalSchema mapSchema) {
-      for (int i = 0; i < mapSchema.count(); i++) {
-        LogicalColumn colSchema = mapSchema.column(i);
-        MajorType type = colSchema.field.getType();
-        Class<? extends ValueVector> vectorClass = TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode());
-        @SuppressWarnings("resource")
-        ValueVector v = mapVector.addOrGet(colSchema.field.getName(), type, vectorClass);
-        if (type.getMinorType() == MinorType.MAP) {
-          MapVector mv = (MapVector) v;
-          mapVectors[mapIndex++] = mv;
-          buildMap(mv, colSchema.mapSchema);
-        } else {
-          valueVectors[vectorIndex++] = v;
-        }
-      }
-    }
-  }
-
-  /**
-   * Build a row set given an existing vector container. In this case,
-   * the vectors exist and we simply need to pull them out of the container
-   * and maps and put them into the row set arrays.
-   */
-
-  public static class VectorMapper extends StructureBuilder {
-
-    public VectorMapper(BufferAllocator allocator, RowSetSchema schema) {
-      super(allocator, schema);
-    }
+  public static class RowSetReaderBuilder extends BaseReaderBuilder {
 
-    public ValueVector[] mapContainer(VectorContainer container) {
-      for (VectorWrapper<?> w : container) {
-        @SuppressWarnings("resource")
-        ValueVector v = w.getValueVector();
-        if (v.getField().getType().getMinorType() == MinorType.MAP) {
-          MapVector mv = (MapVector) v;
-          mapVectors[mapIndex++] = mv;
-          buildMap(mv);
-        } else {
-          valueVectors[vectorIndex++] = v;
-        }
-      }
-      return valueVectors;
+    public RowSetReader buildReader(AbstractSingleRowSet rowSet, ReaderIndex rowIndex) {
+      TupleMetadata schema = rowSet.schema();
+      return new RowSetReaderImpl(schema, rowIndex,
+          buildContainerChildren(rowSet.container(),
+          new MetadataRetrieval(schema)));
     }
-
-    private void buildMap(MapVector mapVector) {
-      for (ValueVector v : mapVector) {
-        if (v.getField().getType().getMinorType() == MinorType.MAP) {
-          MapVector mv = (MapVector) v;
-          mapVectors[mapIndex++] = mv;
-          buildMap(mv);
-        } else {
-          valueVectors[vectorIndex++] = v;
-        }
-      }
-    }
-  }
-
-  /**
-   * Flattened representation of value vectors using a depth-first
-   * traversal of maps. Order of vectors here correspond to the column
-   * indexes used to access columns in a reader or writer.
-   */
-
-  protected final ValueVector[] valueVectors;
-
-  public AbstractSingleRowSet(BufferAllocator allocator, BatchSchema schema) {
-    super(allocator, schema, new VectorContainer());
-    valueVectors = new VectorBuilder(allocator, super.schema).buildContainer(container);
-  }
-
-  public AbstractSingleRowSet(BufferAllocator allocator, VectorContainer container) {
-    super(allocator, container.getSchema(), container);
-    valueVectors = new VectorMapper(allocator, super.schema).mapContainer(container);
   }
 
   public AbstractSingleRowSet(AbstractSingleRowSet rowSet) {
-    super(rowSet.allocator, rowSet.schema.batch(), rowSet.container);
-    valueVectors = rowSet.valueVectors;
+    super(rowSet.container, rowSet.schema);
   }
 
-  @Override
-  public ValueVector[] vectors() { return valueVectors; }
+  public AbstractSingleRowSet(VectorContainer container, TupleMetadata schema) {
+    super(container, schema);
+  }
 
   @Override
   public long size() {
-    RecordBatchSizer sizer = new RecordBatchSizer(container);
+    RecordBatchSizer sizer = new RecordBatchSizer(container());
     return sizer.actualSize();
   }
 
@@ -197,21 +63,7 @@ public abstract class AbstractSingleRowSet extends AbstractRowSet implements Sin
    * (non-map) vectors.
    */
 
-  protected RowSetReader buildReader(RowSetIndex rowIndex) {
-    FlattenedSchema accessSchema = schema().flatAccess();
-    ValueVector[] valueVectors = vectors();
-    AbstractColumnReader[] readers = new AbstractColumnReader[valueVectors.length];
-    for (int i = 0; i < readers.length; i++) {
-      MinorType type = accessSchema.column(i).getType().getMinorType();
-      if (type == MinorType.MAP) {
-        readers[i] = null; // buildMapAccessor(i);
-      } else if (type == MinorType.LIST) {
-        readers[i] = null; // buildListAccessor(i);
-      } else {
-        readers[i] = ColumnAccessorFactory.newReader(valueVectors[i].getField().getType());
-        readers[i].bind(rowIndex, valueVectors[i]);
-      }
-    }
-    return new RowSetReaderImpl(accessSchema, rowIndex, readers);
+  protected RowSetReader buildReader(ReaderIndex rowIndex) {
+    return new RowSetReaderBuilder().buildReader(this, rowIndex);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
index 29a1702..5972f05 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/DirectRowSet.java
@@ -18,19 +18,21 @@
 package org.apache.drill.test.rowSet;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
+import org.apache.drill.exec.physical.rowSet.model.SchemaInference;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
+import org.apache.drill.exec.physical.rowSet.model.single.BaseWriterBuilder;
+import org.apache.drill.exec.physical.rowSet.model.single.BuildVectorsFromMetadata;
+import org.apache.drill.exec.physical.rowSet.model.single.VectorAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorAccessible;
-import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.TupleAccessor.TupleSchema;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnWriter;
-import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
-import org.apache.drill.exec.vector.accessor.impl.TupleWriterImpl;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSetWriterImpl.WriterIndexImpl;
 
 /**
  * Implementation of a single row set with no indirection (selection)
@@ -46,118 +48,54 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
    * the first. (This is the JDBC RecordSet convention.)
    */
 
-  private static class DirectRowIndex extends BoundedRowIndex {
+  private static class DirectRowIndex extends ReaderIndex {
 
     public DirectRowIndex(int rowCount) {
       super(rowCount);
     }
 
     @Override
-    public int index() { return rowIndex; }
+    public int vectorIndex() { return rowIndex; }
 
     @Override
-    public int batch() { return 0; }
+    public int batchIndex() { return 0; }
   }
 
-  /**
-   * Writer index that points to each row in the row set. The index starts at
-   * the 0th row and advances one row on each increment. This allows writers to
-   * start positioned at the first row. Writes happen in the current row.
-   * Calling <tt>next()</tt> advances to the next position, effectively saving
-   * the current row. The most recent row can be abandoned easily simply by not
-   * calling <tt>next()</tt>. This means that the number of completed rows is
-   * the same as the row index.
-   */
-
-  private static class ExtendableRowIndex extends RowSetIndex {
-
-    private final int maxSize;
-
-    public ExtendableRowIndex(int maxSize) {
-      this.maxSize = maxSize;
-      rowIndex = 0;
-    }
+  public static class RowSetWriterBuilder extends BaseWriterBuilder {
 
-    @Override
-    public int index() { return rowIndex; }
-
-    @Override
-    public boolean next() {
-      if (++rowIndex <= maxSize ) {
-        return true;
-      } else {
-        rowIndex--;
-        return false;
-      }
+    public RowSetWriter buildWriter(DirectRowSet rowSet) {
+      WriterIndexImpl index = new WriterIndexImpl();
+      TupleMetadata schema = rowSet.schema();
+      RowSetWriterImpl writer = new RowSetWriterImpl(rowSet, schema, index,
+          buildContainerChildren(rowSet.container(),
+          new MetadataRetrieval(schema)));
+      return writer;
     }
-
-    @Override
-    public int size() { return rowIndex; }
-
-    @Override
-    public boolean valid() { return rowIndex < maxSize; }
-
-    @Override
-    public int batch() { return 0; }
   }
 
-  /**
-   * Implementation of a row set writer. Only available for newly-created,
-   * empty, direct, single row sets. Rewriting is not allowed, nor is writing
-   * to a hyper row set.
-   */
-
-  public class RowSetWriterImpl extends TupleWriterImpl implements RowSetWriter {
-
-    private final ExtendableRowIndex index;
-    private final ExtendableRowSet rowSet;
-
-    protected RowSetWriterImpl(ExtendableRowSet rowSet, TupleSchema schema, ExtendableRowIndex index, AbstractColumnWriter[] writers) {
-      super(schema, writers);
-      this.rowSet = rowSet;
-      this.index = index;
-      start();
-    }
-
-    @Override
-    public void setRow(Object...values) {
-      if (! index.valid()) {
-        throw new IndexOutOfBoundsException("Write past end of row set");
-      }
-      for (int i = 0; i < values.length;  i++) {
-        set(i, values[i]);
-      }
-      save();
-    }
-
-    @Override
-    public boolean valid() { return index.valid(); }
-
-    @Override
-    public int index() { return index.position(); }
+  private DirectRowSet(VectorContainer container, TupleMetadata schema) {
+    super(container, schema);
+  }
 
-    @Override
-    public void save() {
-      index.next();
-      start();
-    }
+  public DirectRowSet(AbstractSingleRowSet from) {
+    super(from);
+  }
 
-    @Override
-    public void done() {
-      rowSet.setRowCount(index.size());
-    }
+  public static DirectRowSet fromSchema(BufferAllocator allocator, BatchSchema schema) {
+    return fromSchema(allocator, TupleSchema.fromFields(schema));
   }
 
-  public DirectRowSet(BufferAllocator allocator, BatchSchema schema) {
-    super(allocator, schema);
+  public static DirectRowSet fromSchema(BufferAllocator allocator, TupleMetadata schema) {
+    BuildVectorsFromMetadata builder = new BuildVectorsFromMetadata(allocator);
+    return new DirectRowSet(builder.build(schema), schema);
   }
 
-  public DirectRowSet(BufferAllocator allocator, VectorContainer container) {
-    super(allocator, container);
+  public static DirectRowSet fromContainer(VectorContainer container) {
+    return new DirectRowSet(container, new SchemaInference().infer(container));
   }
 
-  public DirectRowSet(BufferAllocator allocator, VectorAccessible va) {
-    super(allocator, toContainer(va, allocator));
+  public static DirectRowSet fromVectorAccessible(BufferAllocator allocator, VectorAccessible va) {
+    return fromContainer(toContainer(va, allocator));
   }
 
   private static VectorContainer toContainer(VectorAccessible va, BufferAllocator allocator) {
@@ -168,16 +106,8 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
   }
 
   @Override
-  public void allocate(int recordCount) {
-    for (final ValueVector v : valueVectors) {
-      AllocationHelper.allocate(v, recordCount, 50, 10);
-    }
-  }
-
-  @Override
-  public void setRowCount(int rowCount) {
-    container.setRecordCount(rowCount);
-    VectorAccessibleUtilities.setValueCount(container, rowCount);
+  public void allocate(int rowCount) {
+    new VectorAllocator(container()).allocate(rowCount, schema());
   }
 
   @Override
@@ -187,29 +117,11 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
 
   @Override
   public RowSetWriter writer(int initialRowCount) {
-    if (container.hasRecordCount()) {
+    if (container().hasRecordCount()) {
       throw new IllegalStateException("Row set already contains data");
     }
     allocate(initialRowCount);
-    return buildWriter(new ExtendableRowIndex(Character.MAX_VALUE));
-  }
-
-  /**
-   * Build writer objects for each column based on the column type.
-   *
-   * @param rowIndex the index which points to each row
-   * @return an array of writers
-   */
-
-  protected RowSetWriter buildWriter(ExtendableRowIndex rowIndex) {
-    ValueVector[] valueVectors = vectors();
-    AbstractColumnWriter[] writers = new AbstractColumnWriter[valueVectors.length];
-    for (int i = 0; i < writers.length; i++) {
-      writers[i] = ColumnAccessorFactory.newWriter(valueVectors[i].getField().getType());
-      writers[i].bind(rowIndex, valueVectors[i]);
-    }
-    TupleSchema accessSchema = schema().hierarchicalAccess();
-    return new RowSetWriterImpl(this, accessSchema, rowIndex, writers);
+    return new RowSetWriterBuilder().buildWriter(this);
   }
 
   @Override
@@ -233,9 +145,4 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
 
   @Override
   public SelectionVector2 getSv2() { return null; }
-
-  @Override
-  public RowSet merge(RowSet other) {
-    return new DirectRowSet(allocator, container().merge(other.container()));
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
index afc2e6e..8a3db9f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
@@ -17,27 +17,14 @@
  */
 package org.apache.drill.test.rowSet;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.rowSet.model.MetadataProvider.MetadataRetrieval;
+import org.apache.drill.exec.physical.rowSet.model.SchemaInference;
+import org.apache.drill.exec.physical.rowSet.model.hyper.BaseReaderBuilder;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.HyperVectorWrapper;
-import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.AccessorUtilities;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader.VectorAccessor;
-import org.apache.drill.exec.vector.accessor.impl.ColumnAccessorFactory;
-import org.apache.drill.exec.vector.complex.AbstractMapVector;
 import org.apache.drill.test.rowSet.RowSet.HyperRowSet;
-import org.apache.drill.test.rowSet.RowSetSchema.FlattenedSchema;
-import org.apache.drill.test.rowSet.RowSetSchema.LogicalColumn;
-import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
 
 /**
  * Implements a row set wrapper around a collection of "hyper vectors."
@@ -52,176 +39,14 @@ import org.apache.drill.test.rowSet.RowSetSchema.PhysicalSchema;
 
 public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
 
-  /**
-   * Read-only row index into the hyper row set with batch and index
-   * values mapping via an SV4.
-   */
-
-  public static class HyperRowIndex extends BoundedRowIndex {
-
-    private final SelectionVector4 sv4;
-
-    public HyperRowIndex(SelectionVector4 sv4) {
-      super(sv4.getCount());
-      this.sv4 = sv4;
-    }
-
-    @Override
-    public int index() {
-      return AccessorUtilities.sv4Index(sv4.get(rowIndex));
-    }
-
-    @Override
-    public int batch( ) {
-      return AccessorUtilities.sv4Batch(sv4.get(rowIndex));
-    }
-  }
-
-  /**
-   * Vector accessor used by the column accessors to obtain the vector for
-   * each column value. That is, position 0 might be batch 4, index 3,
-   * while position 1 might be batch 1, index 7, and so on.
-   */
-
-  public static class HyperVectorAccessor implements VectorAccessor {
+  public static class RowSetReaderBuilder extends BaseReaderBuilder {
 
-    private final HyperRowIndex rowIndex;
-    private final ValueVector[] vectors;
-
-    public HyperVectorAccessor(HyperVectorWrapper<ValueVector> hvw, HyperRowIndex rowIndex) {
-      this.rowIndex = rowIndex;
-      vectors = hvw.getValueVectors();
-    }
-
-    @Override
-    public ValueVector vector() {
-      return vectors[rowIndex.batch()];
-    }
-  }
-
-  /**
-   * Build a hyper row set by restructuring a hyper vector bundle into a uniform
-   * shape. Consider this schema: <pre><code>
-   * { a: 10, b: { c: 20, d: { e: 30 } } }</code></pre>
-   * <p>
-   * The hyper container, with two batches, has this structure:
-   * <table border="1">
-   * <tr><th>Batch</th><th>a</th><th>b</th></tr>
-   * <tr><td>0</td><td>Int vector</td><td>Map Vector(Int vector, Map Vector(Int vector))</td></th>
-   * <tr><td>1</td><td>Int vector</td><td>Map Vector(Int vector, Map Vector(Int vector))</td></th>
-   * </table>
-   * <p>
-   * The above table shows that top-level scalar vectors (such as the Int Vector for column
-   * a) appear "end-to-end" as a hyper-vector. Maps also appear end-to-end. But, the
-   * contents of the map (column c) do not appear end-to-end. Instead, they appear as
-   * contents in the map vector. To get to c, one indexes into the map vector, steps inside
-   * the map to find c and indexes to the right row.
-   * <p>
-   * Similarly, the maps for d do not appear end-to-end, one must step to the right batch
-   * in b, then step to d.
-   * <p>
-   * Finally, to get to e, one must step
-   * into the hyper vector for b, then steps to the proper batch, steps to d, step to e
-   * and finally step to the row within e. This is a very complex, costly indexing scheme
-   * that differs depending on map nesting depth.
-   * <p>
-   * To simplify access, this class restructures the maps to flatten the scalar vectors
-   * into end-to-end hyper vectors. For example, for the above:
-   * <p>
-   * <table border="1">
-   * <tr><th>Batch</th><th>a</th><th>c</th><th>d</th></tr>
-   * <tr><td>0</td><td>Int vector</td><td>Int vector</td><td>Int vector</td></th>
-   * <tr><td>1</td><td>Int vector</td><td>Int vector</td><td>Int vector</td></th>
-   * </table>
-   *
-   * The maps are still available as hyper vectors, but separated into map fields.
-   * (Scalar access no longer needs to access the maps.) The result is a uniform
-   * addressing scheme for both top-level and nested vectors.
-   */
-
-  public static class HyperVectorBuilder {
-
-    protected final HyperVectorWrapper<?> valueVectors[];
-    protected final HyperVectorWrapper<AbstractMapVector> mapVectors[];
-    private final List<ValueVector> nestedScalars[];
-    private int vectorIndex;
-    private int mapIndex;
-    private final PhysicalSchema physicalSchema;
-
-    @SuppressWarnings("unchecked")
-    public HyperVectorBuilder(RowSetSchema schema) {
-      physicalSchema = schema.physical();
-      FlattenedSchema flatSchema = schema.flatAccess();
-      valueVectors = new HyperVectorWrapper<?>[schema.hierarchicalAccess().count()];
-      if (flatSchema.mapCount() == 0) {
-        mapVectors = null;
-        nestedScalars = null;
-      } else {
-        mapVectors = (HyperVectorWrapper<AbstractMapVector>[])
-            new HyperVectorWrapper<?>[flatSchema.mapCount()];
-        nestedScalars = new ArrayList[flatSchema.count()];
-      }
-    }
-
-    @SuppressWarnings("unchecked")
-    public HyperVectorWrapper<ValueVector>[] mapContainer(VectorContainer container) {
-      int i = 0;
-      for (VectorWrapper<?> w : container) {
-        HyperVectorWrapper<?> hvw = (HyperVectorWrapper<?>) w;
-        if (w.getField().getType().getMinorType() == MinorType.MAP) {
-          HyperVectorWrapper<AbstractMapVector> mw = (HyperVectorWrapper<AbstractMapVector>) hvw;
-          mapVectors[mapIndex++] = mw;
-          buildHyperMap(physicalSchema.column(i).mapSchema(), mw);
-        } else {
-          valueVectors[vectorIndex++] = hvw;
-        }
-        i++;
-      }
-      if (nestedScalars != null) {
-        buildNestedHyperVectors();
-      }
-      return (HyperVectorWrapper<ValueVector>[]) valueVectors;
-    }
-
-    private void buildHyperMap(PhysicalSchema mapSchema, HyperVectorWrapper<AbstractMapVector> mapWrapper) {
-      createHyperVectors(mapSchema);
-      for (AbstractMapVector mapVector : mapWrapper.getValueVectors()) {
-        buildMap(mapSchema, mapVector);
-      }
-    }
-
-    private void buildMap(PhysicalSchema mapSchema, AbstractMapVector mapVector) {
-      for (ValueVector v : mapVector) {
-        LogicalColumn col = mapSchema.column(v.getField().getName());
-        if (col.isMap()) {
-          buildMap(col.mapSchema, (AbstractMapVector) v);
-        } else {
-          nestedScalars[col.accessIndex()].add(v);
-        }
-      }
-    }
-
-    private void createHyperVectors(PhysicalSchema mapSchema) {
-      for (int i = 0; i < mapSchema.count(); i++) {
-        LogicalColumn col = mapSchema.column(i);
-        if (col.isMap()) {
-          createHyperVectors(col.mapSchema);
-        } else {
-          nestedScalars[col.accessIndex()] = new ArrayList<ValueVector>();
-        }
-      }
-    }
-
-    private void buildNestedHyperVectors() {
-      for (int i = 0;  i < nestedScalars.length; i++) {
-        if (nestedScalars[i] == null) {
-          continue;
-        }
-        ValueVector vectors[] = new ValueVector[nestedScalars[i].size()];
-        nestedScalars[i].toArray(vectors);
-        assert valueVectors[i] == null;
-        valueVectors[i] = new HyperVectorWrapper<ValueVector>(vectors[0].getField(), vectors, false);
-      }
+    public RowSetReader buildReader(HyperRowSet rowSet, SelectionVector4 sv4) {
+      TupleMetadata schema = rowSet.schema();
+      HyperRowIndex rowIndex = new HyperRowIndex(sv4);
+      return new RowSetReaderImpl(schema, rowIndex,
+          buildContainerChildren(rowSet.container(),
+              new MetadataRetrieval(schema)));
     }
   }
 
@@ -231,18 +56,9 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
 
   private final SelectionVector4 sv4;
 
-  /**
-   * Collection of hyper vectors in flattened order: a left-to-right,
-   * depth first ordering of vectors in maps. Order here corresponds to
-   * the order used for column indexes in the row set reader.
-   */
-
-  private final HyperVectorWrapper<ValueVector> hvw[];
-
-  public HyperRowSetImpl(BufferAllocator allocator, VectorContainer container, SelectionVector4 sv4) {
-    super(allocator, container.getSchema(), container);
+  public HyperRowSetImpl(VectorContainer container, SelectionVector4 sv4) {
+    super(container, new SchemaInference().infer(container));
     this.sv4 = sv4;
-    hvw = new HyperVectorBuilder(schema).mapContainer(container);
   }
 
   @Override
@@ -252,33 +68,8 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
   public boolean isWritable() { return false; }
 
   @Override
-  public RowSetWriter writer() {
-    throw new UnsupportedOperationException("Cannot write to a hyper vector");
-  }
-
-  @Override
   public RowSetReader reader() {
-    return buildReader(new HyperRowIndex(sv4));
-  }
-
-  /**
-   * Internal method to build the set of column readers needed for
-   * this row set. Used when building a row set reader.
-   * @param rowIndex object that points to the current row
-   * @return an array of column readers: in the same order as the
-   * (non-map) vectors.
-   */
-
-  protected RowSetReader buildReader(HyperRowIndex rowIndex) {
-    FlattenedSchema accessSchema = schema().flatAccess();
-    AbstractColumnReader readers[] = new AbstractColumnReader[accessSchema.count()];
-    for (int i = 0; i < readers.length; i++) {
-      MaterializedField field = accessSchema.column(i);
-      readers[i] = ColumnAccessorFactory.newReader(field.getType());
-      HyperVectorWrapper<ValueVector> hvw = getHyperVector(i);
-      readers[i].bind(rowIndex, field, new HyperVectorAccessor(hvw, rowIndex));
-    }
-    return new RowSetReaderImpl(accessSchema, rowIndex, readers);
+    return new RowSetReaderBuilder().buildReader(this, sv4);
   }
 
   @Override
@@ -288,13 +79,5 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
   public SelectionVector4 getSv4() { return sv4; }
 
   @Override
-  public HyperVectorWrapper<ValueVector> getHyperVector(int i) { return hvw[i]; }
-
-  @Override
   public int rowCount() { return sv4.getCount(); }
-
-  @Override
-  public RowSet merge(RowSet other) {
-    return new HyperRowSetImpl(allocator, container().merge(other.container()), sv4);
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/IndirectRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/IndirectRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/IndirectRowSet.java
index 1914705..e729bba 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/IndirectRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/IndirectRowSet.java
@@ -20,6 +20,8 @@ package org.apache.drill.test.rowSet;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.spill.RecordBatchSizer;
+import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
+import org.apache.drill.exec.physical.rowSet.model.SchemaInference;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;
@@ -33,14 +35,14 @@ public class IndirectRowSet extends AbstractSingleRowSet {
 
   /**
    * Reader index that points to each row indirectly through the
-   * selection vector. The {@link #index()} method points to the
+   * selection vector. The {@link #vectorIndex()} method points to the
    * actual data row, while the {@link #position()} method gives
    * the position relative to the indirection vector. That is,
    * the position increases monotonically, but the index jumps
    * around as specified by the indirection vector.
    */
 
-  private static class IndirectRowIndex extends BoundedRowIndex {
+  private static class IndirectRowIndex extends ReaderIndex {
 
     private final SelectionVector2 sv2;
 
@@ -50,21 +52,25 @@ public class IndirectRowSet extends AbstractSingleRowSet {
     }
 
     @Override
-    public int index() { return sv2.getIndex(rowIndex); }
+    public int vectorIndex() { return sv2.getIndex(rowIndex); }
 
     @Override
-    public int batch() { return 0; }
+    public int batchIndex() { return 0; }
   }
 
   private final SelectionVector2 sv2;
 
-  public IndirectRowSet(BufferAllocator allocator, VectorContainer container) {
-    this(allocator, container, makeSv2(allocator, container));
+  private IndirectRowSet(VectorContainer container, SelectionVector2 sv2) {
+    super(container, new SchemaInference().infer(container));
+    this.sv2 = sv2;
   }
 
-  public IndirectRowSet(BufferAllocator allocator, VectorContainer container, SelectionVector2 sv2) {
-    super(allocator, container);
-    this.sv2 = sv2;
+  public static IndirectRowSet fromContainer(VectorContainer container) {
+    return new IndirectRowSet(container, makeSv2(container.getAllocator(), container));
+  }
+
+  public static IndirectRowSet fromSv2(VectorContainer container, SelectionVector2 sv2) {
+    return new IndirectRowSet(container, sv2);
   }
 
   private static SelectionVector2 makeSv2(BufferAllocator allocator, VectorContainer container) {
@@ -83,7 +89,7 @@ public class IndirectRowSet extends AbstractSingleRowSet {
 
   public IndirectRowSet(DirectRowSet directRowSet) {
     super(directRowSet);
-    sv2 = makeSv2(allocator, container);
+    sv2 = makeSv2(allocator(), container());
   }
 
   @Override
@@ -96,11 +102,6 @@ public class IndirectRowSet extends AbstractSingleRowSet {
   }
 
   @Override
-  public RowSetWriter writer() {
-    throw new UnsupportedOperationException("Cannot write to an existing row set");
-  }
-
-  @Override
   public RowSetReader reader() {
     return buildReader(new IndirectRowIndex(getSv2()));
   }
@@ -119,12 +120,7 @@ public class IndirectRowSet extends AbstractSingleRowSet {
 
   @Override
   public long size() {
-    RecordBatchSizer sizer = new RecordBatchSizer(container, sv2);
+    RecordBatchSizer sizer = new RecordBatchSizer(container(), sv2);
     return sizer.actualSize();
   }
-
-  @Override
-  public RowSet merge(RowSet other) {
-    return new IndirectRowSet(allocator, container().merge(other.container()), sv2);
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
index 474508c..f2435de 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSet.java
@@ -20,25 +20,24 @@ package org.apache.drill.test.rowSet;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.HyperVectorWrapper;
+import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
-import org.apache.drill.exec.vector.accessor.ColumnWriter;
-import org.apache.drill.exec.vector.accessor.TupleReader;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.parquet.column.ColumnWriter;
 
 /**
  * A row set is a collection of rows stored as value vectors. Elsewhere in
  * Drill we call this a "record batch", but that term has been overloaded to
- * mean the runtime implementation of an operator...
+ * mean the runtime implementation of an operator.
  * <p>
  * A row set encapsulates a set of vectors and provides access to Drill's
  * various "views" of vectors: {@link VectorContainer},
- * {@link VectorAccessible}, etc.
+ * {@link VectorAccessible}, etc. The row set wraps a {#link TupleModel}
+ * which holds the vectors and column metadata. This form is optimized
+ * for easy use in testing; use other implementations for production code.
  * <p>
  * A row set is defined by a {@link RowSetSchema}. For testing purposes, a row
  * set has a fixed schema; we don't allow changing the set of vectors
@@ -52,7 +51,7 @@ import org.apache.drill.exec.vector.accessor.TupleWriter;
  * Drill provides a large number of vector (data) types. Each requires a
  * type-specific way to set data. The row set writer uses a {@link ColumnWriter}
  * to set each value in a way unique to the specific data type. Similarly, the
- * row set reader provides a {@link ColumnReader} interface. In both cases,
+ * row set reader provides a {@link ScalarReader} interface. In both cases,
  * columns can be accessed by index number (as defined in the schema) or
  * by name.
  * <p>
@@ -78,54 +77,6 @@ import org.apache.drill.exec.vector.accessor.TupleWriter;
 
 public interface RowSet {
 
-  /**
-   * Interface for writing values to a row set. Only available
-   * for newly-created, single, direct row sets. Eventually, if
-   * we want to allow updating a row set, we have to create a
-   * new row set with the updated columns, then merge the new
-   * and old row sets to create a new immutable row set.
-   */
-  interface RowSetWriter extends TupleWriter {
-    void setRow(Object...values);
-    boolean valid();
-    int index();
-    void save();
-    void done();
-  }
-
-  /**
-   * Reader for all types of row sets.
-   */
-  interface RowSetReader extends TupleReader {
-
-    /**
-     * Total number of rows in the row set.
-     * @return total number of rows
-     */
-    int size();
-
-    boolean next();
-    int index();
-    void set(int index);
-
-    /**
-     * Batch index: 0 for a single batch, batch for the current
-     * row is a hyper-batch.
-     * @return index of the batch for the current row
-     */
-    int batchIndex();
-
-    /**
-     * The index of the underlying row which may be indexed by an
-     * Sv2 or Sv4.
-     *
-     * @return
-     */
-
-    int rowIndex();
-    boolean valid();
-  }
-
   boolean isExtendable();
 
   boolean isWritable();
@@ -136,13 +87,11 @@ public interface RowSet {
 
   int rowCount();
 
-  RowSetWriter writer();
-
   RowSetReader reader();
 
   void clear();
 
-  RowSetSchema schema();
+  TupleMetadata schema();
 
   BufferAllocator allocator();
 
@@ -157,17 +106,16 @@ public interface RowSet {
    *
    * @return memory size in bytes
    */
-  long size();
 
-  RowSet merge(RowSet other);
+  long size();
 
   BatchSchema batchSchema();
 
   /**
    * Row set that manages a single batch of rows.
    */
-  interface SingleRowSet extends RowSet {
-    ValueVector[] vectors();
+
+  public interface SingleRowSet extends RowSet {
     SingleRowSet toIndirect();
     SelectionVector2 getSv2();
   }
@@ -177,9 +125,10 @@ public interface RowSet {
    * Once writing is complete, the row set becomes an
    * immutable direct row set.
    */
+
   interface ExtendableRowSet extends SingleRowSet {
     void allocate(int recordCount);
-    void setRowCount(int rowCount);
+    RowSetWriter writer();
     RowSetWriter writer(int initialRowCount);
   }
 
@@ -187,8 +136,8 @@ public interface RowSet {
    * Row set comprised of multiple single row sets, along with
    * an indirection vector (SV4).
    */
+
   interface HyperRowSet extends RowSet {
     SelectionVector4 getSv4();
-    HyperVectorWrapper<ValueVector> getHyperVector(int i);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
index 6f9a8d9..7b1554c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBuilder.java
@@ -19,7 +19,10 @@ package org.apache.drill.test.rowSet;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 
 /**
@@ -40,14 +43,20 @@ public final class RowSetBuilder {
   private boolean withSv2;
 
   public RowSetBuilder(BufferAllocator allocator, BatchSchema schema) {
+    this(allocator, TupleSchema.fromFields(schema), 10);
+  }
+
+  public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema) {
     this(allocator, schema, 10);
   }
 
-  public RowSetBuilder(BufferAllocator allocator, BatchSchema schema, int capacity) {
-    rowSet = new DirectRowSet(allocator, schema);
+  public RowSetBuilder(BufferAllocator allocator, TupleMetadata schema, int capacity) {
+    rowSet = DirectRowSet.fromSchema(allocator, schema);
     writer = rowSet.writer(capacity);
   }
 
+  public TupleWriter writer() { return writer; }
+
   /**
    * Add a new row using column values passed as variable-length arguments. Expects
    * map values to be flattened. a schema of (a:int, b:map(c:varchar)) would be>
@@ -56,17 +65,18 @@ public final class RowSetBuilder {
    * <tt>add(10, new int[] {100, 200});</tt><br>
    * @param values column values in column index order
    * @return this builder
-   * @see {@link #addSingleCol(Object)} to create a row of a single column when
-   * the value to <tt>add()</tt> is ambiguous
+   * @throws IllegalStateException if the batch, or any vector in the batch,
+   * becomes full. This method is designed to be used in tests where we will
+   * seldom create a full vector of data.
    */
 
-  public RowSetBuilder add(Object...values) {
+  public RowSetBuilder addRow(Object...values) {
     writer.setRow(values);
     return this;
   }
 
   /**
-   * The {@link #add(Object...)} method uses Java variable-length arguments to
+   * The {@link #addRow(Object...)} method uses Java variable-length arguments to
    * pass a row of values. But, when the row consists of a single array, Java
    * gets confused: is that an array for variable-arguments or is it the value
    * of the first argument? This method clearly states that the single value
@@ -93,7 +103,7 @@ public final class RowSetBuilder {
    */
 
   public RowSetBuilder addSingleCol(Object value) {
-    return add(new Object[] { value });
+    return addRow(new Object[] { value });
   }
 
   /**
@@ -110,10 +120,10 @@ public final class RowSetBuilder {
   }
 
   public SingleRowSet build() {
-    writer.done();
+    SingleRowSet result = writer.done();
     if (withSv2) {
-      return rowSet.toIndirect();
+      return result.toIndirect();
     }
-    return rowSet;
+    return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
index 6e72923..1cae64f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
@@ -21,8 +21,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.drill.exec.vector.accessor.ArrayReader;
-import org.apache.drill.exec.vector.accessor.ColumnReader;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
+import org.apache.drill.exec.vector.accessor.ObjectReader;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.TupleReader;
 import org.bouncycastle.util.Arrays;
 
 import java.util.Comparator;
@@ -31,19 +33,48 @@ import java.util.Comparator;
  * For testing, compare the contents of two row sets (record batches)
  * to verify that they are identical. Supports masks to exclude certain
  * columns from comparison.
+ * <p>
+ * Drill rows are analogous to JSON documents: they can have scalars,
+ * arrays and maps, with maps and lists holding maps, arrays and scalars.
+ * This class walks the row structure tree to compare each structure
+ * of two row sets checking counts, types and values to ensure that the
+ * "actual" result set (result of a test) matches the "expected" result
+ * set.
+ * <p>
+ * This class acts as an example of how to use the suite of reader
+ * abstractions.
  */
 
 public class RowSetComparison {
 
+  /**
+   * Row set with the expected outcome of a test. This is the "golden"
+   * copy defined in the test itself.
+   */
   private RowSet expected;
+  /**
+   * Some tests wish to ignore certain (top-level) columns. If a
+   * mask is provided, then only those columns with a <tt>true</tt>
+   * will be verified.
+   */
   private boolean mask[];
+  /**
+   * Floats and doubles do not compare exactly. This delta is used
+   * by JUnit for such comparisons.
+   */
   private double delta = 0.001;
+  /**
+   * Tests can skip the first n rows.
+   */
   private int offset;
   private int span = -1;
 
   public RowSetComparison(RowSet expected) {
     this.expected = expected;
-    mask = new boolean[expected.schema().hierarchicalAccess().count()];
+
+    // TODO: The mask only works at the top level presently
+
+    mask = new boolean[expected.schema().size()];
     for (int i = 0; i < mask.length; i++) {
       mask[i] = true;
     }
@@ -134,7 +165,8 @@ public class RowSetComparison {
     for (int i = 0; i < testLength; i++) {
       er.next();
       ar.next();
-      verifyRow(er, ar);
+      String label = Integer.toString(er.index() + 1);
+      verifyRow(label, er, ar);
     }
   }
 
@@ -167,22 +199,50 @@ public class RowSetComparison {
     }
   }
 
-  private void verifyRow(RowSetReader er, RowSetReader ar) {
+  private void verifyRow(String label, TupleReader er, TupleReader ar) {
+    String prefix = label + ":";
     for (int i = 0; i < mask.length; i++) {
       if (! mask[i]) {
         continue;
       }
-      ColumnReader ec = er.column(i);
-      ColumnReader ac = ar.column(i);
-      String label = (er.index() + 1) + ":" + i;
-      assertEquals(label, ec.valueType(), ac.valueType());
-      if (ec.isNull()) {
-        assertTrue(label + " - column not null", ac.isNull());
-        continue;
-      }
-      if (! ec.isNull()) {
-        assertTrue(label + " - column is null", ! ac.isNull());
-      }
+      verifyColumn(prefix + i, er.column(i), ar.column(i));
+    }
+  }
+
+  private void verifyColumn(String label, ObjectReader ec, ObjectReader ac) {
+    assertEquals(label, ec.type(), ac.type());
+    switch (ec.type()) {
+    case ARRAY:
+      verifyArray(label, ec.array(), ac.array());
+      break;
+    case SCALAR:
+      verifyScalar(label, ec.scalar(), ac.scalar());
+      break;
+    case TUPLE:
+      verifyTuple(label, ec.tuple(), ac.tuple());
+      break;
+    default:
+      throw new IllegalStateException( "Unexpected type: " + ec.type());
+    }
+  }
+
+  private void verifyTuple(String label, TupleReader er, TupleReader ar) {
+    assertEquals(label + " - tuple count", er.columnCount(), ar.columnCount());
+    String prefix = label + ":";
+    for (int i = 0; i < er.columnCount(); i++) {
+      verifyColumn(prefix + i, er.column(i), ar.column(i));
+    }
+  }
+
+  private void verifyScalar(String label, ScalarReader ec, ScalarReader ac) {
+    assertEquals(label + " - value type", ec.valueType(), ac.valueType());
+    if (ec.isNull()) {
+      assertTrue(label + " - column not null", ac.isNull());
+      return;
+    }
+    if (! ec.isNull()) {
+      assertTrue(label + " - column is null", ! ac.isNull());
+    }
     switch (ec.valueType()) {
     case BYTES: {
         byte expected[] = ac.getBytes();
@@ -209,24 +269,42 @@ public class RowSetComparison {
      case PERIOD:
        assertEquals(label, ec.getPeriod(), ac.getPeriod());
        break;
-     case ARRAY:
-       verifyArray(label, ec.array(), ac.array());
-       break;
      default:
         throw new IllegalStateException( "Unexpected type: " + ec.valueType());
-      }
     }
   }
 
-  private void verifyArray(String colLabel, ArrayReader ea,
+  private void verifyArray(String label, ArrayReader ea,
       ArrayReader aa) {
+    assertEquals(label, ea.entryType(), aa.entryType());
+    assertEquals(label, ea.size(), aa.size());
+    switch (ea.entryType()) {
+    case ARRAY:
+      throw new UnsupportedOperationException();
+    case SCALAR:
+      verifyScalarArray(label, ea.elements(), aa.elements());
+      break;
+    case TUPLE:
+      verifyTupleArray(label, ea, aa);
+      break;
+    default:
+      throw new IllegalStateException( "Unexpected type: " + ea.entryType());
+    }
+  }
+
+  private void verifyTupleArray(String label, ArrayReader ea, ArrayReader aa) {
+    for (int i = 0; i < ea.size(); i++) {
+      verifyTuple(label + "[" + i + "]", ea.tuple(i), aa.tuple(i));
+    }
+  }
+
+  private void verifyScalarArray(String colLabel, ScalarElementReader ea,
+      ScalarElementReader aa) {
     assertEquals(colLabel, ea.valueType(), aa.valueType());
     assertEquals(colLabel, ea.size(), aa.size());
     for (int i = 0; i < ea.size(); i++) {
       String label = colLabel + "[" + i + "]";
       switch (ea.valueType()) {
-      case ARRAY:
-        throw new IllegalStateException("Arrays of arrays not supported yet");
       case BYTES: {
         byte expected[] = ea.getBytes(i);
         byte actual[] = aa.getBytes(i);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
index 42a7e63..e730987 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetPrinter.java
@@ -20,8 +20,8 @@ package org.apache.drill.test.rowSet;
 import java.io.PrintStream;
 
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.vector.accessor.TupleAccessor.TupleSchema;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.TupleMetadata;
 
 /**
  * Print a row set in CSV-like format. Primarily for debugging.
@@ -41,21 +41,21 @@ public class RowSetPrinter {
   public void print(PrintStream out) {
     SelectionVectorMode selectionMode = rowSet.indirectionType();
     RowSetReader reader = rowSet.reader();
-    int colCount = reader.schema().count();
-    printSchema(out, selectionMode);
+    int colCount = reader.schema().size();
+    printSchema(out, selectionMode, reader);
     while (reader.next()) {
       printHeader(out, reader, selectionMode);
       for (int i = 0; i < colCount; i++) {
         if (i > 0) {
           out.print(", ");
         }
-        out.print(reader.getAsString(i));
+        out.print(reader.column(i).getAsString());
       }
       out.println();
     }
   }
 
-  private void printSchema(PrintStream out, SelectionVectorMode selectionMode) {
+  private void printSchema(PrintStream out, SelectionVectorMode selectionMode, RowSetReader reader) {
     out.print("#");
     switch (selectionMode) {
     case FOUR_BYTE:
@@ -68,14 +68,24 @@ public class RowSetPrinter {
       break;
     }
     out.print(": ");
-    TupleSchema schema = rowSet.schema().hierarchicalAccess();
-    for (int i = 0; i < schema.count(); i++) {
+    TupleMetadata schema = reader.schema();
+    printTupleSchema(out, schema);
+    out.println();
+  }
+
+  private void printTupleSchema(PrintStream out, TupleMetadata schema) {
+    for (int i = 0; i < schema.size(); i++) {
       if (i > 0) {
         out.print(", ");
       }
-      out.print(schema.column(i).getName());
+      ColumnMetadata colSchema = schema.metadata(i);
+      out.print(colSchema.name());
+      if (colSchema.isMap()) {
+        out.print("(");
+        printTupleSchema(out, colSchema.mapSchema());
+        out.print(")");
+      }
     }
-    out.println();
   }
 
   private void printHeader(PrintStream out, RowSetReader reader, SelectionVectorMode selectionMode) {

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReader.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReader.java
new file mode 100644
index 0000000..3e27529
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReader.java
@@ -0,0 +1,54 @@
+/*
+ * 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.test.rowSet;
+
+import org.apache.drill.exec.vector.accessor.TupleReader;
+
+/**
+ * Reader for all types of row sets.
+ */
+
+public interface RowSetReader extends TupleReader {
+
+  /**
+   * Total number of rows in the row set.
+   * @return total number of rows
+   */
+  int rowCount();
+
+  boolean next();
+  int index();
+  void set(int index);
+
+  /**
+   * Batch index: 0 for a single batch, batch for the current
+   * row is a hyper-batch.
+   * @return index of the batch for the current row
+   */
+  int batchIndex();
+
+  /**
+   * The index of the underlying row which may be indexed by an
+   * Sv2 or Sv4.
+   *
+   * @return
+   */
+
+  int rowIndex();
+  boolean valid();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
new file mode 100644
index 0000000..2bae085
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetReaderImpl.java
@@ -0,0 +1,76 @@
+/*
+ * 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.test.rowSet;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.rowSet.model.ReaderIndex;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
+import org.apache.drill.exec.vector.accessor.reader.AbstractTupleReader;
+
+/**
+ * Reader implementation for a row set.
+ */
+
+public class RowSetReaderImpl extends AbstractTupleReader implements RowSetReader {
+
+  protected final ReaderIndex readerIndex;
+
+  public RowSetReaderImpl(TupleMetadata schema, ReaderIndex index, AbstractObjectReader[] readers) {
+    super(schema, readers);
+    this.readerIndex = index;
+    bindIndex(index);
+  }
+
+  public RowSetReaderImpl(TupleMetadata schema, ReaderIndex index,
+      List<AbstractObjectReader> readers) {
+    this(schema, index,
+        readers.toArray(new AbstractObjectReader[readers.size()]));
+  }
+
+  @Override
+  public boolean next() {
+    if (! readerIndex.next()) {
+      return false;
+    }
+    reposition();
+    return true;
+  }
+
+  @Override
+  public boolean valid() { return readerIndex.valid(); }
+
+  @Override
+  public int index() { return readerIndex.position(); }
+
+  @Override
+  public int rowCount() { return readerIndex.size(); }
+
+  @Override
+  public int rowIndex() { return readerIndex.vectorIndex(); }
+
+  @Override
+  public int batchIndex() { return readerIndex.batchIndex(); }
+
+  @Override
+  public void set(int index) {
+    this.readerIndex.set(index);
+    reposition();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetSchema.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetSchema.java
deleted file mode 100644
index 55b5f12..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetSchema.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.test.rowSet;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.vector.accessor.TupleAccessor.TupleSchema;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Row set schema presented as a number of distinct "views" for various
- * purposes:
- * <ul>
- * <li>Batch schema: the schema used by a VectorContainer.</li>
- * <li>Physical schema: the schema expressed as a hierarchy of
- * tuples with the top tuple representing the row, nested tuples
- * representing maps.</li>
- * <li>Access schema: a flattened schema with all scalar columns
- * at the top level, and with map columns pulled out into a separate
- * collection. The flattened-scalar view is the one used to write to,
- * and read from, the row set.</li>
- * </ul>
- * Allows easy creation of multiple row sets from the same schema.
- * Each schema is immutable, which is fine for tests in which we
- * want known inputs and outputs.
- */
-
-public class RowSetSchema {
-
-  /**
-   * Logical description of a column. A logical column is a
-   * materialized field. For maps, also includes a logical schema
-   * of the map.
-   */
-
-  public static class LogicalColumn {
-    protected final String fullName;
-    protected final int accessIndex;
-    protected int flatIndex;
-    protected final MaterializedField field;
-
-    /**
-     * Schema of the map. Includes only those fields directly within
-     * the map; does not include fields from nested tuples.
-     */
-
-    protected PhysicalSchema mapSchema;
-
-    public LogicalColumn(String fullName, int accessIndex, MaterializedField field) {
-      this.fullName = fullName;
-      this.accessIndex = accessIndex;
-      this.field = field;
-    }
-
-    private void updateStructure(int index, PhysicalSchema children) {
-      flatIndex = index;
-      mapSchema = children;
-    }
-
-    public int accessIndex() { return accessIndex; }
-    public int flatIndex() { return flatIndex; }
-    public boolean isMap() { return mapSchema != null; }
-    public PhysicalSchema mapSchema() { return mapSchema; }
-    public MaterializedField field() { return field; }
-    public String fullName() { return fullName; }
-  }
-
-  /**
-   * Implementation of a tuple name space. Tuples allow both indexed and
-   * named access to their members.
-   *
-   * @param <T> the type of object representing each column
-   */
-
-  public static class NameSpace<T> {
-    private final Map<String,Integer> nameSpace = new HashMap<>();
-    private final List<T> columns = new ArrayList<>();
-
-    public int add(String key, T value) {
-      int index = columns.size();
-      nameSpace.put(key, index);
-      columns.add(value);
-      return index;
-    }
-
-    public T get(int index) {
-      return columns.get(index);
-    }
-
-    public T get(String key) {
-      int index = getIndex(key);
-      if (index == -1) {
-        return null;
-      }
-      return get(index);
-    }
-
-    public int getIndex(String key) {
-      Integer index = nameSpace.get(key);
-      if (index == null) {
-        return -1;
-      }
-      return index;
-    }
-
-    public int count() { return columns.size(); }
-  }
-
-  /**
-   * Provides a non-flattened, physical view of the schema. The top-level
-   * row includes maps, maps expand to a nested tuple schema. This view
-   * corresponds, more-or-less, to the physical storage of vectors in
-   * a vector accessible or vector container.
-   */
-
-  private static class TupleSchemaImpl implements TupleSchema {
-
-    private NameSpace<LogicalColumn> columns;
-
-    public TupleSchemaImpl(NameSpace<LogicalColumn> ns) {
-      this.columns = ns;
-    }
-
-    @Override
-    public MaterializedField column(int index) {
-      return logicalColumn(index).field();
-    }
-
-    public LogicalColumn logicalColumn(int index) { return columns.get(index); }
-
-    @Override
-    public MaterializedField column(String name) {
-      LogicalColumn col = columns.get(name);
-      return col == null ? null : col.field();
-    }
-
-    @Override
-    public int columnIndex(String name) {
-      return columns.getIndex(name);
-    }
-
-    @Override
-    public int count() { return columns.count(); }
-  }
-
-  /**
-   * Represents the flattened view of the schema used to get and set columns.
-   * Represents a left-to-right, depth-first traversal of the row and map
-   * columns. Holds only materialized vectors (non-maps). For completeness,
-   * provides access to maps also via separate methods, but this is generally
-   * of little use.
-   */
-
-  public static class FlattenedSchema extends TupleSchemaImpl {
-    protected final TupleSchemaImpl maps;
-
-    public FlattenedSchema(NameSpace<LogicalColumn> cols, NameSpace<LogicalColumn> maps) {
-      super(cols);
-      this.maps = new TupleSchemaImpl(maps);
-    }
-
-    public LogicalColumn logicalMap(int index) { return maps.logicalColumn(index); }
-    public MaterializedField map(int index) { return maps.column(index); }
-    public MaterializedField map(String name) { return maps.column(name); }
-    public int mapIndex(String name) { return maps.columnIndex(name); }
-    public int mapCount() { return maps.count(); }
-  }
-
-  /**
-   * Physical schema of a row set showing the logical hierarchy of fields
-   * with map fields as first-class fields. Map members appear as children
-   * under the map, much as they appear in the physical value-vector
-   * implementation.
-   */
-
-  public static class PhysicalSchema {
-    protected final NameSpace<LogicalColumn> schema = new NameSpace<>();
-
-    public LogicalColumn column(int index) {
-      return schema.get(index);
-    }
-
-    public LogicalColumn column(String name) {
-      return schema.get(name);
-    }
-
-    public int count() { return schema.count(); }
-
-    public NameSpace<LogicalColumn> nameSpace() { return schema; }
-  }
-
-  private static class SchemaExpander {
-    private final PhysicalSchema physicalSchema;
-    private final NameSpace<LogicalColumn> cols = new NameSpace<>();
-    private final NameSpace<LogicalColumn> maps = new NameSpace<>();
-
-    public SchemaExpander(BatchSchema schema) {
-      physicalSchema = expand("", schema);
-    }
-
-    private PhysicalSchema expand(String prefix, Iterable<MaterializedField> fields) {
-      PhysicalSchema physical = new PhysicalSchema();
-      for (MaterializedField field : fields) {
-        String name = prefix + field.getName();
-        int index;
-        LogicalColumn colSchema = new LogicalColumn(name, physical.count(), field);
-        physical.schema.add(field.getName(), colSchema);
-        PhysicalSchema children = null;
-        if (field.getType().getMinorType() == MinorType.MAP) {
-          index = maps.add(name, colSchema);
-          children = expand(name + ".", field.getChildren());
-        } else {
-          index = cols.add(name, colSchema);
-        }
-        colSchema.updateStructure(index, children);
-      }
-      return physical;
-    }
-  }
-
-  private final BatchSchema batchSchema;
-  private final TupleSchemaImpl accessSchema;
-  private final FlattenedSchema flatSchema;
-  private final PhysicalSchema physicalSchema;
-
-  public RowSetSchema(BatchSchema schema) {
-    batchSchema = schema;
-    SchemaExpander expander = new SchemaExpander(schema);
-    physicalSchema = expander.physicalSchema;
-    accessSchema = new TupleSchemaImpl(physicalSchema.nameSpace());
-    flatSchema = new FlattenedSchema(expander.cols, expander.maps);
-  }
-
-  /**
-   * A hierarchical schema that includes maps, with maps expanding
-   * to a nested tuple schema. Not used at present; this is intended
-   * to be the bases of non-flattened accessors if we find the need.
-   * @return the hierarchical access schema
-   */
-
-  public TupleSchema hierarchicalAccess() { return accessSchema; }
-
-  /**
-   * A flattened (left-to-right, depth-first traversal) of the non-map
-   * columns in the row. Used to define the column indexes in the
-   * get methods for row readers and the set methods for row writers.
-   * @return the flattened access schema
-   */
-
-  public FlattenedSchema flatAccess() { return flatSchema; }
-
-  /**
-   * Internal physical schema in hierarchical order. Mostly used to create
-   * the other schemas, but may be of use in special cases. Has the same
-   * structure as the batch schema, but with additional information.
-   * @return a tree-structured physical schema
-   */
-
-  public PhysicalSchema physical() { return physicalSchema; }
-
-  /**
-   * The batch schema used by the Drill runtime. Represents a tree-structured
-   * list of top-level fields, including maps. Maps contain a nested schema.
-   * @return the batch schema used by the Drill runtime
-   */
-
-  public BatchSchema batch() { return batchSchema; }
-
-  /**
-   * Convert this schema to a new batch schema that includes the specified
-   * selection vector mode.
-   * @param svMode selection vector mode for the new schema
-   * @return the new batch schema
-   */
-
-  public BatchSchema toBatchSchema(SelectionVectorMode svMode) {
-    List<MaterializedField> fields = new ArrayList<>();
-    for (MaterializedField field : batchSchema) {
-      fields.add(field);
-    }
-    return new BatchSchema(svMode, fields);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetUtilities.java
index 261a9c1..32b61ca 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetUtilities.java
@@ -17,12 +17,18 @@
  */
 package org.apache.drill.test.rowSet;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.vector.accessor.AccessorUtilities;
-import org.apache.drill.exec.vector.accessor.ColumnAccessor.ValueType;
-import org.apache.drill.exec.vector.accessor.ColumnWriter;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.bouncycastle.util.Arrays;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
@@ -62,11 +68,42 @@ public class RowSetUtilities {
    */
 
   public static void setFromInt(RowSetWriter rowWriter, int index, int value) {
-    ColumnWriter writer = rowWriter.column(index);
-    if (writer.valueType() == ValueType.PERIOD) {
-      setPeriodFromInt(writer, rowWriter.schema().column(index).getType().getMinorType(), value);
-    } else {
-      AccessorUtilities.setFromInt(writer, value);
+    ScalarWriter writer = rowWriter.scalar(index);
+    MaterializedField field = rowWriter.schema().column(index);
+    writer.setObject(testDataFromInt(writer.valueType(), field.getType(), value));
+  }
+
+  public static Object testDataFromInt(ValueType valueType, MajorType dataType, int value) {
+    switch (valueType) {
+    case BYTES:
+      return Integer.toHexString(value).getBytes();
+    case DOUBLE:
+      return (double) value;
+    case INTEGER:
+      switch (dataType.getMinorType()) {
+      case BIT:
+        return value & 0x01;
+      case SMALLINT:
+        return value % 32768;
+      case UINT2:
+        return value & 0xFFFF;
+      case TINYINT:
+        return value % 128;
+      case UINT1:
+        return value & 0xFF;
+      default:
+        return value;
+      }
+    case LONG:
+      return (long) value;
+    case STRING:
+      return Integer.toString(value);
+    case DECIMAL:
+      return BigDecimal.valueOf(value, dataType.getScale());
+    case PERIOD:
+      return periodFromInt(dataType.getMinorType(), value);
+    default:
+      throw new IllegalStateException("Unknown writer type: " + valueType);
     }
   }
 
@@ -81,26 +118,56 @@ public class RowSetUtilities {
    * @param writer column writer for a period column
    * @param minorType the Drill data type
    * @param value the integer value to apply
+   * @throws VectorOverflowException
    */
 
-  public static void setPeriodFromInt(ColumnWriter writer, MinorType minorType,
-      int value) {
+  public static Period periodFromInt(MinorType minorType, int value) {
     switch (minorType) {
     case INTERVAL:
-      writer.setPeriod(Duration.millis(value).toPeriod());
-      break;
+      return Duration.millis(value).toPeriod();
     case INTERVALYEAR:
-      writer.setPeriod(Period.years(value / 12).withMonths(value % 12));
-      break;
+      return Period.years(value / 12).withMonths(value % 12);
     case INTERVALDAY:
       int sec = value % 60;
       value = value / 60;
       int min = value % 60;
       value = value / 60;
-      writer.setPeriod(Period.days(value).withMinutes(min).withSeconds(sec));
-      break;
+      return Period.days(value).withMinutes(min).withSeconds(sec);
     default:
       throw new IllegalArgumentException("Writer is not an interval: " + minorType);
     }
   }
+
+  public static void assertEqualValues(ValueType type, Object expectedObj, Object actualObj) {
+    assertEqualValues(type.toString(), type, expectedObj, actualObj);
+  }
+
+  public static void assertEqualValues(String msg, ValueType type, Object expectedObj, Object actualObj) {
+    switch (type) {
+    case BYTES: {
+        byte expected[] = (byte[]) expectedObj;
+        byte actual[] = (byte[]) actualObj;
+        assertEquals(msg + " - byte lengths differ", expected.length, actual.length);
+        assertTrue(msg, Arrays.areEqual(expected, actual));
+        break;
+     }
+     case DOUBLE:
+       assertEquals(msg, (double) expectedObj, (double) actualObj, 0.0001);
+       break;
+     case INTEGER:
+     case LONG:
+     case STRING:
+     case DECIMAL:
+       assertEquals(msg, expectedObj, actualObj);
+       break;
+     case PERIOD: {
+       Period expected = (Period) expectedObj;
+       Period actual = (Period) actualObj;
+       assertEquals(msg, expected.normalizedStandard(), actual.normalizedStandard());
+       break;
+     }
+     default:
+        throw new IllegalStateException( "Unexpected type: " + type);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriter.java
new file mode 100644
index 0000000..874c0e1
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriter.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.test.rowSet;
+
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+
+/**
+ * Interface for writing values to a row set. Only available
+ * for newly-created, single, direct row sets. Eventually, if
+ * we want to allow updating a row set, we have to create a
+ * new row set with the updated columns, then merge the new
+ * and old row sets to create a new immutable row set.
+ * <p>
+ * Typical usage:
+ * <pre></code>
+ * void writeABatch() {
+ *   RowSetWriter writer = ...
+ *   while (! writer.isFull()) {
+ *     writer.scalar(0).setInt(10);
+ *     writer.scalar(1).setString("foo");
+ *     ...
+ *     writer.save();
+ *   }
+ * }</code></pre>
+ * The above writes until the batch is full, based on size. If values
+ * are large enough to potentially cause vector overflow, do the
+ * following instead:
+ * <pre></code>
+ * void writeABatch() {
+ *   RowSetWriter writer = ...
+ *   while (! writer.isFull()) {
+ *     writer.column(0).setInt(10);
+ *     try {
+ *        writer.column(1).setString("foo");
+ *     } catch (VectorOverflowException e) { break; }
+ *     ...
+ *     writer.save();
+ *   }
+ *   // Do something with the partially-written last row.
+ * }</code></pre>
+ * <p>
+ * This writer is for testing, so no provision is available to handle a
+ * partial last row. (Elsewhere n Drill there are classes that handle that case.)
+ */
+
+public interface RowSetWriter extends TupleWriter {
+
+  /**
+   * Write a row of values, given by Java objects. Object type must
+   * match expected column type. Stops writing, and returns false,
+   * if any value causes vector overflow. Value format:
+   * <ul>
+   * <li>For scalars, the value as a suitable Java type (int or
+   * Integer, say, for <tt>INTEGER</tt> values.)</li>
+   * <li>For scalar arrays, an array of a suitable Java primitive type
+   * for scalars. For example, <tt>int[]</tt> for an <tt>INTEGER</tt>
+   * column.</li>
+   * <li>For a Map, an <tt>Object<tt> array with values encoded as above.
+   * (In fact, the list here is the same as the map format.</li>
+   * <li>For a list (repeated map, list of list), an <tt>Object</tt>
+   * array with values encoded as above. (So, for a repeated map, an outer
+   * <tt>Object</tt> map encodes the array, an inner one encodes the
+   * map members.</li>
+   * </ul>
+   *
+   * @param values variable-length argument list of column values
+   */
+
+  void setRow(Object...values);
+
+  /**
+   * Indicates if the current row position is valid for
+   * writing. Will be false on the first row, and all subsequent
+   * rows until either the maximum number of rows are written,
+   * or a vector overflows. After that, will return true. The
+   * method returns false as soon as any column writer overflows
+   * even in the middle of a row write. That is, this writer
+   * does not automatically handle overflow rows because that
+   * added complexity is seldom needed for tests.
+   *
+   * @return true if the current row can be written, false
+   * if not
+   */
+
+  boolean isFull();
+  int rowIndex();
+
+  /**
+   * Saves the current row and moves to the next row.
+   * Done automatically if using <tt>setRow()</tt>.
+   */
+
+  void save();
+
+  /**
+   * Finish writing and finalize the row set being
+   * written.
+   * @return the completed, read-only row set without a
+   * selection vector
+   */
+
+  SingleRowSet done();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
new file mode 100644
index 0000000..074842d
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetWriterImpl.java
@@ -0,0 +1,155 @@
+/*
+ * 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.test.rowSet;
+
+import java.util.List;
+
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter;
+import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+
+/**
+ * Implementation of a row set writer. Only available for newly-created,
+ * empty, direct, single row sets. Rewriting is not allowed, nor is writing
+ * to a hyper row set.
+ */
+
+public class RowSetWriterImpl extends AbstractTupleWriter implements RowSetWriter {
+
+  /**
+   * Writer index that points to each row in the row set. The index starts at
+   * the 0th row and advances one row on each increment. This allows writers to
+   * start positioned at the first row. Writes happen in the current row.
+   * Calling <tt>next()</tt> advances to the next position, effectively saving
+   * the current row. The most recent row can be abandoned easily simply by not
+   * calling <tt>next()</tt>. This means that the number of completed rows is
+   * the same as the row index.
+   */
+
+  static class WriterIndexImpl implements ColumnWriterIndex {
+
+    public enum State { OK, VECTOR_OVERFLOW, END_OF_BATCH }
+
+    private int rowIndex = 0;
+    private State state = State.OK;
+
+    @Override
+    public final int vectorIndex() { return rowIndex; }
+
+    public final boolean next() {
+      if (++rowIndex < ValueVector.MAX_ROW_COUNT) {
+        return true;
+      }
+      // Should not call next() again once batch is full.
+      assert rowIndex == ValueVector.MAX_ROW_COUNT;
+      rowIndex = ValueVector.MAX_ROW_COUNT;
+      state = state == State.OK ? State.END_OF_BATCH : state;
+      return false;
+    }
+
+    public int size() {
+      // The index always points to the next slot past the
+      // end of valid rows.
+      return rowIndex;
+    }
+
+    public boolean valid() { return state == State.OK; }
+
+    public boolean hasOverflow() { return state == State.VECTOR_OVERFLOW; }
+
+    @Override
+    public final void nextElement() { }
+
+    @Override
+    public void rollover() {
+      throw new UnsupportedOperationException("Rollover not supported in the row set writer.");
+    }
+
+    @Override
+    public int rowStartIndex() { return rowIndex; }
+
+    @Override
+    public ColumnWriterIndex outerIndex() { return null; }
+
+    @Override
+    public String toString() {
+      return new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" state = ")
+        .append(state)
+        .append(", rowIndex = ")
+        .append(rowIndex)
+        .append("]")
+        .toString();
+    }
+  }
+
+  private final WriterIndexImpl writerIndex;
+  private final ExtendableRowSet rowSet;
+
+  protected RowSetWriterImpl(ExtendableRowSet rowSet, TupleMetadata schema, WriterIndexImpl index, List<AbstractObjectWriter> writers) {
+    super(schema, writers);
+    this.rowSet = rowSet;
+    this.writerIndex = index;
+    bindIndex(index);
+    startWrite();
+    startRow();
+  }
+
+  @Override
+  public void setRow(Object...values) {
+    setObject(values);
+    save();
+  }
+
+  @Override
+  public int rowIndex() { return writerIndex.vectorIndex(); }
+
+  @Override
+  public void save() {
+    endArrayValue();
+    saveRow();
+
+    // For convenience, start a new row after each save.
+    // The last (unused) row is abandoned when the batch is full.
+
+    if (writerIndex.next()) {
+      startRow();
+    }
+  }
+
+  @Override
+  public boolean isFull( ) { return ! writerIndex.valid(); }
+
+  @Override
+  public SingleRowSet done() {
+    endWrite();
+    rowSet.container().setRecordCount(writerIndex.vectorIndex());
+    return rowSet;
+  }
+
+  @Override
+  public int lastWriteIndex() {
+    return writerIndex.vectorIndex();
+  }
+}


[10/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
new file mode 100644
index 0000000..ffcc84a
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderProtocol.java
@@ -0,0 +1,586 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter.UndefinedColumnException;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Tests of the overall result set loader protocol focusing on which operations
+ * are valid in each state, basics of column lookup, basics of adding columns
+ * and so on. Uses the simplest possible type: a required int.
+ * <p>
+ * Run this test first to do a sanity check of the result set loader after making
+ * changes.
+ * <p>
+ * You will find that the result set loader creates a very complex tree of
+ * objects that can be quite hard to understand and debug. Please read the
+ * material in the various subsystems to see how the classes fit together
+ * to implement Drill's rich JSON-like data model.
+ * <p>
+ * To aid in debugging, you can also dump the result set loader, and all its
+ * child objects as follows:<pre><code>
+ * ((ResultSetLoaderImpl) rsLoader).dump(new HierarchicalPrinter());
+ * </code></pre>
+ * Simply insert that line into these tests anywhere you want to visualize
+ * the structure. The object tree will show all the components and their
+ * current state.
+ */
+
+public class TestResultSetLoaderProtocol extends SubOperatorTest {
+
+  @Test
+  public void testBasics() {
+    ResultSetLoaderImpl rsLoaderImpl = new ResultSetLoaderImpl(fixture.allocator());
+    ResultSetLoader rsLoader = rsLoaderImpl;
+    assertEquals(0, rsLoader.schemaVersion());
+    assertEquals(ResultSetLoader.DEFAULT_ROW_COUNT, rsLoader.targetRowCount());
+    assertEquals(ValueVector.MAX_BUFFER_SIZE, rsLoader.targetVectorSize());
+    assertEquals(0, rsLoader.writer().rowCount());
+    assertEquals(0, rsLoader.batchCount());
+    assertEquals(0, rsLoader.totalRowCount());
+
+    // Failures due to wrong state (Start)
+
+    try {
+      rsLoader.harvest();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    // Can define schema before starting the first batch.
+
+    RowSetLoader rootWriter = rsLoader.writer();
+    TupleMetadata schema = rootWriter.schema();
+    assertEquals(0, schema.size());
+
+    MaterializedField fieldA = SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED);
+    rootWriter.addColumn(fieldA);
+
+    assertEquals(1, schema.size());
+    assertSame(fieldA, schema.column(0));
+    assertSame(fieldA, schema.column("a"));
+
+    // Error to start a row before the first batch.
+
+    try {
+      rootWriter.start();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    // Error to end a row before the first batch.
+
+    try {
+      rootWriter.save();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    // Because writing is an inner loop; no checks are
+    // done to ensure that writing occurs only in the proper
+    // state. So, can't test setInt() in the wrong state.
+
+    rsLoader.startBatch();
+    try {
+      rsLoader.startBatch();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    assertFalse(rootWriter.isFull());
+
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(100);
+    assertEquals(0, rootWriter.rowCount());
+    assertEquals(0, rsLoader.batchCount());
+    rootWriter.save();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(1, rsLoader.batchCount());
+    assertEquals(1, rsLoader.totalRowCount());
+
+    // Can add a field after first row, prior rows are
+    // "back-filled".
+
+    MaterializedField fieldB = SchemaBuilder.columnSchema("b", MinorType.INT, DataMode.OPTIONAL);
+    rootWriter.addColumn(fieldB);
+
+    assertEquals(2, schema.size());
+    assertSame(fieldB, schema.column(1));
+    assertSame(fieldB, schema.column("b"));
+
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(200);
+    rootWriter.scalar(1).setInt(210);
+    rootWriter.save();
+    assertEquals(2, rootWriter.rowCount());
+    assertEquals(1, rsLoader.batchCount());
+    assertEquals(2, rsLoader.totalRowCount());
+
+    // Harvest the first batch. Version number is the number
+    // of columns added.
+
+    assertFalse(rootWriter.isFull());
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(2, rsLoader.schemaVersion());
+    assertEquals(0, rootWriter.rowCount());
+    assertEquals(1, rsLoader.batchCount());
+    assertEquals(2, rsLoader.totalRowCount());
+
+    SingleRowSet expected = fixture.rowSetBuilder(result.batchSchema())
+        .addRow(100, null)
+        .addRow(200, 210)
+        .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(result);
+
+    // Between batches: batch-based operations fail
+
+    try {
+      rootWriter.start();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rsLoader.harvest();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rootWriter.save();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    // Create a second batch
+
+    rsLoader.startBatch();
+    assertEquals(0, rootWriter.rowCount());
+    assertEquals(1, rsLoader.batchCount());
+    assertEquals(2, rsLoader.totalRowCount());
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(300);
+    rootWriter.scalar(1).setInt(310);
+    rootWriter.save();
+    assertEquals(1, rootWriter.rowCount());
+    assertEquals(2, rsLoader.batchCount());
+    assertEquals(3, rsLoader.totalRowCount());
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(400);
+    rootWriter.scalar(1).setInt(410);
+    rootWriter.save();
+
+    // Harvest. Schema has not changed.
+
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(2, rsLoader.schemaVersion());
+    assertEquals(0, rootWriter.rowCount());
+    assertEquals(2, rsLoader.batchCount());
+    assertEquals(4, rsLoader.totalRowCount());
+
+    expected = fixture.rowSetBuilder(result.batchSchema())
+        .addRow(300, 310)
+        .addRow(400, 410)
+        .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(result);
+
+    // Next batch. Schema has changed.
+
+    rsLoader.startBatch();
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(500);
+    rootWriter.scalar(1).setInt(510);
+    rootWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL));
+    rootWriter.scalar(2).setInt(520);
+    rootWriter.save();
+    rootWriter.start();
+    rootWriter.scalar(0).setInt(600);
+    rootWriter.scalar(1).setInt(610);
+    rootWriter.scalar(2).setInt(620);
+    rootWriter.save();
+
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(3, rsLoader.schemaVersion());
+    expected = fixture.rowSetBuilder(result.batchSchema())
+        .addRow(500, 510, 520)
+        .addRow(600, 610, 620)
+        .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(result);
+
+    rsLoader.close();
+
+    // Key operations fail after close.
+
+    try {
+      rootWriter.start();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rsLoader.writer();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rsLoader.startBatch();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rsLoader.harvest();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+    try {
+      rootWriter.save();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    // Benign to close twice
+
+    rsLoader.close();
+  }
+
+  /**
+   * Schemas are case insensitive by default. Verify that
+   * the schema mechanism works, with emphasis on the
+   * case insensitive case.
+   */
+
+  @Test
+  public void testCaseInsensitiveSchema() {
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator());
+    RowSetLoader rootWriter = rsLoader.writer();
+    TupleMetadata schema = rootWriter.schema();
+
+    // No columns defined in schema
+
+    assertNull(schema.column("a"));
+    try {
+      schema.column(0);
+      fail();
+    } catch (IndexOutOfBoundsException e) {
+      // Expected
+    }
+
+    // No columns defined in writer
+
+    try {
+      rootWriter.column("a");
+      fail();
+    } catch (UndefinedColumnException e) {
+      // Expected
+    }
+    try {
+      rootWriter.column(0);
+      fail();
+    } catch (IndexOutOfBoundsException e) {
+      // Expected
+    }
+
+    // Define a column
+
+    MaterializedField colSchema = SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED);
+    rootWriter.addColumn(colSchema);
+
+    // Can now be found, case insensitive
+
+    assertSame(colSchema, schema.column(0));
+    assertSame(colSchema, schema.column("a"));
+    assertSame(colSchema, schema.column("A"));
+    assertNotNull(rootWriter.column(0));
+    assertNotNull(rootWriter.column("a"));
+    assertNotNull(rootWriter.column("A"));
+    assertEquals(1, schema.size());
+    assertEquals(0, schema.index("a"));
+    assertEquals(0, schema.index("A"));
+
+    // Reject a duplicate name, case insensitive
+
+    try {
+      rootWriter.addColumn(colSchema);
+      fail();
+    } catch(IllegalArgumentException e) {
+      // Expected
+    }
+    try {
+      MaterializedField testCol = SchemaBuilder.columnSchema("A", MinorType.VARCHAR, DataMode.REQUIRED);
+      rootWriter.addColumn(testCol);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+      assertTrue(e.getMessage().contains("Duplicate"));
+    }
+
+    // Can still add required fields while writing the first row.
+
+    rsLoader.startBatch();
+    rootWriter.start();
+    rootWriter.scalar(0).setString("foo");
+
+    MaterializedField col2 = SchemaBuilder.columnSchema("b", MinorType.VARCHAR, DataMode.REQUIRED);
+    rootWriter.addColumn(col2);
+    assertSame(col2, schema.column(1));
+    assertSame(col2, schema.column("b"));
+    assertSame(col2, schema.column("B"));
+    assertEquals(2, schema.size());
+    assertEquals(1, schema.index("b"));
+    assertEquals(1, schema.index("B"));
+    rootWriter.scalar(1).setString("second");
+
+    // After first row, can add an optional or repeated.
+    // Also allows a required field: values will be back-filled.
+
+    rootWriter.save();
+    rootWriter.start();
+    rootWriter.scalar(0).setString("bar");
+    rootWriter.scalar(1).setString("");
+
+    MaterializedField col3 = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REQUIRED);
+    rootWriter.addColumn(col3);
+    assertSame(col3, schema.column(2));
+    assertSame(col3, schema.column("c"));
+    assertSame(col3, schema.column("C"));
+    assertEquals(3, schema.size());
+    assertEquals(2, schema.index("c"));
+    assertEquals(2, schema.index("C"));
+    rootWriter.scalar("c").setString("c.2");
+
+    MaterializedField col4 = SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.OPTIONAL);
+    rootWriter.addColumn(col4);
+    assertSame(col4, schema.column(3));
+    assertSame(col4, schema.column("d"));
+    assertSame(col4, schema.column("D"));
+    assertEquals(4, schema.size());
+    assertEquals(3, schema.index("d"));
+    assertEquals(3, schema.index("D"));
+    rootWriter.scalar("d").setString("d.2");
+
+    MaterializedField col5 = SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.REPEATED);
+    rootWriter.addColumn(col5);
+    assertSame(col5, schema.column(4));
+    assertSame(col5, schema.column("e"));
+    assertSame(col5, schema.column("E"));
+    assertEquals(5, schema.size());
+    assertEquals(4, schema.index("e"));
+    assertEquals(4, schema.index("E"));
+    rootWriter.array(4).set("e1", "e2", "e3");
+    rootWriter.save();
+
+    // Verify. No reason to expect problems, but might as well check.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(5, rsLoader.schemaVersion());
+    SingleRowSet expected = fixture.rowSetBuilder(result.batchSchema())
+        .addRow("foo", "second", "",    null,  new String[] { } )
+        .addRow("bar", "",       "c.2", "d.2", new String[] {"e1", "e2", "e3"} )
+        .build();
+    new RowSetComparison(expected)
+        .verifyAndClearAll(result);
+
+    // Handy way to test that close works to abort an in-flight batch
+    // and clean up.
+
+    rsLoader.close();
+  }
+
+  /**
+   * Provide a schema up front to the loader; schema is built before
+   * the first row.
+   * <p>
+   * Also verifies the test-time method to set a row of values using
+   * a single method.
+   */
+
+  @Test
+  public void testInitialSchema() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.INT)
+        .add("c", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    rootWriter
+        .addRow(10, 100, "fred")
+        .addRow(20, null, "barney")
+        .addRow(30, 300, "wilma");
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+
+    RowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, 100, "fred")
+        .addRow(20, null, "barney")
+        .addRow(30, 300, "wilma")
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+    rsLoader.close();
+  }
+
+  /**
+   * The writer protocol allows a client to write to a row any number of times
+   * before invoking <tt>save()</tt>. In this case, each new value simply
+   * overwrites the previous value. Here, we test the most basic case: a simple,
+   * flat tuple with no arrays. We use a very large Varchar that would, if
+   * overwrite were not working, cause vector overflow.
+   * <p>
+   * The ability to overwrite rows is seldom needed except in one future use
+   * case: writing a row, then applying a filter "in-place" to discard unwanted
+   * rows, without having to send the row downstream.
+   * <p>
+   * Because of this use case, specific rules apply when discarding row or
+   * overwriting values.
+   * <ul>
+   * <li>Values can be written once per row. Fixed-width columns actually allow
+   * multiple writes. But, because of the way variable-width columns work,
+   * multiple writes will cause undefined results.</li>
+   * <li>To overwrite a row, call <tt>start()</tt> without calling
+   * <tt>save()</tt> on the previous row. Doing so ignores data for the
+   * previous row and starts a new row in place of the old one.</li>
+   * </ul>
+   * Note that there is no explicit method to discard a row. Instead,
+   * the rule is that a row is not saved until <tt>save()</tt> is called.
+   */
+
+  @Test
+  public void testOverwriteRow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Can't use the shortcut to populate rows when doing overwrites.
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    ScalarWriter bWriter = rootWriter.scalar("b");
+
+    // Write 100,000 rows, overwriting 99% of them. This will cause vector
+    // overflow and data corruption if overwrite does not work; but will happily
+    // produce the correct result if everything works as it should.
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (count < 100_000) {
+      rootWriter.start();
+      count++;
+      aWriter.setInt(count);
+      bWriter.setBytes(value, value.length);
+      if (count % 100 == 0) {
+        rootWriter.save();
+      }
+    }
+
+    // Verify using a reader.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(count / 100, result.rowCount());
+    RowSetReader reader = result.reader();
+    int rowId = 1;
+    while (reader.next()) {
+      assertEquals(rowId * 100, reader.scalar("a").getInt());
+      assertTrue(Arrays.equals(value, reader.scalar("b").getBytes()));
+      rowId++;
+    }
+
+    result.clear();
+    rsLoader.close();
+  }
+
+  /**
+   * Test that memory is released if the loader is closed with an active
+   * batch (that is, before the batch is harvested.)
+   */
+
+  @Test
+  public void testCloseWithoutHarvest() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .add("b", MinorType.VARCHAR)
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    rsLoader.startBatch();
+    for (int i = 0; i < 100; i++) {
+      rootWriter.start();
+      rootWriter.scalar("a").setInt(i);
+      rootWriter.scalar("b").setString("b-" + i);
+      rootWriter.save();
+    }
+
+    // Don't harvest the batch. Allocator will complain if the
+    // loader does not release memory.
+
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
new file mode 100644
index 0000000..33b9826
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderTorture.java
@@ -0,0 +1,453 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.test.LogFixture;
+import org.apache.drill.test.LogFixture.LogFixtureBuilder;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+
+/**
+ * Runs a worst-case scenario test that combines aspects of all
+ * previous tests. Run this test only <i>after</i> all other tests
+ * pass. Combined conditions tested:
+ * <ul>
+ * <li>Nested maps and map arrays.</li>
+ * <li>Nullable VarChar (which has an offset vector and null-bit vector
+ * be kept in sync.)
+ * <li>Repeated Varchar (which requires to offset vectors be kept in
+ * sync.)</li>
+ * <li>Null values.</li>
+ * <li>Omitted values.</li>
+ * <li>Skipped rows.</li>
+ * <li>Vector overflow deep in the structure.</li>
+ * <li>Multiple batches.</li>
+ * </ul>
+ * The proposition that this test asserts is that if this test passes,
+ * then most clients will also work as they generally do not do all these
+ * things in a single query.
+ */
+
+public class TestResultSetLoaderTorture extends SubOperatorTest {
+
+  private static class TestSetup {
+    int n1Cycle = 5;
+    int n2Cycle = 7;
+    int s2Cycle = 11;
+    int m2Cycle = 13;
+    int n3Cycle = 17;
+    int s3Cycle = 19;
+    int skipCycle = 23;
+    int nullCycle = 3;
+    int m2Count = 9;
+    int s3Count = 29;
+
+    String s3Value;
+
+    public TestSetup() {
+      byte s3Bytes[] = new byte[512];
+      Arrays.fill(s3Bytes, (byte) 'X');
+      s3Value = new String(s3Bytes, Charsets.UTF_8);
+    }
+  }
+
+  // Write rows, skipping every 10th.
+  // n0 is the row id, so appears in every row.
+  // For n1, n2 and n3 and s2, omit selected values and makes others null.
+  // For s3, write values large enough to cause overflow; but skip some
+  // values and write 0 values for others.
+
+  private static class BatchWriter {
+
+    TestSetup setup;
+    RowSetLoader rootWriter;
+    ScalarWriter n1Writer;
+    ArrayWriter a2Writer;
+    ScalarWriter n2Writer;
+    ScalarWriter s2Writer;
+    ScalarWriter n3Writer;
+    ScalarWriter s3Writer;
+    int rowId = 0;
+    int innerCount = 0;
+    int writeRowCount = 0;
+    int startPrint = -1;
+    int endPrint = -1;
+    boolean lastRowDiscarded;
+
+    public BatchWriter(TestSetup setup, RowSetLoader rootWriter) {
+      this.setup = setup;
+      this.rootWriter = rootWriter;
+
+      TupleWriter m1Writer = rootWriter.tuple("m1");
+      n1Writer = m1Writer.scalar("n1");
+      a2Writer = m1Writer.array("m2");
+      TupleWriter m2Writer = a2Writer.tuple();
+      n2Writer = m2Writer.scalar("n2");
+      s2Writer = m2Writer.scalar("s2");
+      TupleWriter m3Writer = m2Writer.tuple("m3");
+      n3Writer = m3Writer.scalar("n3");
+      s3Writer = m3Writer.array("s3").scalar();
+    }
+
+    public void writeBatch() {
+
+      // Write until overflow
+
+      writeRowCount = rootWriter.rowCount();
+      //System.out.println("Start count: " + writeRowCount);
+      while (! rootWriter.isFull()) {
+        lastRowDiscarded = false;
+        writeRow();
+        rowId++;
+      }
+//      System.out.println("End of batch: rowId: " + rowId +
+//          ", count: " + writeRowCount +
+//          ", writer count:" + rootWriter.rowCount());
+    }
+
+    private void writeRow() {
+      rootWriter.start();
+
+      // Outer column
+
+      rootWriter.scalar("n0").setInt(rowId);
+      print("n0", rowId);
+
+      // Map 1: non-array
+
+      setInt("n1", n1Writer, rowId, setup.n1Cycle);
+
+      // Map2: an array.
+
+      if (rowId % setup.m2Cycle != 0) {
+        writeM2Array();
+      }
+
+      // Skip some rows
+
+      if (rowId % setup.skipCycle != 0) {
+        rootWriter.save();
+        writeRowCount++;
+      } else {
+        lastRowDiscarded = true;
+//        System.out.println("Skip row ID: " + rowId +
+//            ", count: " + writeRowCount +
+//            ", row set: " + rootWriter.rowCount());
+      }
+      if (rowId >= startPrint &&  rowId <= endPrint) {
+        System.out.println();
+      }
+    }
+
+    private void writeM2Array() {
+      for (int i = 0; i < setup.m2Count; i++) {
+
+        // n2: usual int
+
+        setInt("n2." + i, n2Writer, innerCount, setup.n2Cycle);
+
+        // S2: a nullable Varchar
+
+        if (innerCount % setup.s2Cycle == 0) {
+          // Skip
+        } else if (innerCount % setup.s2Cycle % setup.nullCycle == 0) {
+          s2Writer.setNull();
+          print("s2." + i, null);
+        } else {
+          s2Writer.setString("s2-" + innerCount);
+          print("s2." + i, "s2-" + innerCount);
+        }
+
+        // Map3: a non-repeated map
+
+        // n2: usual int
+
+        setInt("n3." + i, n3Writer, innerCount, setup.n3Cycle);
+
+        // s3: a repeated VarChar
+
+        if (innerCount % setup.s3Cycle != 0) {
+          for (int j = 0; j < setup.s3Count; j++) {
+            s3Writer.setString(setup.s3Value + (innerCount * setup.s3Count + j));
+          }
+          print("s3." + i, setup.s3Count + "x");
+        }
+        innerCount++;
+        a2Writer.save();
+      }
+    }
+
+    public void setInt(String label, ScalarWriter writer, int id, int cycle) {
+      int cycleIndex = id % cycle;
+      if (cycleIndex == 0) {
+        // Skip
+      } else if (cycleIndex % setup.nullCycle == 0) {
+        writer.setNull();
+        print(label, null);
+      } else {
+        writer.setInt(id * cycle);
+        print(label, id * cycle);
+      }
+    }
+
+    public void print(String label, Object value) {
+      if (rowId >= startPrint &&  rowId <= endPrint) {
+        System.out.print(label);
+        System.out.print(" = ");
+        System.out.print(value);
+        System.out.print(" ");
+      }
+    }
+
+    public int rowCount() {
+      return writeRowCount -
+          (lastRowDiscarded ? 0 : 1);
+    }
+  }
+
+  public static class ReadState {
+    int rowId = 0;
+    int innerCount = 0;
+  }
+
+  private static class BatchReader {
+
+    private TestSetup setup;
+    private RowSetReader rootReader;
+    ScalarReader n1Reader;
+    ArrayReader a2Reader;
+    ScalarReader n2Reader;
+    ScalarReader s2Reader;
+    ScalarReader n3Reader;
+    ScalarElementReader s3Reader;
+    ReadState readState;
+
+    public BatchReader(TestSetup setup, RowSetReader reader, ReadState readState) {
+      this.setup = setup;
+      this.rootReader = reader;
+      this.readState = readState;;
+
+      TupleReader m1Reader = rootReader.tuple("m1");
+      n1Reader = m1Reader.scalar("n1");
+      a2Reader = m1Reader.array("m2");
+      TupleReader m2Reader = a2Reader.tuple();
+      n2Reader = m2Reader.scalar("n2");
+      s2Reader = m2Reader.scalar("s2");
+      TupleReader m3Reader = m2Reader.tuple("m3");
+      n3Reader = m3Reader.scalar("n3");
+      s3Reader = m3Reader.array("s3").elements();
+    }
+
+    public void verify() {
+      while (rootReader.next()) {
+//        System.out.println(readState.rowId);
+        verifyRow();
+        readState.rowId++;
+      }
+    }
+
+    private void verifyRow() {
+      // Skipped original row? Bump the row id.
+
+      if (readState.rowId % setup.skipCycle == 0) {
+        if (readState.rowId % setup.m2Cycle != 0) {
+          readState.innerCount += setup.m2Count;
+        }
+        readState.rowId++;
+      }
+
+      // Outer column
+
+      assertEquals(readState.rowId, rootReader.scalar("n0").getInt());
+
+      // Map 1: non-array
+
+      checkInt(n1Reader, readState.rowId, setup.n1Cycle);
+
+      // Map2: an array.
+
+      if (readState.rowId % setup.m2Cycle == 0) {
+        assertEquals(0, a2Reader.size());
+      } else {
+        verifyM2Array();
+      }
+    }
+
+    private void verifyM2Array() {
+      for (int i = 0; i < setup.m2Count; i++) {
+        a2Reader.setPosn(i);
+
+        // n2: usual int
+
+        checkInt(n2Reader, readState.innerCount, setup.n2Cycle);
+
+        if (readState.innerCount % setup.s2Cycle == 0) {
+          // Skipped values should be null
+          assertTrue(
+              String.format("Row %d, entry %d", rootReader.rowIndex(), i),
+              s2Reader.isNull());
+        } else if (readState.innerCount % setup.s2Cycle % setup.nullCycle == 0) {
+          assertTrue(s2Reader.isNull());
+        } else {
+          assertEquals("s2-" + readState.innerCount, s2Reader.getString());
+        }
+
+        // Map3: a non-repeated map
+
+        // n2: usual int
+
+        checkInt(n3Reader, readState.innerCount, setup.n3Cycle);
+
+        // s3: a repeated VarChar
+
+        if (readState.innerCount % setup.s3Cycle == 0) {
+          assertEquals(0, s3Reader.size());
+        } else {
+          for (int j = 0; j < setup.s3Count; j++) {
+            assertEquals(setup.s3Value + (readState.innerCount * setup.s3Count + j), s3Reader.getString(j));
+          }
+        }
+        readState.innerCount++;
+      }
+    }
+
+    public void checkInt(ScalarReader reader, int id, int cycle) {
+      if (id % cycle == 0) {
+        // Skipped values should be null
+        assertTrue("id = " + id + " expected null for skipped", reader.isNull());
+      } else if (id % cycle % setup.nullCycle == 0) {
+        assertTrue(reader.isNull());
+      } else {
+        assertEquals(id * cycle, reader.getInt());
+      }
+    }
+  }
+
+  @Test
+  public void tortureTest() {
+    LogFixtureBuilder logBuilder = new LogFixtureBuilder()
+
+        // Enable to get detailed tracing when things go wrong.
+
+//        .logger("org.apache.drill.exec.physical.rowSet", Level.TRACE)
+        ;
+    try (LogFixture logFixture = logBuilder.build()) {
+      doTortureTest();
+    }
+  }
+
+  private void doTortureTest() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("n0", MinorType.INT)
+        .addMap("m1")
+          .addNullable("n1", MinorType.INT)
+          .addMapArray("m2")
+            .addNullable("n2", MinorType.INT)
+            .addNullable("s2", MinorType.VARCHAR)
+            .addMap("m3")
+              .addNullable("n3", MinorType.INT)
+              .addArray("s3", MinorType.VARCHAR)
+              .buildMap()
+            .buildMap()
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    TestSetup setup = new TestSetup();
+    BatchWriter batchWriter = new BatchWriter(setup, rootWriter);
+
+    int totalRowCount = 0;
+
+    ReadState readState = new ReadState();
+    for (int batchCount = 0; batchCount < 10; batchCount++) {
+      rsLoader.startBatch();
+      batchWriter.writeBatch();
+
+      // Now the hard part. Verify the above batch.
+
+      RowSet result = fixture.wrap(rsLoader.harvest());
+//      result.print();
+
+      // Should have overflowed
+
+      int savedCount = batchWriter.rowCount();
+      assertEquals(savedCount, result.rowCount());
+
+      totalRowCount += savedCount;
+      assertEquals(totalRowCount, rsLoader.totalRowCount());
+      assertEquals(batchCount + 1, rsLoader.batchCount());
+
+      BatchReader reader = new BatchReader(setup, result.reader(), readState);
+      reader.verify();
+      result.clear();
+    }
+
+    // Last row overflow row
+
+    {
+      rsLoader.startBatch();
+
+      // Use this to visualize a string buffer. There is also a method
+      // to visualize offset vectors. These two are the most pesky vectors
+      // to get right.
+
+//      VectorPrinter.printStrings((VarCharVector) ((NullableVarCharVector) ((AbstractScalarWriter) batchWriter.s2Writer).vector()).getValuesVector(), 0, 8);
+      RowSet result = fixture.wrap(rsLoader.harvest());
+
+      // Use this here, or earlier, when things go amiss and you need
+      // to see what the actual results might be.
+
+//      result.print();
+
+      totalRowCount++;
+      assertEquals(totalRowCount, rsLoader.totalRowCount());
+
+      BatchReader reader = new BatchReader(setup, result.reader(), readState);
+      reader.verify();
+      result.clear();
+    }
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetSchemaChange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetSchemaChange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetSchemaChange.java
new file mode 100644
index 0000000..9787189
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetSchemaChange.java
@@ -0,0 +1,245 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+public class TestResultSetSchemaChange extends SubOperatorTest {
+
+  /**
+   * Test the case where the schema changes in the first batch.
+   * Schema changes before the first record are trivial and tested
+   * elsewhere. Here we write some records, then add new columns, as a
+   * JSON reader might do.
+   */
+
+  @Test
+  public void testSchemaChangeFirstBatch() {
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator());
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    // Create initial rows
+
+    rsLoader.startBatch();
+    int rowCount = 0;
+    for (int i = 0; i < 2;  i++) {
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setString("a_" + rowCount);
+      rootWriter.save();
+    }
+
+    // Add a second column: nullable.
+
+    rootWriter.addColumn(SchemaBuilder.columnSchema("b", MinorType.INT, DataMode.OPTIONAL));
+    for (int i = 0; i < 2;  i++) {
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setString("a_" + rowCount);
+      rootWriter.scalar(1).setInt(rowCount);
+      rootWriter.save();
+    }
+
+    // Add a third column. Use variable-width so that offset
+    // vectors must be back-filled.
+
+    rootWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.OPTIONAL));
+    for (int i = 0; i < 2;  i++) {
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setString("a_" + rowCount);
+      rootWriter.scalar(1).setInt(rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.save();
+    }
+
+    // Fourth: Required Varchar. Previous rows are back-filled with empty strings.
+    // And a required int. Back-filled with zeros.
+    // May occasionally be useful. But, does have to work to prevent
+    // vector corruption if some reader decides to go this route.
+
+    rootWriter.addColumn(SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.REQUIRED));
+    rootWriter.addColumn(SchemaBuilder.columnSchema("e", MinorType.INT,     DataMode.REQUIRED));
+    for (int i = 0; i < 2;  i++) {
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setString("a_" + rowCount);
+      rootWriter.scalar(1).setInt(rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.scalar(3).setString("d_" + rowCount);
+      rootWriter.scalar(4).setInt(rowCount * 10);
+      rootWriter.save();
+    }
+
+    // Add an array. Now two offset vectors must be back-filled.
+
+    rootWriter.addColumn(SchemaBuilder.columnSchema("f", MinorType.VARCHAR, DataMode.REPEATED));
+    for (int i = 0; i < 2;  i++) {
+      rootWriter.start();
+      rowCount++;
+      rootWriter.scalar(0).setString("a_" + rowCount);
+      rootWriter.scalar(1).setInt(rowCount);
+      rootWriter.scalar(2).setString("c_" + rowCount);
+      rootWriter.scalar(3).setString("d_" + rowCount);
+      rootWriter.scalar(4).setInt(rowCount * 10);
+      ScalarWriter arrayWriter = rootWriter.column(5).array().scalar();
+      arrayWriter.setString("f_" + rowCount + "-1");
+      arrayWriter.setString("f_" + rowCount + "-2");
+      rootWriter.save();
+    }
+
+    // Harvest the batch and verify.
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.VARCHAR)
+        .addNullable("b", MinorType.INT)
+        .addNullable("c", MinorType.VARCHAR)
+        .add("d", MinorType.VARCHAR)
+        .add("e", MinorType.INT)
+        .addArray("f", MinorType.VARCHAR)
+        .build();
+    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow("a_1", null, null,   "",       0, new String[] {})
+        .addRow("a_2", null, null,   "",       0, new String[] {})
+        .addRow("a_3",    3, null,   "",       0, new String[] {})
+        .addRow("a_4",    4, null,   "",       0, new String[] {})
+        .addRow("a_5",    5, "c_5",  "",       0, new String[] {})
+        .addRow("a_6",    6, "c_6",  "",       0, new String[] {})
+        .addRow("a_7",    7, "c_7",  "d_7",   70, new String[] {})
+        .addRow("a_8",    8, "c_8",  "d_8",   80, new String[] {})
+        .addRow("a_9",    9, "c_9",  "d_9",   90, new String[] {"f_9-1",  "f_9-2"})
+        .addRow("a_10",  10, "c_10", "d_10", 100, new String[] {"f_10-1", "f_10-2"})
+        .build();
+
+    new RowSetComparison(expected)
+        .verifyAndClearAll(actual);
+    rsLoader.close();
+  }
+
+  /**
+   * Test a schema change on the row that overflows. If the
+   * new column is added after overflow, it will appear as
+   * a schema-change in the following batch. This is fine as
+   * we are essentially time-shifting: pretending that the
+   * overflow row was written in the next batch (which, in
+   * fact, it is: that's what overflow means.)
+   */
+
+  @Test
+  public void testSchemaChangeWithOverflow() {
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rsLoader.startBatch();
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+
+      // Relies on fact that isFull becomes true right after
+      // a vector overflows; don't have to wait for saveRow().
+
+      if (rootWriter.isFull()) {
+        rootWriter.addColumn(SchemaBuilder.columnSchema("b", MinorType.INT, DataMode.OPTIONAL));
+        rootWriter.scalar(1).setInt(count);
+
+        // Add a Varchar to ensure its offset fiddling is done properly
+
+        rootWriter.addColumn(SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.OPTIONAL));
+        rootWriter.scalar(2).setString("c-" + count);
+
+        // Allow adding a required column at this point.
+        // (Not intuitively obvious that this should work; we back-fill
+        // with zeros.)
+
+        rootWriter.addColumn(SchemaBuilder.columnSchema("d", MinorType.INT, DataMode.REQUIRED));
+      }
+      rootWriter.save();
+      count++;
+    }
+
+    // Result should include only the first column.
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.VARCHAR)
+        .build();
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertTrue(result.batchSchema().isEquivalent(expectedSchema));
+    assertEquals(count - 1, result.rowCount());
+    result.clear();
+    assertEquals(1, rsLoader.schemaVersion());
+
+    // Double check: still can add a required column after
+    // starting the next batch. (No longer in overflow state.)
+
+    rsLoader.startBatch();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("e", MinorType.INT, DataMode.REQUIRED));
+
+    // Next batch should start with the overflow row, including
+    // the column added at the end of the previous batch, after
+    // overflow.
+
+    result = fixture.wrap(rsLoader.harvest());
+    assertEquals(5, rsLoader.schemaVersion());
+    assertEquals(1, result.rowCount());
+    expectedSchema = new SchemaBuilder(expectedSchema)
+        .addNullable("b", MinorType.INT)
+        .addNullable("c", MinorType.VARCHAR)
+        .add("d", MinorType.INT)
+        .add("e", MinorType.INT)
+        .build();
+    assertTrue(result.batchSchema().isEquivalent(expectedSchema));
+    RowSetReader reader = result.reader();
+    reader.next();
+    assertEquals(count - 1, reader.scalar(1).getInt());
+    assertEquals("c-" + (count - 1), reader.scalar(2).getString());
+    assertEquals(0, reader.scalar("d").getInt());
+    assertEquals(0, reader.scalar("e").getInt());
+    result.clear();
+
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
new file mode 100644
index 0000000..45c0b55
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestTupleSchema.java
@@ -0,0 +1,509 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.TupleSchema.MapColumnMetadata;
+import org.apache.drill.exec.record.TupleSchema.PrimitiveColumnMetadata;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test the tuple and column metadata, including extended attributes.
+ */
+
+public class TestTupleSchema extends SubOperatorTest {
+
+  /**
+   * Test a fixed-width, primitive, required column. Includes basic
+   * tests common to all data types. (Basic tests are not repeated for
+   * other types.)
+   */
+
+  @Test
+  public void testRequiredFixedWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    // Code may depend on the specific column class
+
+    assertTrue(col instanceof PrimitiveColumnMetadata);
+
+    // Generic checks
+
+    assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
+    assertNull(col.mapSchema());
+    assertSame(field, col.schema());
+    assertEquals(field.getName(), col.name());
+    assertEquals(field.getType(), col.majorType());
+    assertEquals(field.getType().getMinorType(), col.type());
+    assertEquals(field.getDataMode(), col.mode());
+    assertFalse(col.isNullable());
+    assertFalse(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+    assertTrue(col.isEquivalent(col));
+
+    ColumnMetadata col2 = TupleSchema.fromField(field);
+    assertTrue(col.isEquivalent(col2));
+
+    MaterializedField field3 = SchemaBuilder.columnSchema("d", MinorType.INT, DataMode.REQUIRED );
+    ColumnMetadata col3 = TupleSchema.fromField(field3);
+    assertFalse(col.isEquivalent(col3));
+
+    MaterializedField field4 = SchemaBuilder.columnSchema("c", MinorType.BIGINT, DataMode.REQUIRED );
+    ColumnMetadata col4 = TupleSchema.fromField(field4);
+    assertFalse(col.isEquivalent(col4));
+
+    MaterializedField field5 = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL );
+    ColumnMetadata col5 = TupleSchema.fromField(field5);
+    assertFalse(col.isEquivalent(col5));
+
+    ColumnMetadata col6 = col.cloneEmpty();
+    assertTrue(col.isEquivalent(col6));
+
+    assertEquals(4, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(4, col.expectedWidth());
+
+    assertEquals(1, col.expectedElementCount());
+    col.setExpectedElementCount(2);
+    assertEquals(1, col.expectedElementCount());
+  }
+
+  @Test
+  public void testNullableFixedWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.OPTIONAL );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
+    assertTrue(col.isNullable());
+    assertFalse(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(4, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(4, col.expectedWidth());
+
+    assertEquals(1, col.expectedElementCount());
+    col.setExpectedElementCount(2);
+    assertEquals(1, col.expectedElementCount());
+  }
+
+  @Test
+  public void testRepeatedFixedWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REPEATED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertFalse(col.isNullable());
+    assertTrue(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(4, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(4, col.expectedWidth());
+
+    assertEquals(ColumnMetadata.DEFAULT_ARRAY_SIZE, col.expectedElementCount());
+
+    col.setExpectedElementCount(2);
+    assertEquals(2, col.expectedElementCount());
+
+    col.setExpectedElementCount(0);
+    assertEquals(1, col.expectedElementCount());
+  }
+
+  @Test
+  public void testRequiredVariableWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REQUIRED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertEquals(ColumnMetadata.StructureType.PRIMITIVE, col.structureType());
+    assertNull(col.mapSchema());
+    assertFalse(col.isNullable());
+    assertFalse(col.isArray());
+    assertTrue(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+
+    // A different precision is a different type.
+
+    MaterializedField field2 = new SchemaBuilder.ColumnBuilder("c", MinorType.VARCHAR)
+        .setMode(DataMode.REQUIRED)
+        .setPrecision(10)
+        .build();
+
+    ColumnMetadata col2 = TupleSchema.fromField(field2);
+    assertFalse(col.isEquivalent(col2));
+
+    assertEquals(50, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(10, col.expectedWidth());
+
+    assertEquals(1, col.expectedElementCount());
+    col.setExpectedElementCount(2);
+    assertEquals(1, col.expectedElementCount());
+
+    // If precision is provided, then that is the default width
+
+    col = TupleSchema.fromField(field2);
+    assertEquals(10, col.expectedWidth());
+  }
+
+  @Test
+  public void testNullableVariableWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.OPTIONAL );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertTrue(col.isNullable());
+    assertFalse(col.isArray());
+    assertTrue(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(50, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(10, col.expectedWidth());
+
+    assertEquals(1, col.expectedElementCount());
+    col.setExpectedElementCount(2);
+    assertEquals(1, col.expectedElementCount());
+  }
+
+  @Test
+  public void testRepeatedVariableWidthColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.VARCHAR, DataMode.REPEATED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertFalse(col.isNullable());
+    assertTrue(col.isArray());
+    assertTrue(col.isVariableWidth());
+    assertFalse(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(50, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(10, col.expectedWidth());
+
+    assertEquals(ColumnMetadata.DEFAULT_ARRAY_SIZE, col.expectedElementCount());
+
+    col.setExpectedElementCount(2);
+    assertEquals(2, col.expectedElementCount());
+  }
+
+  /**
+   * Tests a map column. Maps can only be required or repeated, not nullable.
+   * (But, the columns in the map can be nullable.)
+   */
+
+  @Test
+  public void testMapColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REQUIRED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertTrue(col instanceof MapColumnMetadata);
+    assertNotNull(col.mapSchema());
+    assertEquals(0, col.mapSchema().size());
+    assertSame(col, col.mapSchema().parent());
+
+    MapColumnMetadata mapCol = (MapColumnMetadata) col;
+    assertNull(mapCol.parentTuple());
+
+    assertEquals(ColumnMetadata.StructureType.TUPLE, col.structureType());
+    assertFalse(col.isNullable());
+    assertFalse(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertTrue(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(0, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(0, col.expectedWidth());
+
+    assertEquals(1, col.expectedElementCount());
+    col.setExpectedElementCount(2);
+    assertEquals(1, col.expectedElementCount());
+  }
+
+  @Test
+  public void testRepeatedMapColumn() {
+
+    MaterializedField field = SchemaBuilder.columnSchema("m", MinorType.MAP, DataMode.REPEATED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+
+    assertTrue(col instanceof MapColumnMetadata);
+    assertNotNull(col.mapSchema());
+    assertEquals(0, col.mapSchema().size());
+
+    assertFalse(col.isNullable());
+    assertTrue(col.isArray());
+    assertFalse(col.isVariableWidth());
+    assertTrue(col.isMap());
+    assertFalse(col.isList());
+
+    assertEquals(0, col.expectedWidth());
+    col.setExpectedWidth(10);
+    assertEquals(0, col.expectedWidth());
+
+    assertEquals(ColumnMetadata.DEFAULT_ARRAY_SIZE, col.expectedElementCount());
+
+    col.setExpectedElementCount(2);
+    assertEquals(2, col.expectedElementCount());
+  }
+
+    // List
+
+    // Repeated list
+
+  /**
+   * Test the basics of an empty root tuple (i.e. row) schema.
+   */
+
+  @Test
+  public void testEmptyRootTuple() {
+
+    TupleMetadata root = new TupleSchema();
+
+    assertEquals(0, root.size());
+    assertTrue(root.isEmpty());
+    assertEquals(-1, root.index("foo"));
+
+    try {
+      root.metadata(0);
+      fail();
+    } catch (IndexOutOfBoundsException e) {
+      // Expected
+    }
+    assertNull(root.metadata("foo"));
+
+    try {
+      root.column(0);
+      fail();
+    } catch (IndexOutOfBoundsException e) {
+      // Expected
+    }
+    assertNull(root.column("foo"));
+
+    try {
+      root.fullName(0);
+      fail();
+    } catch (IndexOutOfBoundsException e) {
+      // Expected
+    }
+
+    // The full name method does not check if the column is actually
+    // in the tuple.
+
+    MaterializedField field = SchemaBuilder.columnSchema("c", MinorType.INT, DataMode.REQUIRED );
+    ColumnMetadata col = TupleSchema.fromField(field);
+    assertEquals("c", root.fullName(col));
+
+    assertTrue(root.isEquivalent(root));
+    assertNull(root.parent());
+    assertTrue(root.toFieldList().isEmpty());
+  }
+
+  /**
+   * Test the basics of a non-empty root tuple (i.e. a row) using a pair
+   * of primitive columns.
+   */
+
+  @Test
+  public void testNonEmptyRootTuple() {
+
+    TupleMetadata root = new TupleSchema();
+
+    MaterializedField fieldA = SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED );
+    ColumnMetadata colA = root.add(fieldA);
+
+    assertEquals(1, root.size());
+    assertFalse(root.isEmpty());
+    assertEquals(0, root.index("a"));
+    assertEquals(-1, root.index("b"));
+
+    assertSame(fieldA, root.column(0));
+    assertSame(fieldA, root.column("a"));
+    assertSame(fieldA, root.column("A"));
+
+    assertSame(colA, root.metadata(0));
+    assertSame(colA, root.metadata("a"));
+
+    assertEquals("a", root.fullName(0));
+    assertEquals("a", root.fullName(colA));
+
+    try {
+      root.add(fieldA);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+
+    MaterializedField fieldB = SchemaBuilder.columnSchema("b", MinorType.VARCHAR, DataMode.OPTIONAL );
+    ColumnMetadata colB = TupleSchema.fromField(fieldB);
+    int indexB = root.addColumn(colB);
+
+    assertEquals(1, indexB);
+    assertEquals(2, root.size());
+    assertFalse(root.isEmpty());
+    assertEquals(indexB, root.index("b"));
+
+    assertSame(fieldB, root.column(1));
+    assertSame(fieldB, root.column("b"));
+
+    assertSame(colB, root.metadata(1));
+    assertSame(colB, root.metadata("b"));
+
+    assertEquals("b", root.fullName(1));
+    assertEquals("b", root.fullName(colB));
+
+    try {
+      root.add(fieldB);
+      fail();
+    } catch (IllegalArgumentException e) {
+      // Expected
+    }
+
+    List<MaterializedField> fieldList = root.toFieldList();
+    assertSame(fieldA, fieldList.get(0));
+    assertSame(fieldB, fieldList.get(1));
+
+    TupleMetadata emptyRoot = new TupleSchema();
+    assertFalse(emptyRoot.isEquivalent(root));
+
+    // Same schema: the tuples are equivalent
+
+    TupleMetadata root3 = new TupleSchema();
+    root3.add(fieldA);
+    root3.addColumn(colB);
+    assertTrue(root3.isEquivalent(root));
+    assertTrue(root.isEquivalent(root3));
+
+    // Same columns, different order. The tuples are not equivalent.
+
+    TupleMetadata root4 = new TupleSchema();
+    root4.addColumn(colB);
+    root4.add(fieldA);
+    assertFalse(root4.isEquivalent(root));
+    assertFalse(root.isEquivalent(root4));
+
+    // A tuple is equivalent to its copy.
+
+    assertTrue(root.isEquivalent(((TupleSchema) root).copy()));
+
+    // And it is equivalent to the round trip to a batch schema.
+
+    BatchSchema batchSchema = ((TupleSchema) root).toBatchSchema(SelectionVectorMode.NONE);
+    assertTrue(root.isEquivalent(TupleSchema.fromFields(batchSchema)));
+  }
+
+  /**
+   * Test a complex map schema of the form:<br>
+   * a.`b.x`.`c.y`.d<br>
+   * in which columns "a", "b.x" and "c.y" are maps, "b.x" and "c.y" are names
+   * that contains dots, and d is primitive.
+   */
+
+  @Test
+  public void testMapTuple() {
+
+    TupleMetadata root = new TupleSchema();
+
+    MaterializedField fieldA = SchemaBuilder.columnSchema("a", MinorType.MAP, DataMode.REQUIRED);
+    ColumnMetadata colA = root.add(fieldA);
+    TupleMetadata mapA = colA.mapSchema();
+
+    MaterializedField fieldB = SchemaBuilder.columnSchema("b.x", MinorType.MAP, DataMode.REQUIRED);
+    ColumnMetadata colB = mapA.add(fieldB);
+    TupleMetadata mapB = colB.mapSchema();
+
+    MaterializedField fieldC = SchemaBuilder.columnSchema("c.y", MinorType.MAP, DataMode.REQUIRED);
+    ColumnMetadata colC = mapB.add(fieldC);
+    TupleMetadata mapC = colC.mapSchema();
+
+    MaterializedField fieldD = SchemaBuilder.columnSchema("d", MinorType.VARCHAR, DataMode.REQUIRED);
+    ColumnMetadata colD = mapC.add(fieldD);
+
+    MaterializedField fieldE = SchemaBuilder.columnSchema("e", MinorType.INT, DataMode.REQUIRED);
+    ColumnMetadata colE = mapC.add(fieldE);
+
+    assertEquals(1, root.size());
+    assertEquals(1, mapA.size());
+    assertEquals(1, mapB.size());
+    assertEquals(2, mapC.size());
+
+    assertSame(colA, root.metadata("a"));
+    assertSame(colB, mapA.metadata("b.x"));
+    assertSame(colC, mapB.metadata("c.y"));
+    assertSame(colD, mapC.metadata("d"));
+    assertSame(colE, mapC.metadata("e"));
+
+    // The full name contains quoted names if the contain dots.
+    // This name is more for diagnostic than semantic purposes.
+
+    assertEquals("a", root.fullName(0));
+    assertEquals("a.`b.x`", mapA.fullName(0));
+    assertEquals("a.`b.x`.`c.y`", mapB.fullName(0));
+    assertEquals("a.`b.x`.`c.y`.d", mapC.fullName(0));
+    assertEquals("a.`b.x`.`c.y`.e", mapC.fullName(1));
+
+    assertEquals(1, colA.schema().getChildren().size());
+    assertEquals(1, colB.schema().getChildren().size());
+    assertEquals(2, colC.schema().getChildren().size());
+
+    // Yes, it is awful that MaterializedField does not provide indexed
+    // access to its children. That's one reason we have the TupleMetadata
+    // classes..
+
+    assertSame(fieldB, colA.schema().getChildren().iterator().next());
+    assertSame(fieldC, colB.schema().getChildren().iterator().next());
+    Iterator<MaterializedField> iterC = colC.schema().getChildren().iterator();
+    assertSame(fieldD, iterC.next());
+    assertSame(fieldE, iterC.next());
+
+    // Copying should be deep.
+
+    TupleMetadata root2 = ((TupleSchema) root).copy();
+    assertEquals(2, root2.metadata(0).mapSchema().metadata(0).mapSchema().metadata(0).mapSchema().size());
+    assert(root.isEquivalent(root2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
deleted file mode 100644
index b17bf18..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/TestVectorContainer.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.record;
-
-import static org.junit.Assert.*;
-
-import org.apache.drill.categories.VectorTest;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.test.DrillTest;
-import org.apache.drill.test.OperatorFixture;
-import org.apache.drill.test.rowSet.RowSet;
-import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
-import org.apache.drill.test.rowSet.RowSetComparison;
-import org.apache.drill.test.rowSet.SchemaBuilder;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(VectorTest.class)
-public class TestVectorContainer extends DrillTest {
-
-  // TODO: Replace the following with an extension of SubOperatorTest class
-  // once that is available.
-
-  protected static OperatorFixture fixture;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    fixture = OperatorFixture.standardFixture();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    fixture.close();
-  }
-
-  /**
-   * Test of the ability to merge two schemas and to merge
-   * two vector containers. The merge is "horizontal", like
-   * a row-by-row join. Since each container is a list of
-   * vectors, we just combine the two lists to create the
-   * merged result.
-   */
-  @Test
-  public void testContainerMerge() {
-
-    // Simulated data from a reader
-
-    BatchSchema leftSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR)
-        .build();
-    SingleRowSet left = fixture.rowSetBuilder(leftSchema)
-        .add(10, "fred")
-        .add(20, "barney")
-        .add(30, "wilma")
-        .build();
-
-    // Simulated "implicit" coumns: row number and file name
-
-    BatchSchema rightSchema = new SchemaBuilder()
-        .add("x", MinorType.SMALLINT)
-        .add("y", MinorType.VARCHAR)
-        .build();
-    SingleRowSet right = fixture.rowSetBuilder(rightSchema)
-        .add(1, "foo.txt")
-        .add(2, "bar.txt")
-        .add(3, "dino.txt")
-        .build();
-
-    // The merge batch we expect to see
-
-    BatchSchema expectedSchema = new SchemaBuilder()
-        .add("a", MinorType.INT)
-        .addNullable("b", MinorType.VARCHAR)
-        .add("x", MinorType.SMALLINT)
-        .add("y", MinorType.VARCHAR)
-        .build();
-    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .add(10, "fred", 1, "foo.txt")
-        .add(20, "barney", 2, "bar.txt")
-        .add(30, "wilma", 3, "dino.txt")
-        .build();
-
-    // Merge containers without selection vector
-
-    RowSet merged = fixture.wrap(
-        left.container().merge(right.container()));
-
-    RowSetComparison comparison = new RowSetComparison(expected);
-    comparison.verify(merged);
-
-    // Merge containers via row set facade
-
-    RowSet mergedRs = left.merge(right);
-    comparison.verifyAndClearAll(mergedRs);
-
-    // Add a selection vector. Merging is forbidden, in the present code,
-    // for batches that have a selection vector.
-
-    SingleRowSet leftIndirect = left.toIndirect();
-    try {
-      leftIndirect.merge(right);
-      fail();
-    } catch (IllegalArgumentException e) {
-      // Expected
-    }
-    leftIndirect.clear();
-    right.clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 0f8f766..621d288 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -251,6 +251,7 @@ public class TestValueVector extends ExecTest {
     final DrillBuf newBuf = allocator.buffer(size);
     final DrillBuf writeBuf = newBuf;
     for(final DrillBuf buffer : buffers) {
+      @SuppressWarnings("resource")
       final DrillBuf readBuf = (DrillBuf) buffer.slice();
       final int nBytes = readBuf.readableBytes();
       final byte[] bytes = new byte[nBytes];
@@ -266,6 +267,7 @@ public class TestValueVector extends ExecTest {
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, RepeatedIntHolder.TYPE);
 
     // Create a new value vector.
+    @SuppressWarnings("resource")
     final RepeatedIntVector vector1 = new RepeatedIntVector(field, allocator);
 
     // Populate the vector.
@@ -321,6 +323,7 @@ the interface to load has changed
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, VarCharHolder.TYPE);
 
     // Create a new value vector for 1024 variable length strings.
+    @SuppressWarnings("resource")
     final VarCharVector vector1 = new VarCharVector(field, allocator);
     final VarCharVector.Mutator mutator = vector1.getMutator();
     vector1.allocateNew(1024 * 10, 1024);
@@ -337,7 +340,9 @@ the interface to load has changed
 
     // Combine the backing buffers so we can load them into a new vector.
     final DrillBuf[] buffers1 = vector1.getBuffers(false);
+    @SuppressWarnings("resource")
     final DrillBuf buffer1 = combineBuffers(allocator, buffers1);
+    @SuppressWarnings("resource")
     final VarCharVector vector2 = new VarCharVector(field, allocator);
     vector2.load(vector1.getMetadata(), buffer1);
 
@@ -360,6 +365,7 @@ the interface to load has changed
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, NullableVarCharHolder.TYPE);
 
     // Create a new value vector for 1024 nullable variable length strings.
+    @SuppressWarnings("resource")
     final NullableVarCharVector vector1 = new NullableVarCharVector(field, allocator);
     final NullableVarCharVector.Mutator mutator = vector1.getMutator();
     vector1.allocateNew(1024 * 10, 1024);
@@ -394,7 +400,9 @@ the interface to load has changed
 
     // Combine into a single buffer so we can load it into a new vector.
     final DrillBuf[] buffers1 = vector1.getBuffers(false);
+    @SuppressWarnings("resource")
     final DrillBuf buffer1 = combineBuffers(allocator, buffers1);
+    @SuppressWarnings("resource")
     final NullableVarCharVector vector2 = new NullableVarCharVector(field, allocator);
     vector2.load(vector1.getMetadata(), buffer1);
 
@@ -673,6 +681,7 @@ the interface to load has changed
       }
 
       for (int i = 0; i < valueVectors.length; i++) {
+        @SuppressWarnings("resource")
         final ValueVector vv = valueVectors[i];
         final int vvCapacity = vv.getValueCapacity();
 
@@ -718,6 +727,7 @@ the interface to load has changed
    *
    * @param test test function to execute
    */
+  @SuppressWarnings("resource")
   private void testVectors(VectorVerifier test) throws Exception {
     final MaterializedField[] fields = {
         MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE),
@@ -777,6 +787,7 @@ the interface to load has changed
 
   @Test
   public void testVectorCanLoadEmptyBuffer() throws Exception {
+    @SuppressWarnings("resource")
     final DrillBuf empty = allocator.getEmpty();
 
     testVectors(new VectorVerifier() {
@@ -798,6 +809,7 @@ the interface to load has changed
     });
   }
 
+  @SuppressWarnings("resource")
   @Test
   public void testListVectorShouldNotThrowOversizedAllocationException() throws Exception {
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH,

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
index a8eef3c..5af0306 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.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/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
index c792233..5ce8e3f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
@@ -103,7 +103,7 @@ public class TestCsv extends ClusterTest {
         .add("c", MinorType.VARCHAR)
         .build();
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
-        .add("10", "foo", "bar")
+        .addRow("10", "foo", "bar")
         .build();
     new RowSetComparison(expected)
       .verifyAndClearAll(actual);
@@ -129,7 +129,7 @@ public class TestCsv extends ClusterTest {
         .add("c_2_2", MinorType.VARCHAR)
         .build();
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
-        .add("10", "foo", "bar", "fourth", "fifth", "sixth")
+        .addRow("10", "foo", "bar", "fourth", "fifth", "sixth")
         .build();
     new RowSetComparison(expected)
       .verifyAndClearAll(actual);
@@ -151,7 +151,7 @@ public class TestCsv extends ClusterTest {
     assertEquals(expectedSchema, actual.batchSchema());
 
     RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
-        .add("10", "foo", "bar")
+        .addRow("10", "foo", "bar")
         .build();
     new RowSetComparison(expected)
       .verifyAndClearAll(actual);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java
index 8366b7a..69667a8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ExampleTest.java
@@ -123,8 +123,8 @@ public class ExampleTest {
         .build();
 
       final RowSet rowSet = new RowSetBuilder(allocator, schema)
-        .add("1", "kiwi")
-        .add("2", "watermelon")
+        .addRow("1", "kiwi")
+        .addRow("2", "watermelon")
         .build();
 
       new JsonFileBuilder(rowSet).build(tableFile);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
index c03f0b7..a1b8af5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
@@ -42,8 +42,12 @@ import org.apache.drill.exec.ops.OperatorStatReceiver;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.testing.ExecutionControls;
@@ -290,21 +294,29 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   }
 
   public RowSetBuilder rowSetBuilder(BatchSchema schema) {
+    return rowSetBuilder(TupleSchema.fromFields(schema));
+  }
+
+  public RowSetBuilder rowSetBuilder(TupleMetadata schema) {
     return new RowSetBuilder(allocator, schema);
   }
 
   public ExtendableRowSet rowSet(BatchSchema schema) {
-    return new DirectRowSet(allocator, schema);
+    return DirectRowSet.fromSchema(allocator, schema);
+  }
+
+  public ExtendableRowSet rowSet(TupleMetadata schema) {
+    return DirectRowSet.fromSchema(allocator, schema);
   }
 
   public RowSet wrap(VectorContainer container) {
     switch (container.getSchema().getSelectionVectorMode()) {
     case FOUR_BYTE:
-      return new HyperRowSetImpl(allocator(), container, container.getSelectionVector4());
+      return new HyperRowSetImpl(container, container.getSelectionVector4());
     case NONE:
-      return new DirectRowSet(allocator(), container);
+      return DirectRowSet.fromContainer(container);
     case TWO_BYTE:
-      return new IndirectRowSet(allocator(), container);
+      return IndirectRowSet.fromSv2(container, container.getSelectionVector2());
     default:
       throw new IllegalStateException( "Unexpected selection mode" );
     }
@@ -342,4 +354,14 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   public OperatorContext operatorContext(PhysicalOperator config) {
     return new TestOperatorContext(context, allocator(), config, stats);
   }
+
+  public RowSet wrap(VectorContainer container, SelectionVector2 sv2) {
+    if (sv2 == null) {
+      assert container.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE;
+      return DirectRowSet.fromContainer(container);
+    } else {
+      assert container.getSchema().getSelectionVectorMode() == SelectionVectorMode.TWO_BYTE;
+      return IndirectRowSet.fromSv2(container, sv2);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 58f888d..2d1aa9b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -55,7 +55,7 @@ import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
 import org.apache.drill.test.ClientFixture.StatementParser;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSetReader;
 
 import com.google.common.base.Preconditions;
 
@@ -338,7 +338,7 @@ public class QueryBuilder {
       dataBatch.release();
       VectorContainer container = loader.getContainer();
       container.setRecordCount(loader.getRecordCount());
-      return new DirectRowSet(client.allocator(), container);
+      return DirectRowSet.fromContainer(container);
     } catch (SchemaChangeException e) {
       throw new IllegalStateException(e);
     }
@@ -364,7 +364,7 @@ public class QueryBuilder {
     }
     RowSetReader reader = rowSet.reader();
     reader.next();
-    long value = reader.column(0).getLong();
+    long value = reader.scalar(0).getLong();
     rowSet.clear();
     return value;
   }
@@ -385,7 +385,7 @@ public class QueryBuilder {
     }
     RowSetReader reader = rowSet.reader();
     reader.next();
-    int value = reader.column(0).getInt();
+    int value = reader.scalar(0).getInt();
     rowSet.clear();
     return value;
   }
@@ -407,10 +407,10 @@ public class QueryBuilder {
     RowSetReader reader = rowSet.reader();
     reader.next();
     String value;
-    if (reader.column(0).isNull()) {
+    if (reader.scalar(0).isNull()) {
       value = null;
     } else {
-      value = reader.column(0).getString();
+      value = reader.scalar(0).getString();
     }
     rowSet.clear();
     return value;

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/QueryRowSetIterator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryRowSetIterator.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryRowSetIterator.java
index c329690..c1b9253 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryRowSetIterator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryRowSetIterator.java
@@ -93,7 +93,7 @@ public class QueryRowSetIterator implements Iterator<DirectRowSet>, Iterable<Dir
       batch = null;
       VectorContainer container = loader.getContainer();
       container.setRecordCount(loader.getRecordCount());
-      return new DirectRowSet(allocator, container);
+      return DirectRowSet.fromContainer(container);
     } catch (SchemaChangeException e) {
       throw new IllegalStateException(e);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
index 6400a5b..d128e4f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/AbstractRowSet.java
@@ -19,12 +19,10 @@ package org.apache.drill.test.rowSet;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnAccessor.RowIndex;
-import org.apache.drill.exec.vector.accessor.impl.AbstractColumnReader;
-import org.apache.drill.exec.vector.accessor.impl.TupleReaderImpl;
 
 /**
  * Basic implementation of a row set for both the single and multiple
@@ -33,119 +31,36 @@ import org.apache.drill.exec.vector.accessor.impl.TupleReaderImpl;
 
 public abstract class AbstractRowSet implements RowSet {
 
-  /**
-   * Row set index base class used when indexing rows within a row
-   * set for a row set reader. Keeps track of the current position,
-   * which starts before the first row, meaning that the client
-   * must call <tt>next()</tt> to advance to the first row.
-   */
-
-  public static abstract class RowSetIndex implements RowIndex {
-    protected int rowIndex = -1;
-
-    public int position() { return rowIndex; }
-    public abstract boolean next();
-    public abstract int size();
-    public abstract boolean valid();
-    public void set(int index) { rowIndex = index; }
-  }
-
-  /**
-   * Bounded (read-only) version of the row set index. When reading,
-   * the row count is fixed, and set here.
-   */
-
-  public static abstract class BoundedRowIndex extends RowSetIndex {
-
-    protected final int rowCount;
-
-    public BoundedRowIndex(int rowCount) {
-      this.rowCount = rowCount;
-    }
-
-    @Override
-    public boolean next() {
-      if (++rowIndex < rowCount ) {
-        return true;
-      } else {
-        rowIndex--;
-        return false;
-      }
-    }
-
-    @Override
-    public int size() { return rowCount; }
-
-    @Override
-    public boolean valid() { return rowIndex < rowCount; }
-  }
-
-  /**
-   * Reader implementation for a row set.
-   */
-
-  public class RowSetReaderImpl extends TupleReaderImpl implements RowSetReader {
-
-    protected final RowSetIndex index;
-
-    public RowSetReaderImpl(TupleSchema schema, RowSetIndex index, AbstractColumnReader[] readers) {
-      super(schema, readers);
-      this.index = index;
-    }
-
-    @Override
-    public boolean next() { return index.next(); }
-
-    @Override
-    public boolean valid() { return index.valid(); }
-
-    @Override
-    public int index() { return index.position(); }
-
-    @Override
-    public int size() { return index.size(); }
-
-    @Override
-    public int rowIndex() { return index.index(); }
-
-    @Override
-    public int batchIndex() { return index.batch(); }
-
-    @Override
-    public void set(int index) { this.index.set(index); }
-  }
-
-  protected final BufferAllocator allocator;
-  protected final RowSetSchema schema;
-  protected final VectorContainer container;
   protected SchemaChangeCallBack callBack = new SchemaChangeCallBack();
+  protected VectorContainer container;
+  protected TupleMetadata schema;
 
-  public AbstractRowSet(BufferAllocator allocator, BatchSchema schema, VectorContainer container) {
-    this.allocator = allocator;
-    this.schema = new RowSetSchema(schema);
+  public AbstractRowSet(VectorContainer container, TupleMetadata schema) {
     this.container = container;
+    this.schema = schema;
   }
 
   @Override
-  public VectorAccessible vectorAccessible() { return container; }
+  public VectorAccessible vectorAccessible() { return container(); }
 
   @Override
   public VectorContainer container() { return container; }
 
   @Override
-  public int rowCount() { return container.getRecordCount(); }
+  public int rowCount() { return container().getRecordCount(); }
 
   @Override
   public void clear() {
+    VectorContainer container = container();
     container.zeroVectors();
     container.setRecordCount(0);
   }
 
   @Override
-  public RowSetSchema schema() { return schema; }
+  public TupleMetadata schema() { return schema; }
 
   @Override
-  public BufferAllocator allocator() { return allocator; }
+  public BufferAllocator allocator() { return container.getAllocator(); }
 
   @Override
   public void print() {
@@ -158,7 +73,5 @@ public abstract class AbstractRowSet implements RowSet {
   }
 
   @Override
-  public BatchSchema batchSchema() {
-    return container.getSchema();
-  }
+  public BatchSchema batchSchema() { return container().getSchema(); }
 }


[04/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
new file mode 100644
index 0000000..7e225c9
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ColumnWriterIndex.java
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+/**
+ * A Drill record batch consists of a variety of vectors, including maps and lists.
+ * Each vector is written independently. A reader may skip some values in each row
+ * if no values appear for those columns.
+ * <p>
+ * This index provides a single view of the "current row" or "current array index"
+ * across a set of vectors. Each writer consults this index to determine:
+ * <ul>
+ * <li>The position to which to write a value.</li>
+ * <li>Whether the write position is beyond the "last write" position, which
+ * would require filling in any "missing" values.</li>
+ * </ul>
+ */
+
+public interface ColumnWriterIndex {
+
+  /**
+   * Index of the first entry for the current row
+   * @return index of the first entry for the current row
+   */
+
+  int rowStartIndex();
+
+  /**
+   * Current row or array index.
+   * @return row or array index
+   */
+
+  int vectorIndex();
+
+  /**
+   * Index for array elements that allows the caller to increment the
+   * index. For arrays, writing (or saving) one value automatically
+   * moves to the next value. Ignored for non-element indexes.
+   */
+
+  void nextElement();
+
+  /**
+   * When handling overflow, the index must be reset so that the current row
+   * starts at the start of the vector. Relative offsets must be preserved.
+   * (That is, if the current write position for an array is four greater than
+   * the start, then that offset must now be reset to four from the start of
+   * the vector.)
+   */
+
+  void rollover();
+
+  /**
+   * If this index represents a repeat level, return the index of the
+   * next higher repeat level.
+   *
+   * @return the outer repeat level index, if any
+   */
+
+  ColumnWriterIndex outerIndex();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
new file mode 100644
index 0000000..9c53e58
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectReader.java
@@ -0,0 +1,60 @@
+/*
+ * 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 reader to get values for value vectors using
+ * a simple, uniform interface modeled after a JSON object.
+ * Every column value is an object of one of three types:
+ * scalar, array or tuple. Methods exist to "cast" this object
+ * to the proper type. This model allows a very simple representation:
+ * tuples (rows, maps) consist of objects. Arrays are lists of
+ * objects.
+ * <p>
+ * {@see ObjectWriter>
+ */
+
+public interface ObjectReader {
+
+  /**
+   * The type of this reader.
+   *
+   * @return type of reader
+   */
+
+  ObjectType type();
+  ScalarReader scalar();
+  ScalarElementReader elements();
+  TupleReader tuple();
+  ArrayReader array();
+
+  /**
+   * Return the value of the underlying data as a Java object.
+   * Primarily for testing
+   * @return Java object that represents the underlying value
+   */
+
+  Object getObject();
+
+  /**
+   * Return the entire object as a string. Primarily for debugging.
+   * @return string representation of the object
+   */
+
+  String getAsString();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
new file mode 100644
index 0000000..e07ea75
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectType.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Type of writer. Follows the JSON-style model, with the
+ * most abstract object types being a scalar (primitive),
+ * tuple (map or row) or an array (repeated type.)
+ */
+
+public enum ObjectType {
+  SCALAR, TUPLE, ARRAY
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
new file mode 100644
index 0000000..a49b0d8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ObjectWriter.java
@@ -0,0 +1,101 @@
+/*
+ * 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.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+
+/**
+ * Represents a column within a tuple. A column can be an array, a scalar or a
+ * tuple. Each has an associated column metadata (schema) and a writer. The
+ * writer is one of three kinds, depending on the kind of the column. If the
+ * column is a map, then the column also has an associated tuple loader to
+ * define and write to the tuple.
+ * <p>
+ * This interface defines a writer to set values for value vectors using a
+ * simple, uniform interface modeled after a JSON object. Every column value is
+ * an object of one of three types: scalar, array or tuple. Methods exist to
+ * "cast" this object to the proper type. This model allows a very simple
+ * representation: tuples (rows, maps) consist of objects. Arrays are lists of
+ * objects.
+ * <p>
+ * Every column resides at an index, is defined by a schema, is backed by a
+ * value vector, and and is written to by a writer. Each column also tracks the
+ * schema version in which it was added to detect schema evolution. Each column
+ * has an optional overflow vector that holds overflow record values when a
+ * batch becomes full.
+ * <p>
+ * {@see ObjectReader}
+ */
+
+public interface ObjectWriter {
+
+  /**
+   * Returns the schema of the column associated with this writer.
+   *
+   * @return schema for this writer's column
+   */
+
+  ColumnMetadata schema();
+
+  /**
+   * Bind a listener to the underlying scalar column, or array of scalar
+   * columns. Not valid if the underlying writer is a map or array of maps.
+   *
+   * @param listener
+   *          the column listener to bind
+   */
+
+  void bindListener(ColumnWriterListener listener);
+
+  /**
+   * Bind a listener to the underlying map or map array column. Not valid if the
+   * underlying writer is a scalar or scalar array.
+   *
+   * @param listener
+   *          the tuple listener to bind
+   */
+
+  void bindListener(TupleWriterListener listener);
+
+  /**
+   * Return the object (structure) type of this writer.
+   *
+   * @return type indicating if this is a scalar, tuple or array
+   */
+
+  ObjectType type();
+
+  ScalarWriter scalar();
+
+  TupleWriter tuple();
+
+  ArrayWriter array();
+
+  /**
+   * For debugging, set the object to the proper form of Java object as defined
+   * by the underlying writer type.
+   *
+   * @param value
+   *          Java object value to write
+   * @throws VectorOverflowException
+   */
+
+  void set(Object value);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
new file mode 100644
index 0000000..d1f31a8
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarElementReader.java
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ * <p>
+ * {@see ScalarWriter}
+ */
+
+public interface ScalarElementReader {
+  /**
+   * 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
+   */
+
+  ValueType valueType();
+  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);
+
+  Object getObject(int index);
+  String getAsString(int index);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
new file mode 100644
index 0000000..e1c26bf
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ScalarReader.java
@@ -0,0 +1,75 @@
+/*
+ * 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.
+ * <p>
+ * {@see ScalarWriter}
+ */
+
+public interface ScalarReader {
+  /**
+   * 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
+   */
+
+  ValueType valueType();
+
+  /**
+   * 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();
+
+  Object getObject();
+  String getAsString();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index 5cbe80a..776dc9c 100644
--- 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
@@ -22,16 +22,81 @@ import java.math.BigDecimal;
 import org.joda.time.Period;
 
 /**
- * Methods common to the {@link ColumnWriter} and
- * {@link ArrayWriter} interfaces.
+ * Represents a scalar value: a required column, a nullable column,
+ * or one element within an array of scalars.
+ * <p>
+ * 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>
+ * {@see ScalarReader}
+ * {@see ScalarElementReader}
  */
 
 public interface ScalarWriter {
+
+  /**
+   * Listener (callback) for vector overflow events. To be optionally
+   * implemented and bound by the client code of the writer. If no
+   * listener is bound, and a vector overflows, then an exception is
+   * thrown.
+   */
+
+  public interface ColumnWriterListener {
+
+    /**
+     * Alert the listener that a vector has overflowed. Upon return,
+     * all writers must have a new set of buffers available, ready
+     * to accept the in-flight value that triggered the overflow.
+     *
+     * @param writer the writer that triggered the overflow
+     */
+
+    void overflowed(ScalarWriter writer);
+
+    /**
+     * A writer wants to expand its vector. Allows the listener to
+     * either allow the growth, or trigger and overflow to limit
+     * batch size.
+     *
+     * @param writer the writer that wishes to grow its vector
+     * @param delta the amount by which the vector is to grow
+     * @return true if the vector can be grown, false if the writer
+     * should instead trigger an overflow by calling
+     * <tt>overflowed()</tt>
+     */
+
+    boolean canExpand(ScalarWriter writer, int delta);
+  }
+
+  void bindListener(ColumnWriterListener listener);
+
+  /**
+   * 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
+   */
+
+  ValueType valueType();
+  void setNull();
   void setInt(int value);
   void setLong(long value);
   void setDouble(double value);
   void setString(String value);
-  void setBytes(byte[] value);
+  void setBytes(byte[] value, int len);
   void setDecimal(BigDecimal value);
   void setPeriod(Period value);
+
+  void setObject(Object value);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index ea9b869..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/TupleAccessor.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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);
-
-    /**
-     * Returns {@code MaterializedField} instance from schema using the name specified in param.
-     *
-     * @param name name of the column in the schema
-     * @return {@code MaterializedField} instance
-     */
-    MaterializedField column(String name);
-
-    /**
-     * Returns index of the column in the schema with name specified in param.
-     *
-     * @param name name of the column in the schema
-     * @return index of the column in the schema
-     */
-    int columnIndex(String name);
-
-    int count();
-  }
-
-  TupleSchema schema();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index acca767..908d6a0 100644
--- 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
@@ -17,24 +17,38 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
+import org.apache.drill.exec.record.TupleMetadata;
+
 /**
  * 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.
+ * <p>
+ * {@see TupleWriter}
  */
 
-public interface TupleReader extends TupleAccessor {
-  ColumnReader column(int colIndex);
+public interface TupleReader {
+  TupleMetadata schema();
+  int columnCount();
+
+  ObjectReader column(int colIndex);
+  ObjectReader column(String colName);
+
+  // Convenience methods
+
+  ObjectType type(int colIndex);
+  ObjectType type(String colName);
+  ScalarReader scalar(int colIndex);
+  ScalarReader scalar(String colName);
+  TupleReader tuple(int colIndex);
+  TupleReader tuple(String colName);
+  ArrayReader array(int colIndex);
+  ArrayReader array(String colName);
+  ScalarElementReader elements(int colIndex);
+  ScalarElementReader elements(String colName);
 
-  /**
-   * Returns column reader for the column with name specified in param.
-   *
-   * @param colName name of the column in the schema
-   * @return column reader
-   */
-  ColumnReader column(String colName);
-  Object get(int colIndex);
-  String getAsString(int colIndex);
+  Object getObject();
+  String getAsString();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
index 563734e..056c9b3 100644
--- 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
@@ -17,25 +17,153 @@
  */
 package org.apache.drill.exec.vector.accessor;
 
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+
 /**
- * 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.)
+ * Writer for a tuple. A tuple is composed of columns with a fixed order and
+ * unique names: either can be used to reference columns. Columns are scalar
+ * (simple values), tuples (i.e. maps), or arrays (of scalars, tuples or
+ * arrays.) The row itself is just the top-level (anonymous) tuple. Generally,
+ * implementers of this interface provide additional services on the
+ * implementation of the top-level tuple (often called a "row writer.") Columns
+ * are accessible via the associated column writer by name or index. Column
+ * indexes are defined by the tuple schema.
+ * <p>
+ * Consumers of this interface can define the schema up front, or can define the
+ * schema as the write progresses. To avoid redundant checks to see if a column
+ * is already defined, consumers can simply ask for a column by name. The
+ * <tt>column()</tt> (and related) methods will throw an (unchecked)
+ * {@link UndefinedColumnException} exception if the column is undefined. The
+ * consumer can catch the exception, define the column, and fetch the column
+ * writer again. New columns may be added via this interface at any time; the
+ * new column takes the next available index.
+ * <p>
+ * 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.)
+ * <p>
+ * Convenience methods allow getting a column as a scalar, tuple or array. These
+ * methods throw an exception if the column is not of the requested type.
+ *
+ * @see {@link SingleMapWriter}, the class which this class replaces
  */
 
-public interface TupleWriter extends TupleAccessor {
-  ColumnWriter column(int colIndex);
+public interface TupleWriter {
+
+  /**
+   * Listener (callback) to handle requests to add a new column to a tuple (row
+   * or map). Implemented and bound by the client code that creates or uses the
+   * tuple writer. If no listener is bound, then an attempt to add a column
+   * throws an exception.
+   */
+
+  public interface TupleWriterListener {
+    ObjectWriter addColumn(TupleWriter tuple, ColumnMetadata column);
+
+    ObjectWriter addColumn(TupleWriter tuple, MaterializedField field);
+  }
+
+  /**
+   * Unchecked exception thrown when attempting to access a column writer by
+   * name for an undefined columns. Clients that use a fixed schema can simply
+   * omit catch blocks for the exception since it is unchecked and won't be
+   * thrown if the schema can't evolve. Clients that can discover new columns
+   * should catch the exception and define the column (using an implementation
+   * that allows dynamic schema definition.)
+   */
+
+  @SuppressWarnings("serial")
+  public static class UndefinedColumnException extends RuntimeException {
+    public UndefinedColumnException(String colName) {
+      super("Undefined column: " + colName);
+    }
+  }
+
+  void bindListener(TupleWriterListener listener);
 
   /**
-   * Returns column writer for the column with name specified in param.
+   * Add a column to the tuple (row or map) that backs this writer. Support for
+   * this operation depends on whether the client code has registered a listener
+   * to implement the addition. Throws an exception if no listener is
+   * implemented, or if the add request is otherwise invalid (duplicate name,
+   * etc.)
    *
-   * @param colName name of the column in the schema
-   * @return column writer
+   * @param column
+   *          the metadata for the column to add
+   * @return the index of the newly added column which can be used to access the
+   *         newly added writer
    */
-  ColumnWriter column(String colName);
+
+  int addColumn(ColumnMetadata column);
+
+  int addColumn(MaterializedField schema);
+
+  TupleMetadata schema();
+
+  int size();
+
+  // Return the column as a generic object
+
+  ObjectWriter column(int colIndex);
+
+  ObjectWriter column(String colName);
+
+  // Convenience methods
+
+  ScalarWriter scalar(int colIndex);
+
+  ScalarWriter scalar(String colName);
+
+  TupleWriter tuple(int colIndex);
+
+  TupleWriter tuple(String colName);
+
+  ArrayWriter array(int colIndex);
+
+  ArrayWriter array(String colName);
+
+  ObjectType type(int colIndex);
+
+  ObjectType type(String colName);
+
+  /**
+   * Write a value to the given column, automatically calling the proper
+   * <tt>set<i>Type</i></tt> method for the data. While this method is
+   * convenient for testing, it incurs quite a bit of type-checking overhead and
+   * is not suitable for production code.
+   *
+   * @param colIndex
+   *          the index of the column to set
+   * @param value
+   *          the value to set. The type of the object must be compatible with
+   *          the type of the target column
+   */
+
   void set(int colIndex, Object value);
+
+  /**
+   * Write a row or map of values, given by Java objects. Object type must match
+   * expected column type.
+   * <p>
+   * Note that a single-column tuple is ambiguous if that column is an array. To
+   * avoid ambiguity, use <tt>set(0, value)</tt> in this case.
+   *
+   * @param values
+   *          variable-length argument list of column values
+   * @return true if the row was written, false if any column caused vector
+   *         overflow.
+   */
+
+  void setTuple(Object... values);
+
+  /**
+   * Set the tuple from an array of objects. Primarily for use in test tools.
+   *
+   * @param value
+   *          the object to set, which must be a generic <tt>Object</tt> array
+   */
+
+  void setObject(Object value);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java
new file mode 100644
index 0000000..e6687dc
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/ValueType.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+/**
+ * Represents the primitive types supported to read and write data
+ * from value vectors. Vectors support many data widths. For simplicity
+ * (and because of no difference in performance), the get/set methods
+ * use a reduced set of types. In general, each reader and writer
+ * supports just one type. Though some may provide more than one
+ * (such as access to bytes for a <tt>STRING</tt> value.)
+ */
+
+public enum ValueType {
+  INTEGER, LONG, DOUBLE, STRING, BYTES, DECIMAL, PERIOD
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index deea7f8..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayReader.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
deleted file mode 100644
index d1d1263..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractArrayWriter.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
deleted file mode 100644
index 5b751c5..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnAccessor.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
deleted file mode 100644
index b88b08b..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnReader.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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 Object getObject() {
-    switch (valueType()) {
-    case ARRAY:
-      // TODO: build an array. Just a bit tedious...
-      throw new UnsupportedOperationException();
-    case BYTES:
-      return getBytes();
-    case DECIMAL:
-      return getDecimal();
-    case DOUBLE:
-      return getDouble();
-    case INTEGER:
-      return getInt();
-    case LONG:
-      return getLong();
-    case MAP:
-      // TODO: build an array. Just a bit tedious...
-      throw new UnsupportedOperationException();
-    case PERIOD:
-      return getPeriod();
-    case STRING:
-      return getString();
-    default:
-      throw new IllegalStateException("Unexpected type: " + valueType());
-    }
-  }
-
-  @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/40de8ca4/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
deleted file mode 100644
index 5071e03..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractColumnWriter.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/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
deleted file mode 100644
index 98ea6ac..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AbstractTupleAccessor.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
new file mode 100644
index 0000000..4bf8f98
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/AccessorUtilities.java
@@ -0,0 +1,53 @@
+/*
+ * 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;
+
+public class AccessorUtilities {
+
+  private AccessorUtilities() { }
+
+  public static int sv4Batch(int sv4Index) {
+    return sv4Index >>> 16;
+  }
+
+  public static int sv4Index(int sv4Index) {
+    return sv4Index & 0xFFFF;
+  }
+
+  public static String bytesToString(byte[] value) {
+    StringBuilder buf = new StringBuilder()
+        .append("[");
+    int len = Math.min(value.length, 20);
+    for (int i = 0; i < len;  i++) {
+      if (i > 0) {
+        buf.append(", ");
+      }
+      String str = Integer.toHexString(value[i] & 0xFF);
+      if (str.length() < 2) {
+        buf.append("0");
+      }
+      buf.append(str);
+    }
+    if (value.length > len) {
+      buf.append("...");
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 019d3be..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/ColumnAccessorFactory.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.java
new file mode 100644
index 0000000..35bd445
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalFormatter.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;
+
+/**
+ * Interface for dumping object state in a hierarchical fashion during
+ * debugging.
+ */
+
+public interface HierarchicalFormatter {
+
+  HierarchicalFormatter startObject(Object obj);
+  void extend();
+  HierarchicalFormatter attribute(String label);
+  HierarchicalFormatter attribute(String label, Object value);
+  HierarchicalFormatter attributeIdentity(String label, Object value);
+  HierarchicalFormatter attributeArray(String label);
+  HierarchicalFormatter element(int index, Object value);
+  HierarchicalFormatter elementIdentity(int index, Object value);
+  HierarchicalFormatter element(int index);
+  HierarchicalFormatter endArray();
+  HierarchicalFormatter endObject();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
new file mode 100644
index 0000000..0c37d23
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/HierarchicalPrinter.java
@@ -0,0 +1,238 @@
+/*
+ * 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.io.PrintStream;
+import java.util.ArrayDeque;
+import java.util.Deque;
+
+/**
+ * Prints a complex object structure in a quasi-JSON format for use
+ * in debugging. Generally only used in an ad-hoc fashion during
+ * debugging sessions; never in production.
+ */
+
+public class HierarchicalPrinter implements HierarchicalFormatter {
+
+  public enum State { OBJECT, OBJECT_ATTRIB, ARRAY, OBJECT_ELEMENT }
+
+  private static class ObjState {
+    private State state = State.OBJECT;
+    private String attrib;
+    private int index = -1;
+    private int extensions;
+
+    public ObjState(int extensions) {
+      this.extensions = extensions;
+    }
+  }
+
+  private final PrintStream out;
+  private Deque<ObjState> stack = new ArrayDeque<ObjState>();
+  private int pendingExtensions = 0;
+  private ObjState curObject;
+  private int level;
+
+  public HierarchicalPrinter() {
+    out = System.out;
+  }
+
+  @Override
+  public void extend() {
+    pendingExtensions++;
+  }
+
+  @Override
+  public HierarchicalFormatter startObject(Object obj) {
+    if (curObject != null) {
+      stack.push(curObject);
+      switch (curObject.state) {
+      case OBJECT_ATTRIB:
+        startAttrib(curObject.attrib);
+        curObject.attrib = null;
+        curObject.state = State.OBJECT;
+        break;
+      case OBJECT:
+        startAttrib("missing-attrib");
+        curObject.state = State.OBJECT;
+        break;
+      case OBJECT_ELEMENT:
+        startElement(curObject.index);
+        curObject.state = State.ARRAY;
+        curObject.index = -1;
+        break;
+      default:
+        assert false;
+      }
+    }
+
+    printObjIdentity(obj);
+    out.println(" {");
+    level++;
+    curObject = new ObjState(pendingExtensions);
+    pendingExtensions = 0;
+    return this;
+  }
+
+  private void printObjIdentity(Object value) {
+    out.print(value.getClass().getSimpleName());
+    out.print( " (");
+    out.print(System.identityHashCode(value) % 1000);
+    out.print(")");
+  }
+
+  private void startElement(int index) {
+    indent();
+    out.print("[");
+    out.print(index);
+    out.print("] = ");
+  }
+
+  private void startAttrib(String label) {
+    indent();
+    out.print(label);
+    out.print(" = ");
+  }
+
+  @Override
+  public HierarchicalFormatter attribute(String label) {
+    curObject.attrib = label;
+    curObject.state = State.OBJECT_ATTRIB;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter attribute(String label, Object value) {
+    attribPrefix();
+    startAttrib(label);
+    printValue(value);
+    out.println();
+    return this;
+  }
+
+  private void attribPrefix() {
+    switch (curObject.state) {
+    case OBJECT_ATTRIB:
+      startAttrib(curObject.attrib);
+      out.println("<Unknown> {}");
+      break;
+    case OBJECT:
+      break;
+    default:
+      assert false;
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter attributeIdentity(String label, Object obj) {
+    attribPrefix();
+    startAttrib(label);
+    objIdentity(obj);
+    out.println();
+    return this;
+  }
+
+  private void objIdentity(Object obj) {
+    if (obj == null) {
+      out.print("null");
+    } else {
+      printObjIdentity(obj);
+    }
+  }
+
+  private void printValue(Object value) {
+    if (value == null) {
+      out.print("null");
+    } else if (value instanceof String) {
+      out.print("\"");
+      out.print(value);
+      out.print("\"");
+    } else {
+      out.print(value.toString());
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter endObject() {
+    if (level == 0) {
+      out.println( "} // Mismatch!");
+      return this;
+    }
+    if (curObject.extensions == 0) {
+      level--;
+      indent();
+      out.println("}");
+      if (level == 0) {
+        curObject = null;
+      } else {
+        curObject = stack.pop();
+      }
+    } else {
+      curObject.extensions--;
+    }
+    return this;
+  }
+
+  private void indent() {
+    for (int i = 0; i < level; i++) {
+      out.print("  ");
+    }
+  }
+
+  @Override
+  public HierarchicalFormatter attributeArray(String label) {
+    startAttrib(label);
+    out.println("[");
+    level++;
+    curObject.state = State.ARRAY;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter element(int index, Object value) {
+    startElement(index);
+    printValue(value);
+    out.println();
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter element(int index) {
+    curObject.index = index;
+    curObject.state = State.OBJECT_ELEMENT;
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter elementIdentity(int index, Object obj) {
+    startElement(index);
+    objIdentity(obj);
+    out.println();
+    return this;
+  }
+
+  @Override
+  public HierarchicalFormatter endArray() {
+    level--;
+    indent();
+    out.println("]");
+    curObject.state = State.OBJECT;
+    return this;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 97a6e3c..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleReaderImpl.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.vector.accessor.impl;
-
-import org.apache.drill.exec.vector.accessor.ArrayReader;
-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:
-      return bytesToString(colReader.getBytes());
-    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();
-    case ARRAY:
-      return getArrayAsString(colReader.array());
-    default:
-      throw new IllegalArgumentException("Unsupported type " + colReader.valueType());
-    }
-  }
-
-  private String bytesToString(byte[] value) {
-    StringBuilder buf = new StringBuilder()
-        .append("[");
-    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();
-  }
-
-  private String getArrayAsString(ArrayReader array) {
-    StringBuilder buf = new StringBuilder();
-    buf.append("[");
-    for (int i = 0; i < array.size(); i++) {
-      if (i > 0) {
-        buf.append( ", " );
-      }
-      switch (array.valueType()) {
-      case BYTES:
-        buf.append(bytesToString(array.getBytes(i)));
-        break;
-      case DOUBLE:
-        buf.append(Double.toString(array.getDouble(i)));
-        break;
-      case INTEGER:
-        buf.append(Integer.toString(array.getInt(i)));
-        break;
-      case LONG:
-        buf.append(Long.toString(array.getLong(i)));
-        break;
-      case STRING:
-        buf.append("\"" + array.getString(i) + "\"");
-        break;
-      case DECIMAL:
-        buf.append(array.getDecimal(i).toPlainString());
-        break;
-      case MAP:
-      case ARRAY:
-        throw new UnsupportedOperationException("Unsupported type " + array.valueType());
-      default:
-        throw new IllegalArgumentException("Unexpected type " + array.valueType());
-      }
-    }
-    buf.append("]");
-    return buf.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/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
deleted file mode 100644
index 015b099..0000000
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/impl/TupleWriterImpl.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.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 );
-    }
-  }
-}


[07/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestOffsetVectorWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestOffsetVectorWriter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestOffsetVectorWriter.java
new file mode 100644
index 0000000..82d4d08
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestOffsetVectorWriter.java
@@ -0,0 +1,425 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.exec.vector.accessor.writer.OffsetVectorWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.drill.test.rowSet.test.TestFixedWidthWriter.TestIndex;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * The offset vector writer is unique: it follows the same API as
+ * the other writers, but has a unique twist because offsets are written
+ * into the slot one after the other vectors. That is, if we are writing
+ * row 5, the offset vector writer writes to position 6. This is done to
+ * write the end offset of row 5 as the start offset of row 6. (It does,
+ * however, waste space as we need twice the number of elements in the
+ * offset vector as other vectors when writing power-of-two record
+ * counts.)
+ */
+
+public class TestOffsetVectorWriter extends SubOperatorTest {
+
+  /**
+   * Party over enough memory that the uninitialized nature of
+   * vectors under the new writers will cause test to fail if
+   * the writer's don't correctly fill in all values.
+   */
+
+  @BeforeClass
+  public static void setup() {
+    DrillBuf bufs[] = new DrillBuf[100];
+    for (int i = 0; i < bufs.length; i++) {
+      bufs[i] = fixture.allocator().buffer(ValueVector.MAX_BUFFER_SIZE);
+      for (int j = 0; j < ValueVector.MAX_BUFFER_SIZE; j++) {
+        bufs[i].setByte(j, (byte) (j & 0x7f));
+      }
+    }
+    for (int i = 0; i < bufs.length; i++) {
+      bufs[i].close();
+    }
+  }
+
+  /**
+   * Basic test to write a contiguous set of offsets, enough to cause
+   * the vector to double in size twice, then read back the values.
+   */
+
+  @Test
+  public void testWrite() {
+    try (UInt4Vector vector = allocVector(1000)) {
+
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+
+      // Start write sets initial position to 0.
+
+      writer.startWrite();
+      assertEquals(0, vector.getAccessor().get(0));
+
+      // Pretend to write offsets for values of width 10. We write
+      // the end position of each field.
+      // Write enough that the vector is resized.
+
+      long origAddr = vector.getBuffer().addr();
+      for (int i = 0; i < 3000; i++) {
+        index.index = i;
+        writer.startRow();
+        assertEquals(i * 10, writer.nextOffset());
+        writer.setNextOffset((i+1) * 10);
+        assertEquals((i+1) * 10, writer.nextOffset());
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Should have been reallocated.
+
+      assertNotEquals(origAddr, vector.getBuffer().addr());
+
+      // Verify values
+
+      for (int i = 0; i < 3001; i++) {
+        assertEquals(i * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  @Test
+  public void testRestartRow() {
+    try (UInt4Vector vector = allocVector(1000)) {
+
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write rows, rewriting every other row.
+
+      writer.startRow();
+      index.index = 0;
+      for (int i = 0; i < 50; i++) {
+        if (i % 2 == 0) {
+          assertEquals(i == 0 ? 0 : (i - 1) * 10, writer.nextOffset());
+          writer.setNextOffset((i + 1) * 10);
+          writer.saveRow();
+          writer.startRow();
+          index.index++;
+        } else {
+          writer.setNextOffset((i + 1) * 10);
+          writer.restartRow();
+        }
+      }
+      writer.endWrite();
+
+      // Verify values
+
+      assertEquals(0, vector.getAccessor().get(0));
+      for (int i = 1; i < 25; i++) {
+        assertEquals((2 * i - 1) * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+
+  /**
+   * Offset vectors have specific behavior when back-filling missing values:
+   * the last offset must be carried forward into the missing slots. The
+   * slots cannot be zero-filled, or entries will end up with a negative
+   * length.
+   */
+
+  @Test
+  public void testFillEmpties() {
+    try (UInt4Vector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Pretend to write offsets for values of width 10, but
+      // skip four out of five values, forcing backfill.
+      // The loop will cause the vector to double in size.
+      // The number of values is odd, forcing the writer to
+      // back-fill at the end as well as between values.
+
+      long origAddr = vector.getBuffer().addr();
+      for (int i = 5; i < 3001; i += 5) {
+        index.index = i;
+        writer.startRow();
+        int startOffset = writer.nextOffset();
+        assertEquals((i/5 - 1) * 10, startOffset);
+        writer.setNextOffset(startOffset + 10);
+        writer.saveRow();
+      }
+      index.index = 3003;
+      writer.endWrite();
+
+      // Should have been reallocated.
+
+      assertNotEquals(origAddr, vector.getBuffer().addr());
+
+      // Verify values
+
+      for (int i = 0; i < 3004; i++) {
+        assertEquals(((i-1)/5) * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * The rollover method is used during vector overflow.
+   */
+
+  @Test
+  public void testRollover() {
+    try (UInt4Vector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of ten values.
+
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setNextOffset((i+1) * 10);
+        writer.saveRow();
+      }
+
+      // Overflow occurs after writing the 11th row
+
+      index.index = 10;
+      writer.startRow();
+      writer.setNextOffset(110);
+
+      // Overflow occurs
+
+      writer.preRollover();
+
+      // Simulate rollover
+
+      for (int i = 0; i < 15; i++) {
+        vector.getMutator().set(i, 0xdeadbeef);
+      }
+
+      // Simulate shifting the last value down (which changes
+      // the offset.)
+
+      vector.getMutator().set(1, 10);
+
+      // Post rollover, slot 0 should be initialized
+
+      writer.postRollover();
+      index.index = 0;
+      writer.saveRow();
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setNextOffset((i + 1) * 10);
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 6; i++) {
+        assertEquals(i * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Simulate the case in which the tail end of an overflow
+   * batch has empties. <tt>preRollover()</tt> should back-fill
+   * them with the next offset prior to rollover.
+   */
+
+  @Test
+  public void testRolloverWithEmpties() {
+    try (UInt4Vector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of 15 values,
+      // of which 5 are empty.
+
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setNextOffset((i+1) * 10);
+        writer.saveRow();
+      }
+
+      for (int i = 10; i < 15; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Overflow occurs before writing the 16th row
+
+      index.index = 15;
+      writer.startRow();
+
+      // Overflow occurs. This should fill empty offsets.
+
+      writer.preRollover();
+
+      // Verify the first "batch" results
+
+      for (int i = 0; i < 11; i++) {
+        assertEquals(i * 10, vector.getAccessor().get(i));
+      }
+      for (int i = 11; i < 16; i++) {
+        assertEquals("i = " + i, 100, vector.getAccessor().get(i));
+      }
+
+      // Simulate rollover
+
+      for (int i = 0; i < 20; i++) {
+        vector.getMutator().set(i, 0xdeadbeef);
+      }
+
+      // Post rollover, slot 0 should be initialized.
+      // This is a rollover. This row must set the value
+      // for the new row 0 (which was presumably set/filled
+      // after the overflow.)
+
+      writer.postRollover();
+      index.index = 0;
+      writer.setNextOffset(0);
+      writer.saveRow();
+
+      // Skip more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 5; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setNextOffset((i - 4) * 10);
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 6; i++) {
+        assertEquals(0, vector.getAccessor().get(i));
+      }
+      for (int i = 6; i < 11; i++) {
+        assertEquals((i - 5) * 10, vector.getAccessor().get(i));
+      }
+    }
+  }
+
+  /**
+   * Test resize monitoring. Add a listener to an offsets writer,
+   * capture each resize, and refuse a resize when the number
+   * of ints exceeds 8K values. This will trigger an overflow,
+   * which will throw an exception which we then check for.
+   */
+
+  @Test
+  public void testSizeLimit() {
+    try (UInt4Vector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      OffsetVectorWriter writer = makeWriter(vector, index);
+      writer.bindListener(new ColumnWriterListener() {
+        int totalAlloc = 4096;
+
+        @Override
+        public void overflowed(ScalarWriter writer) {
+          throw new IllegalStateException("overflow called");
+        }
+
+        @Override
+        public boolean canExpand(ScalarWriter writer, int delta) {
+//          System.out.println("Delta: " + delta);
+          totalAlloc += delta;
+          return totalAlloc < 16_384 * 4;
+        }
+      });
+      writer.startWrite();
+      try {
+        for (int i = 0; ; i++ ) {
+          index.index = i;
+          writer.startRow();
+          writer.setNextOffset(i);
+          writer.saveRow();
+        }
+      }
+      catch(IllegalStateException e) {
+        assertTrue(e.getMessage().contains("overflow called"));
+      }
+
+      // Should have failed on 8191, which doubled vector
+      // to 16K, which was rejected. Note the 8191 value,
+      // because offsets are one ahead of the index.
+
+      assertEquals(8191, index.index);
+    }
+  }
+
+  private UInt4Vector allocVector(int size) {
+    MaterializedField field = SchemaBuilder.columnSchema("x", MinorType.UINT4,
+        DataMode.REQUIRED);
+    UInt4Vector vector = new UInt4Vector(field, fixture.allocator());
+    vector.allocateNew(size);
+
+    // Party on the bytes of the vector so we start dirty
+
+    for (int i = 0; i < size; i++) {
+      vector.getMutator().set(i, 0xdeadbeef);
+    }
+    assertNotEquals(0, vector.getAccessor().get(0));
+    return vector;
+  }
+
+  private OffsetVectorWriter makeWriter(UInt4Vector vector, TestIndex index) {
+    OffsetVectorWriter writer = new OffsetVectorWriter(vector);
+    writer.bindIndex(index);
+
+    assertEquals(ValueType.INTEGER, writer.valueType());
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestScalarAccessors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestScalarAccessors.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestScalarAccessors.java
new file mode 100644
index 0000000..939377a
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestScalarAccessors.java
@@ -0,0 +1,1266 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+
+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.record.BatchSchema;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.joda.time.Period;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.junit.Test;
+
+/**
+ * Verify that simple scalar (non-repeated) column readers
+ * and writers work as expected. The focus is on the generated
+ * and type-specific functions for each type.
+ */
+
+// The following types are not fully supported in Drill
+// TODO: Var16Char
+// TODO: Bit
+// TODO: Decimal28Sparse
+// TODO: Decimal38Sparse
+
+public class TestScalarAccessors extends SubOperatorTest {
+
+  @Test
+  public void testTinyIntRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.TINYINT)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0)
+        .addRow(Byte.MAX_VALUE)
+        .addRow(Byte.MIN_VALUE)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.INTEGER, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, colReader.getInt());
+
+    assertTrue(reader.next());
+    assertEquals(Byte.MAX_VALUE, colReader.getInt());
+    assertEquals((int) Byte.MAX_VALUE, colReader.getObject());
+    assertEquals(Byte.toString(Byte.MAX_VALUE), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(Byte.MIN_VALUE, colReader.getInt());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  private void nullableIntTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(10)
+        .addSingleCol(null)
+        .addRow(30)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(10, colReader.getInt());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+    // Data value is undefined, may be garbage
+
+    assertTrue(reader.next());
+    assertEquals(30, colReader.getInt());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableTinyInt() {
+    nullableIntTester(MinorType.TINYINT);
+  }
+
+  private void intArrayTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new int[] {})
+        .addSingleCol(new int[] {0, 20, 30})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.INTEGER, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(0, colReader.getInt(0));
+    assertEquals(20, colReader.getInt(1));
+    assertEquals(30, colReader.getInt(2));
+    assertEquals(0, colReader.getObject(0));
+    assertEquals(20, colReader.getObject(1));
+    assertEquals(30, colReader.getObject(2));
+    assertEquals("0", colReader.getAsString(0));
+    assertEquals("20", colReader.getAsString(1));
+    assertEquals("30", colReader.getAsString(2));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testTinyIntArray() {
+    intArrayTester(MinorType.TINYINT);
+  }
+
+  @Test
+  public void testSmallIntRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.SMALLINT)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0)
+        .addRow(Short.MAX_VALUE)
+        .addRow(Short.MIN_VALUE)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.INTEGER, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, colReader.getInt());
+
+    assertTrue(reader.next());
+    assertEquals(Short.MAX_VALUE, colReader.getInt());
+    assertEquals((int) Short.MAX_VALUE, colReader.getObject());
+    assertEquals(Short.toString(Short.MAX_VALUE), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(Short.MIN_VALUE, colReader.getInt());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableSmallInt() {
+    nullableIntTester(MinorType.SMALLINT);
+  }
+
+  @Test
+  public void testSmallArray() {
+    intArrayTester(MinorType.SMALLINT);
+  }
+
+  @Test
+  public void testIntRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.INT)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0)
+        .addRow(Integer.MAX_VALUE)
+        .addRow(Integer.MIN_VALUE)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.INTEGER, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, reader.scalar(0).getInt());
+
+    assertTrue(reader.next());
+    assertEquals(Integer.MAX_VALUE, colReader.getInt());
+    assertEquals(Integer.MAX_VALUE, colReader.getObject());
+    assertEquals(Integer.toString(Integer.MAX_VALUE), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(Integer.MIN_VALUE, colReader.getInt());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableInt() {
+    nullableIntTester(MinorType.INT);
+  }
+
+  @Test
+  public void testIntArray() {
+    intArrayTester(MinorType.INT);
+  }
+
+  private void longRWTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0L)
+        .addRow(Long.MAX_VALUE)
+        .addRow(Long.MIN_VALUE)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.LONG, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, colReader.getLong());
+
+    assertTrue(reader.next());
+    assertEquals(Long.MAX_VALUE, colReader.getLong());
+    assertEquals(Long.MAX_VALUE, colReader.getObject());
+    assertEquals(Long.toString(Long.MAX_VALUE), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(Long.MIN_VALUE, colReader.getLong());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testLongRW() {
+    longRWTester(MinorType.BIGINT);
+  }
+
+  private void nullableLongTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(10L)
+        .addSingleCol(null)
+        .addRow(30L)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(10, colReader.getLong());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+    // Data value is undefined, may be garbage
+
+    assertTrue(reader.next());
+    assertEquals(30, colReader.getLong());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableLong() {
+    nullableLongTester(MinorType.BIGINT);
+  }
+
+  private void longArrayTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new long[] {})
+        .addSingleCol(new long[] {0, 20, 30})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.LONG, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(0, colReader.getLong(0));
+    assertEquals(20, colReader.getLong(1));
+    assertEquals(30, colReader.getLong(2));
+    assertEquals(0L, colReader.getObject(0));
+    assertEquals(20L, colReader.getObject(1));
+    assertEquals(30L, colReader.getObject(2));
+    assertEquals("0", colReader.getAsString(0));
+    assertEquals("20", colReader.getAsString(1));
+    assertEquals("30", colReader.getAsString(2));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testLongArray() {
+    longArrayTester(MinorType.BIGINT);
+  }
+
+  @Test
+  public void testFloatRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.FLOAT4)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0F)
+        .addRow(Float.MAX_VALUE)
+        .addRow(Float.MIN_VALUE)
+        .addRow(100F)
+        .build();
+    assertEquals(4, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.DOUBLE, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, colReader.getDouble(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(Float.MAX_VALUE, colReader.getDouble(), 0.000001);
+    assertEquals((double) Float.MAX_VALUE, (double) colReader.getObject(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(Float.MIN_VALUE, colReader.getDouble(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(100, colReader.getDouble(), 0.000001);
+    assertEquals("100.0", colReader.getAsString());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  private void nullableDoubleTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(10F)
+        .addSingleCol(null)
+        .addRow(30F)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(10, colReader.getDouble(), 0.000001);
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+    // Data value is undefined, may be garbage
+
+    assertTrue(reader.next());
+    assertEquals(30, colReader.getDouble(), 0.000001);
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableFloat() {
+    nullableDoubleTester(MinorType.FLOAT4);
+  }
+
+  private void doubleArrayTester(MinorType type) {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", type)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new double[] {})
+        .addSingleCol(new double[] {0, 20.5, 30.0})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.DOUBLE, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(0, colReader.getDouble(0), 0.00001);
+    assertEquals(20.5, colReader.getDouble(1), 0.00001);
+    assertEquals(30.0, colReader.getDouble(2), 0.00001);
+    assertEquals(0, (double) colReader.getObject(0), 0.00001);
+    assertEquals(20.5, (double) colReader.getObject(1), 0.00001);
+    assertEquals(30.0, (double) colReader.getObject(2), 0.00001);
+    assertEquals("0.0", colReader.getAsString(0));
+    assertEquals("20.5", colReader.getAsString(1));
+    assertEquals("30.0", colReader.getAsString(2));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testFloatArray() {
+    doubleArrayTester(MinorType.FLOAT4);
+  }
+
+  @Test
+  public void testDoubleRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.FLOAT8)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(0D)
+        .addRow(Double.MAX_VALUE)
+        .addRow(Double.MIN_VALUE)
+        .addRow(100D)
+        .build();
+    assertEquals(4, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.DOUBLE, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, colReader.getDouble(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(Double.MAX_VALUE, colReader.getDouble(), 0.000001);
+    assertEquals(Double.MAX_VALUE, (double) colReader.getObject(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(Double.MIN_VALUE, colReader.getDouble(), 0.000001);
+
+    assertTrue(reader.next());
+    assertEquals(100, colReader.getDouble(), 0.000001);
+    assertEquals("100.0", colReader.getAsString());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableDouble() {
+    nullableDoubleTester(MinorType.FLOAT8);
+  }
+
+  @Test
+  public void testDoubleArray() {
+    doubleArrayTester(MinorType.FLOAT8);
+  }
+
+  @Test
+  public void testStringRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow("")
+        .addRow("abcd")
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.STRING, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals("", colReader.getString());
+
+    assertTrue(reader.next());
+    assertEquals("abcd", colReader.getString());
+    assertEquals("abcd", colReader.getObject());
+    assertEquals("\"abcd\"", colReader.getAsString());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableString() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow("")
+        .addSingleCol(null)
+        .addRow("abcd")
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals("", colReader.getString());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals("abcd", colReader.getString());
+    assertEquals("abcd", colReader.getObject());
+    assertEquals("\"abcd\"", colReader.getAsString());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testStringArray() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", MinorType.VARCHAR)
+        .build();
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new String[] {})
+        .addSingleCol(new String[] {"fred", "", "wilma"})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.STRING, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals("fred", colReader.getString(0));
+    assertEquals("", colReader.getString(1));
+    assertEquals("wilma", colReader.getString(2));
+    assertEquals("fred", colReader.getObject(0));
+    assertEquals("", colReader.getObject(1));
+    assertEquals("wilma", colReader.getObject(2));
+    assertEquals("\"fred\"", colReader.getAsString(0));
+    assertEquals("\"\"", colReader.getAsString(1));
+    assertEquals("\"wilma\"", colReader.getAsString(2));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalYearRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.INTERVALYEAR)
+        .build();
+
+    Period p1 = Period.years(0);
+    Period p2 = Period.years(2).plusMonths(3);
+    Period p3 = Period.years(1234).plusMonths(11);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addRow(p2)
+        .addRow(p3)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(p1, colReader.getPeriod());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod());
+    assertEquals(p2, colReader.getObject());
+    assertEquals(p2.toString(), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p3, colReader.getPeriod());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableIntervalYear() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", MinorType.INTERVALYEAR)
+        .build();
+
+    Period p1 = Period.years(0);
+    Period p2 = Period.years(2).plusMonths(3);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addSingleCol(null)
+        .addRow(p2)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(p1, colReader.getPeriod());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getPeriod());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalYearArray() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", MinorType.INTERVALYEAR)
+        .build();
+
+    Period p1 = Period.years(0);
+    Period p2 = Period.years(2).plusMonths(3);
+    Period p3 = Period.years(1234).plusMonths(11);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new Period[] {})
+        .addSingleCol(new Period[] {p1, p2, p3})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(p1, colReader.getPeriod(0));
+    assertEquals(p2, colReader.getPeriod(1));
+    assertEquals(p3, colReader.getPeriod(2));
+    assertEquals(p2, colReader.getObject(1));
+    assertEquals(p2.toString(), colReader.getAsString(1));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalDayRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.INTERVALDAY)
+        .build();
+
+    Period p1 = Period.days(0);
+    Period p2 = Period.days(3).plusHours(4).plusMinutes(5).plusSeconds(23);
+    Period p3 = Period.days(999).plusHours(23).plusMinutes(59).plusSeconds(59);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addRow(p2)
+        .addRow(p3)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    // The normalizedStandard() call is a hack. See DRILL-5689.
+    assertEquals(p1, colReader.getPeriod().normalizedStandard());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod().normalizedStandard());
+    assertEquals(p2, ((Period) colReader.getObject()).normalizedStandard());
+    assertEquals(p2.toString(), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p3.normalizedStandard(), colReader.getPeriod().normalizedStandard());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableIntervalDay() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", MinorType.INTERVALDAY)
+        .build();
+
+    Period p1 = Period.years(0);
+    Period p2 = Period.days(3).plusHours(4).plusMinutes(5).plusSeconds(23);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addSingleCol(null)
+        .addRow(p2)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(p1, colReader.getPeriod().normalizedStandard());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getPeriod());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod().normalizedStandard());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalDayArray() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", MinorType.INTERVALDAY)
+        .build();
+
+    Period p1 = Period.days(0);
+    Period p2 = Period.days(3).plusHours(4).plusMinutes(5).plusSeconds(23);
+    Period p3 = Period.days(999).plusHours(23).plusMinutes(59).plusSeconds(59);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new Period[] {})
+        .addSingleCol(new Period[] {p1, p2, p3})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(p1, colReader.getPeriod(0).normalizedStandard());
+    assertEquals(p2, colReader.getPeriod(1).normalizedStandard());
+    assertEquals(p3.normalizedStandard(), colReader.getPeriod(2).normalizedStandard());
+    assertEquals(p2, ((Period) colReader.getObject(1)).normalizedStandard());
+    assertEquals(p2.toString(), colReader.getAsString(1));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.INTERVAL)
+        .build();
+
+    Period p1 = Period.days(0);
+    Period p2 = Period.years(7).plusMonths(8)
+                      .plusDays(3).plusHours(4)
+                      .plusMinutes(5).plusSeconds(23);
+    Period p3 = Period.years(9999).plusMonths(11)
+                      .plusDays(365).plusHours(23)
+                      .plusMinutes(59).plusSeconds(59);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addRow(p2)
+        .addRow(p3)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    // The normalizedStandard() call is a hack. See DRILL-5689.
+    assertEquals(p1, colReader.getPeriod().normalizedStandard());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod().normalizedStandard());
+    assertEquals(p2, ((Period) colReader.getObject()).normalizedStandard());
+    assertEquals(p2.toString(), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p3.normalizedStandard(), colReader.getPeriod().normalizedStandard());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableInterval() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", MinorType.INTERVAL)
+        .build();
+
+    Period p1 = Period.years(0);
+    Period p2 = Period.years(7).plusMonths(8)
+                      .plusDays(3).plusHours(4)
+                      .plusMinutes(5).plusSeconds(23);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(p1)
+        .addSingleCol(null)
+        .addRow(p2)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(p1, colReader.getPeriod().normalizedStandard());
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getPeriod());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(p2, colReader.getPeriod().normalizedStandard());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testIntervalArray() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", MinorType.INTERVAL)
+        .build();
+
+    Period p1 = Period.days(0);
+    Period p2 = Period.years(7).plusMonths(8)
+                      .plusDays(3).plusHours(4)
+                      .plusMinutes(5).plusSeconds(23);
+    Period p3 = Period.years(9999).plusMonths(11)
+                      .plusDays(365).plusHours(23)
+                      .plusMinutes(59).plusSeconds(59);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new Period[] {})
+        .addSingleCol(new Period[] {p1, p2, p3})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.PERIOD, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(p1, colReader.getPeriod(0).normalizedStandard());
+    assertEquals(p2, colReader.getPeriod(1).normalizedStandard());
+    assertEquals(p3.normalizedStandard(), colReader.getPeriod(2).normalizedStandard());
+    assertEquals(p2, ((Period) colReader.getObject(1)).normalizedStandard());
+    assertEquals(p2.toString(), colReader.getAsString(1));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testDecimal9RW() {
+    MajorType type = MajorType.newBuilder()
+        .setMinorType(MinorType.DECIMAL9)
+        .setScale(3)
+        .setPrecision(9)
+        .setMode(DataMode.REQUIRED)
+        .build();
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", type)
+        .build();
+
+    BigDecimal v1 = BigDecimal.ZERO;
+    BigDecimal v2 = BigDecimal.valueOf(123_456_789, 3);
+    BigDecimal v3 = BigDecimal.valueOf(999_999_999, 3);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(v1)
+        .addRow(v2)
+        .addRow(v3)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.DECIMAL, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, v1.compareTo(colReader.getDecimal()));
+
+    assertTrue(reader.next());
+    assertEquals(0, v2.compareTo(colReader.getDecimal()));
+    assertEquals(0, v2.compareTo((BigDecimal) colReader.getObject()));
+    assertEquals(v2.toString(), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(0, v3.compareTo(colReader.getDecimal()));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  private void nullableDecimalTester(MinorType type, int precision) {
+    MajorType majorType = MajorType.newBuilder()
+        .setMinorType(type)
+        .setScale(3)
+        .setPrecision(precision)
+        .setMode(DataMode.OPTIONAL)
+        .build();
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", majorType)
+        .build();
+
+    BigDecimal v1 = BigDecimal.ZERO;
+    BigDecimal v2 = BigDecimal.valueOf(123_456_789, 3);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(v1)
+        .addSingleCol(null)
+        .addRow(v2)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.DECIMAL, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, v1.compareTo(colReader.getDecimal()));
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(0, v2.compareTo(colReader.getDecimal()));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableDecimal9() {
+    nullableDecimalTester(MinorType.DECIMAL9, 9);
+  }
+
+  private void decimalArrayTester(MinorType type, int precision) {
+    MajorType majorType = MajorType.newBuilder()
+        .setMinorType(type)
+        .setScale(3)
+        .setPrecision(precision)
+        .setMode(DataMode.REPEATED)
+        .build();
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", majorType)
+        .build();
+
+    BigDecimal v1 = BigDecimal.ZERO;
+    BigDecimal v2 = BigDecimal.valueOf(123_456_789, 3);
+    BigDecimal v3 = BigDecimal.TEN;
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new BigDecimal[] {})
+        .addSingleCol(new BigDecimal[] {v1, v2, v3})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.DECIMAL, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertEquals(0, v1.compareTo(colReader.getDecimal(0)));
+    assertEquals(0, v2.compareTo(colReader.getDecimal(1)));
+    assertEquals(0, v3.compareTo(colReader.getDecimal(2)));
+    assertEquals(0, v2.compareTo((BigDecimal) colReader.getObject(1)));
+    assertEquals(v2.toString(), colReader.getAsString(1));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testDecimal9Array() {
+    decimalArrayTester(MinorType.DECIMAL9, 9);
+  }
+
+  @Test
+  public void testDecimal18RW() {
+    MajorType type = MajorType.newBuilder()
+        .setMinorType(MinorType.DECIMAL18)
+        .setScale(3)
+        .setPrecision(9)
+        .setMode(DataMode.REQUIRED)
+        .build();
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", type)
+        .build();
+
+    BigDecimal v1 = BigDecimal.ZERO;
+    BigDecimal v2 = BigDecimal.valueOf(123_456_789_123_456_789L, 3);
+    BigDecimal v3 = BigDecimal.valueOf(999_999_999_999_999_999L, 3);
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(v1)
+        .addRow(v2)
+        .addRow(v3)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.DECIMAL, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertEquals(0, v1.compareTo(colReader.getDecimal()));
+
+    assertTrue(reader.next());
+    assertEquals(0, v2.compareTo(colReader.getDecimal()));
+    assertEquals(0, v2.compareTo((BigDecimal) colReader.getObject()));
+    assertEquals(v2.toString(), colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertEquals(0, v3.compareTo(colReader.getDecimal()));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableDecimal18() {
+    nullableDecimalTester(MinorType.DECIMAL18, 9);
+  }
+
+  @Test
+  public void testDecimal18Array() {
+    decimalArrayTester(MinorType.DECIMAL18, 9);
+  }
+
+  // From the perspective of the vector, a date vector is just a long.
+
+  @Test
+  public void testDateRW() {
+    longRWTester(MinorType.DATE);
+  }
+
+  @Test
+  public void testNullableDate() {
+    nullableLongTester(MinorType.DATE);
+  }
+
+  @Test
+  public void testDateArray() {
+    longArrayTester(MinorType.DATE);
+  }
+
+  // From the perspective of the vector, a timestamp vector is just a long.
+
+  @Test
+  public void testTimestampRW() {
+    longRWTester(MinorType.TIMESTAMP);
+  }
+
+  @Test
+  public void testNullableTimestamp() {
+    nullableLongTester(MinorType.TIMESTAMP);
+  }
+
+  @Test
+  public void testTimestampArray() {
+    longArrayTester(MinorType.TIMESTAMP);
+  }
+
+  @Test
+  public void testVarBinaryRW() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .add("col", MinorType.VARBINARY)
+        .build();
+
+    byte v1[] = new byte[] {};
+    byte v2[] = new byte[] { (byte) 0x00, (byte) 0x7f, (byte) 0x80, (byte) 0xFF};
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(v1)
+        .addRow(v2)
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.BYTES, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertTrue(Arrays.equals(v1, colReader.getBytes()));
+
+    assertTrue(reader.next());
+    assertTrue(Arrays.equals(v2, colReader.getBytes()));
+    assertTrue(Arrays.equals(v2, (byte[]) colReader.getObject()));
+    assertEquals("[00, 7f, 80, ff]", colReader.getAsString());
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testNullableVarBinary() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addNullable("col", MinorType.VARBINARY)
+        .build();
+
+    byte v1[] = new byte[] {};
+    byte v2[] = new byte[] { (byte) 0x00, (byte) 0x7f, (byte) 0x80, (byte) 0xFF};
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addRow(v1)
+        .addSingleCol(null)
+        .addRow(v2)
+        .build();
+    assertEquals(3, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarReader colReader = reader.scalar(0);
+    assertEquals(ValueType.BYTES, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertFalse(colReader.isNull());
+    assertTrue(Arrays.equals(v1, colReader.getBytes()));
+
+    assertTrue(reader.next());
+    assertTrue(colReader.isNull());
+    assertNull(colReader.getObject());
+    assertEquals("null", colReader.getAsString());
+
+    assertTrue(reader.next());
+    assertTrue(Arrays.equals(v2, colReader.getBytes()));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+
+  @Test
+  public void testVarBinaryArray() {
+    BatchSchema batchSchema = new SchemaBuilder()
+        .addArray("col", MinorType.VARBINARY)
+        .build();
+
+    byte v1[] = new byte[] {};
+    byte v2[] = new byte[] { (byte) 0x00, (byte) 0x7f, (byte) 0x80, (byte) 0xFF};
+    byte v3[] = new byte[] { (byte) 0xDE, (byte) 0xAD, (byte) 0xBE, (byte) 0xAF};
+
+    SingleRowSet rs = fixture.rowSetBuilder(batchSchema)
+        .addSingleCol(new byte[][] {})
+        .addSingleCol(new byte[][] {v1, v2, v3})
+        .build();
+    assertEquals(2, rs.rowCount());
+
+    RowSetReader reader = rs.reader();
+    ScalarElementReader colReader = reader.elements(0);
+    assertEquals(ValueType.BYTES, colReader.valueType());
+
+    assertTrue(reader.next());
+    assertEquals(0, colReader.size());
+
+    assertTrue(reader.next());
+    assertEquals(3, colReader.size());
+    assertTrue(Arrays.equals(v1, colReader.getBytes(0)));
+    assertTrue(Arrays.equals(v2, colReader.getBytes(1)));
+    assertTrue(Arrays.equals(v3, colReader.getBytes(2)));
+    assertTrue(Arrays.equals(v2, (byte[]) colReader.getObject(1)));
+    assertEquals("[00, 7f, 80, ff]", colReader.getAsString(1));
+
+    assertFalse(reader.next());
+    rs.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestVariableWidthWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestVariableWidthWriter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestVariableWidthWriter.java
new file mode 100644
index 0000000..103b212
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestVariableWidthWriter.java
@@ -0,0 +1,418 @@
+/*
+ * 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.test.rowSet.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors.VarCharColumnWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.apache.drill.test.rowSet.test.TestFixedWidthWriter.TestIndex;
+import org.bouncycastle.util.Arrays;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+
+public class TestVariableWidthWriter extends SubOperatorTest {
+
+  /**
+   * Basic test to write a contiguous set of values, enough to cause
+   * the vector to double in size twice, then read back the values.
+   */
+
+  @Test
+  public void testWrite() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+
+      writer.startWrite();
+
+      // Write integers.
+      // Write enough that the vector is resized.
+
+      long origAddr = vector.getBuffer().addr();
+      String base = "sample-value";
+      for (int i = 0; i < 3000; i++) {
+        index.index = i;
+        writer.setString(base + i);
+      }
+      writer.endWrite();
+
+      // Should have been reallocated.
+
+      assertNotEquals(origAddr, vector.getBuffer().addr());
+
+      // Verify values
+
+      for (int i = 0; i < 3000; i++) {
+        assertEquals(base + i, stringAt(vector, i));
+      }
+    }
+  }
+
+  @Test
+  public void testRestartRow() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write rows, rewriting every other row.
+
+      String base = "sample-value";
+      writer.startRow();
+      index.index = 0;
+      for (int i = 0; i < 50; i++) {
+        writer.setString(base + i);
+        if (i % 2 == 0) {
+          writer.saveRow();
+          writer.startRow();
+          index.index++;
+        } else {
+          writer.restartRow();
+        }
+      }
+      writer.endWrite();
+
+      // Verify values
+
+      for (int i = 0; i < 25; i++) {
+        assertEquals(base + (2 * i), stringAt(vector, i));
+      }
+    }
+  }
+
+  /**
+   * Filling empties in a variable-width row means carrying forward
+   * offsets (as tested elsewhere), leaving zero-length values.
+   */
+
+  @Test
+  public void testFillEmpties() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write values, skipping four out of five positions,
+      // forcing backfill.
+      // The number of values is odd, forcing the writer to
+      // back-fill at the end as well as between values.
+
+      String base = "sample-value";
+      for (int i = 0; i < 501; i += 5) {
+        index.index = i;
+        writer.startRow();
+        writer.setString(base + i);
+        writer.saveRow();
+      }
+      // At end, vector index defined to point one past the
+      // last row. That is, the vector index gives the row count.
+
+      index.index = 504;
+      writer.endWrite();
+
+      // Verify values
+
+      for (int i = 0; i < 504; i++) {
+        assertEquals("Mismatch on " + i,
+            (i%5) == 0 ? base + i : "", stringAt(vector, i));
+      }
+    }
+  }
+
+  /**
+   * The rollover method is used during vector overflow.
+   */
+
+  @Test
+  public void testRollover() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of ten values.
+
+      String base = "sample-value";
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setString(base + i);
+        writer.saveRow();
+      }
+
+      // Overflow occurs after writing the 11th row
+
+      index.index = 10;
+      writer.startRow();
+      String overflowValue = base + 10;
+      writer.setString(overflowValue);
+
+      // Overflow occurs
+
+      writer.preRollover();
+
+      // Simulate rollover
+
+      byte dummy[] = new byte[] { (byte) 0x55 };
+      for (int i = 0; i < 500; i++) {
+        vector.getMutator().setSafe(i, dummy);
+      }
+      for (int i = 1; i < 15; i++) {
+        vector.getOffsetVector().getMutator().set(i, 0xdeadbeef);
+      }
+      vector.getMutator().setSafe(0, overflowValue.getBytes(Charsets.UTF_8));
+
+      writer.postRollover();
+      index.index = 0;
+      writer.saveRow();
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setString(base + (i + 10));
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 5; i++) {
+        assertEquals(base + (10 + i), stringAt(vector, i));
+      }
+    }
+  }
+
+  /**
+   * Simulate the case in which the tail end of an overflow
+   * batch has empties. <tt>preRollover()</tt> should back-fill
+   * them with the next offset prior to rollover.
+   */
+
+  @Test
+  public void testRolloverWithEmpties() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Simulate doing an overflow of 15 values,
+      // of which 5 are empty.
+
+      String base = "sample-value";
+      for (int i = 0; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setString(base + i);
+        writer.saveRow();
+      }
+
+      for (int i = 10; i < 15; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Overflow occurs before writing the 16th row
+
+      index.index = 15;
+      writer.startRow();
+
+      // Overflow occurs. This should fill empty offsets.
+
+      writer.preRollover();
+
+      // Verify the first "batch" results
+
+      for (int i = 0; i < 10; i++) {
+        assertEquals(base + i, stringAt(vector, i));
+      }
+      for (int i = 10; i < 15; i++) {
+        assertEquals("", stringAt(vector, i));
+      }
+
+      // Simulate rollover
+
+      byte dummy[] = new byte[] { (byte) 0x55 };
+      for (int i = 0; i < 500; i++) {
+        vector.getMutator().setSafe(i, dummy);
+      }
+      for (int i = 1; i < 15; i++) {
+        vector.getOffsetVector().getMutator().set(i, 0xdeadbeef);
+      }
+      vector.getMutator().setSafe(0, new byte[] {});
+
+      writer.postRollover();
+      index.index = 0;
+      writer.saveRow();
+
+      // Skip more values.
+
+      for (int i = 1; i < 5; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.saveRow();
+      }
+
+      // Simulate resuming with a few more values.
+
+      for (int i = 5; i < 10; i++) {
+        index.index = i;
+        writer.startRow();
+        writer.setString(base + (i + 20));
+        writer.saveRow();
+      }
+      writer.endWrite();
+
+      // Verify the results
+
+      for (int i = 0; i < 5; i++) {
+        assertEquals("", stringAt(vector, i));
+      }
+      for (int i = 5; i < 10; i++) {
+        assertEquals(base + (i + 20), stringAt(vector, i));
+      }
+    }
+  }
+
+
+  /**
+   * Test the case in which a scalar vector is used in conjunction
+   * with a nullable bits vector. The nullable vector will call the
+   * <tt>skipNulls()</tt> method to avoid writing values for null
+   * entries. For variable-width, there is no difference between
+   * filling empties and skipping nulls: both result in zero-sized
+   * entries.
+   */
+
+  @Test
+  public void testSkipNulls() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.startWrite();
+
+      // Write values, skipping four out of five positions,
+      // skipping nulls.
+      // The number of values is odd, forcing the writer to
+      // skip nulls at the end as well as between values.
+
+      String base = "sample-value";
+      for (int i = 0; i < 3000; i += 5) {
+        index.index = i;
+        writer.startRow();
+        writer.skipNulls();
+        writer.setString(base + i);
+        writer.saveRow();
+      }
+      index.index = 3003;
+      writer.startRow();
+      writer.skipNulls();
+      writer.saveRow();
+      writer.endWrite();
+
+      // Verify values. Skipping nulls should back-fill
+      // offsets, resulting in zero-length strings.
+
+      for (int i = 0; i < 3000; i++) {
+        assertEquals("Mismatch at " + i,
+            (i%5) == 0 ? base + i : "", stringAt(vector, i));
+      }
+    }
+  }
+
+  /**
+   * Test resize monitoring. Add a listener to an Varchar writer,
+   * capture each resize, and refuse a resize when the s
+   * of the vector exceeds 1 MB. This will trigger an overflow,
+   * which will throw an exception which we then check for.
+   */
+
+  @Test
+  public void testSizeLimit() {
+    try (VarCharVector vector = allocVector(1000)) {
+      TestIndex index = new TestIndex();
+      VarCharColumnWriter writer = makeWriter(vector, index);
+      writer.bindListener(new ColumnWriterListener() {
+        // Because assumed array size is 10, so 10 * 1000 = 10,000
+        // rounded to 16K
+        int totalAlloc = 16384;
+
+        @Override
+        public void overflowed(ScalarWriter writer) {
+          throw new IllegalStateException("overflow called");
+        }
+
+        @Override
+        public boolean canExpand(ScalarWriter writer, int delta) {
+          System.out.println("Delta: " + delta);
+          totalAlloc += delta;
+          return totalAlloc < 1024 * 1024;
+        }
+      });
+      writer.startWrite();
+
+      byte value[] = new byte[423];
+      Arrays.fill(value, (byte) 'X');
+      try {
+        for (int i = 0; ; i++ ) {
+          index.index = i;
+          writer.startRow();
+          writer.setBytes(value, value.length);
+          writer.saveRow();
+        }
+      }
+      catch(IllegalStateException e) {
+        assertTrue(e.getMessage().contains("overflow called"));
+      }
+    }
+  }
+
+  private String stringAt(VarCharVector vector, int i) {
+    return new String(vector.getAccessor().get(i), Charsets.UTF_8);
+  }
+
+  private VarCharVector allocVector(int size) {
+    MaterializedField field =
+        SchemaBuilder.columnSchema("x", MinorType.VARCHAR, DataMode.REQUIRED);
+    VarCharVector vector = new VarCharVector(field, fixture.allocator());
+    vector.allocateNew(size * 10, size);
+    return vector;
+  }
+
+  private VarCharColumnWriter makeWriter(VarCharVector vector, TestIndex index) {
+    VarCharColumnWriter writer = new VarCharColumnWriter(vector);
+    writer.bindIndex(index);
+
+    assertEquals(ValueType.STRING, writer.valueType());
+    return writer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/VectorPrinter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/VectorPrinter.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/VectorPrinter.java
new file mode 100644
index 0000000..2056220
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/VectorPrinter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.test.rowSet.test;
+
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+
+import com.google.common.base.Charsets;
+
+/**
+ * Handy tool to visualize string and offset vectors for
+ * debugging.
+ */
+
+public class VectorPrinter {
+
+  public static void printOffsets(UInt4Vector vector, int start, int length) {
+    header(vector, start, length);
+    for (int i = start, j = 0; j < length; i++, j++) {
+      if (j > 0) {
+        System.out.print(" ");
+      }
+      System.out.print(vector.getAccessor().get(i));
+    }
+    System.out.print("], addr = ");
+    System.out.println(vector.getBuffer().addr());
+  }
+
+  public static void printStrings(VarCharVector vector, int start, int length) {
+    printOffsets(vector.getOffsetVector(), start, length + 1);
+    header(vector, start, length);
+    System.out.println();
+    for (int i = start, j = 0; j < length; i++, j++) {
+      System.out.print("  ");
+      System.out.print(i);
+      System.out.print(": \"");
+      System.out.print(stringAt(vector, i));
+      System.out.println("\"");
+    }
+    System.out.println("]");
+  }
+
+  public static void header(ValueVector vector, int start, int length) {
+    System.out.print(vector.getClass());
+    System.out.print(": (");
+    System.out.print(start);
+    System.out.print(" - ");
+    System.out.print(start + length - 1);
+    System.out.print("): [");
+  }
+
+  public static String stringAt(VarCharVector vector, int i) {
+    return new String(vector.getAccessor().get(i), Charsets.UTF_8);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java b/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java
index 4da526e..f3390d3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java
@@ -19,59 +19,30 @@
 package org.apache.drill.vector;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
 import org.apache.drill.categories.VectorTest;
 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.record.MaterializedField;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.RepeatedVarCharVector;
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.drill.exec.vector.VectorOverflowException;
-import org.apache.drill.test.DrillTest;
-import org.apache.drill.test.OperatorFixture;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
 import org.junit.Test;
 
 import io.netty.buffer.DrillBuf;
 import org.junit.experimental.categories.Category;
 
 @Category(VectorTest.class)
-public class TestFillEmpties extends DrillTest {
-
-  public static OperatorFixture fixture;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    fixture = OperatorFixture.builder().build();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    fixture.close();
-  }
-
-  // To be replaced by a test method in a separate commit.
-
-  public static MaterializedField makeField(String name, MinorType dataType, DataMode mode) {
-    MajorType type = MajorType.newBuilder()
-        .setMinorType(dataType)
-        .setMode(mode)
-        .build();
-
-    return MaterializedField.create(name, type);
-  }
+public class TestFillEmpties extends SubOperatorTest {
 
   @Test
   public void testNullableVarChar() {
     @SuppressWarnings("resource")
-    NullableVarCharVector vector = new NullableVarCharVector(makeField("a", MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator());
+    NullableVarCharVector vector = new NullableVarCharVector(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator());
     vector.allocateNew( );
 
     // Create "foo", null, "bar", but omit the null.
@@ -91,7 +62,7 @@ public class TestFillEmpties extends DrillTest {
   @Test
   public void testVarChar() {
     @SuppressWarnings("resource")
-    VarCharVector vector = new VarCharVector(makeField("a", MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator());
+    VarCharVector vector = new VarCharVector(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator());
     vector.allocateNew( );
 
     // Create "foo", null, "bar", but omit the null.
@@ -103,11 +74,7 @@ public class TestFillEmpties extends DrillTest {
     // Work around: test fails without this. But, only the new column writers
     // call this method.
 
-    try {
-      mutator.fillEmptiesBounded(0, 2);
-    } catch (VectorOverflowException e) {
-      fail();
-    }
+    mutator.fillEmpties(0, 2);
     value = makeValue("bar");
     mutator.setSafe(2, value, 0, value.length);
 
@@ -119,7 +86,7 @@ public class TestFillEmpties extends DrillTest {
   @Test
   public void testInt() {
     @SuppressWarnings("resource")
-    IntVector vector = new IntVector(makeField("a", MinorType.INT, DataMode.REQUIRED), fixture.allocator());
+    IntVector vector = new IntVector(SchemaBuilder.columnSchema("a", MinorType.INT, DataMode.REQUIRED), fixture.allocator());
     vector.allocateNew( );
 
     // Create 1, 0, 2, but omit the 0.
@@ -136,7 +103,7 @@ public class TestFillEmpties extends DrillTest {
   @Test
   public void testRepeatedVarChar() {
     @SuppressWarnings("resource")
-    RepeatedVarCharVector vector = new RepeatedVarCharVector(makeField("a", MinorType.VARCHAR, DataMode.REPEATED), fixture.allocator());
+    RepeatedVarCharVector vector = new RepeatedVarCharVector(SchemaBuilder.columnSchema("a", MinorType.VARCHAR, DataMode.REPEATED), fixture.allocator());
     vector.allocateNew( );
 
     // Create "foo", null, "bar", but omit the null.
@@ -151,11 +118,7 @@ public class TestFillEmpties extends DrillTest {
     // Work around: test fails without this. But, only the new column writers
     // call this method.
 
-    try {
-      mutator.fillEmptiesBounded(0, 2);
-    } catch (VectorOverflowException e) {
-      fail();
-    }
+    mutator.fillEmpties(0, 2);
     mutator.startNewValue(2);
     value = makeValue( "c" );
     mutator.addSafe(2, value, 0, value.length);


[12/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
new file mode 100644
index 0000000..27a88f0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/TupleSchema.java
@@ -0,0 +1,534 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.record;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+
+/**
+ * Defines the schema of a tuple: either the top-level row or a nested
+ * "map" (really structure). A schema is a collection of columns (backed
+ * by vectors in the loader itself.) Columns are accessible by name or
+ * index. New columns may be added at any time; the new column takes the
+ * next available index.
+ */
+
+public class TupleSchema implements TupleMetadata {
+
+  /**
+   * Abstract definition of column metadata. Allows applications to create
+   * specialized forms of a column metadata object by extending from this
+   * abstract class.
+   * <p>
+   * Note that, by design, primitive columns do not have a link to their
+   * tuple parent, or their index within that parent. This allows the same
+   * metadata to be shared between two views of a tuple, perhaps physical
+   * and projected views. This restriction does not apply to map columns,
+   * since maps (and the row itself) will, by definition, differ between
+   * the two views.
+   */
+
+  public static abstract class AbstractColumnMetadata implements ColumnMetadata {
+
+    protected MaterializedField schema;
+    protected boolean projected = true;
+
+    /**
+     * Predicted number of elements per array entry. Default is
+     * taken from the often hard-coded value of 10.
+     */
+
+    protected int expectedElementCount = 1;
+
+    public AbstractColumnMetadata(MaterializedField schema) {
+      this.schema = schema;
+      if (isArray()) {
+        expectedElementCount = DEFAULT_ARRAY_SIZE;
+      }
+    }
+
+    public AbstractColumnMetadata(AbstractColumnMetadata from) {
+      schema = from.schema;
+      expectedElementCount = from.expectedElementCount;
+    }
+
+    protected void bind(TupleSchema parentTuple) { }
+
+    @Override
+    public MaterializedField schema() { return schema; }
+
+    public void replaceField(MaterializedField field) {
+      this.schema = field;
+    }
+    @Override
+    public String name() { return schema().getName(); }
+
+    @Override
+    public MajorType majorType() { return schema().getType(); }
+
+    @Override
+    public MinorType type() { return schema().getType().getMinorType(); }
+
+    @Override
+    public DataMode mode() { return schema().getDataMode(); }
+
+    @Override
+    public boolean isNullable() { return mode() == DataMode.OPTIONAL; }
+
+    @Override
+    public boolean isArray() { return mode() == DataMode.REPEATED; }
+
+    @Override
+    public boolean isList() { return false; }
+
+    @Override
+    public boolean isVariableWidth() {
+      MinorType type = type();
+      return type == MinorType.VARCHAR || type == MinorType.VAR16CHAR || type == MinorType.VARBINARY;
+    }
+
+    @Override
+    public boolean isEquivalent(ColumnMetadata other) {
+      return schema().isEquivalent(other.schema());
+    }
+
+    @Override
+    public int expectedWidth() { return 0; }
+
+    @Override
+    public void setExpectedWidth(int width) { }
+
+    @Override
+    public void setExpectedElementCount(int childCount) {
+      // The allocation utilities don't like an array size of zero, so set to
+      // 1 as the minimum. Adjusted to avoid trivial errors if the caller
+      // makes an error.
+
+      if (isArray()) {
+        expectedElementCount = Math.max(1, childCount);
+      }
+    }
+
+    @Override
+    public int expectedElementCount() { return expectedElementCount; }
+
+    @Override
+    public void setProjected(boolean projected) {
+      this.projected = projected;
+    }
+
+    @Override
+    public boolean isProjected() { return projected; }
+
+    @Override
+    public String toString() {
+      StringBuilder buf = new StringBuilder()
+          .append("[")
+          .append(getClass().getSimpleName())
+          .append(" ")
+          .append(schema().toString())
+          .append(",")
+          .append(projected ? "" : "not ")
+          .append("projected");
+      if (isArray()) {
+        buf.append(", cardinality: ")
+           .append(expectedElementCount);
+      }
+      return buf
+          .append("]")
+          .toString();
+    }
+
+    public abstract AbstractColumnMetadata copy();
+  }
+
+  /**
+   * Primitive (non-map) column. Describes non-nullable, nullable and
+   * array types (which differ only in mode, but not in metadata structure.)
+   */
+
+  public static class PrimitiveColumnMetadata extends AbstractColumnMetadata {
+
+    protected int expectedWidth;
+
+    public PrimitiveColumnMetadata(MaterializedField schema) {
+      super(schema);
+      expectedWidth = TypeHelper.getSize(majorType());
+      if (isVariableWidth()) {
+
+        // The above getSize() method uses the deprecated getWidth()
+        // method to get the expected VarChar size. If zero (which
+        // it will be), try the revised precision field.
+
+        int precision = majorType().getPrecision();
+        if (precision > 0) {
+          expectedWidth = precision;
+        } else {
+          // TypeHelper includes the offset vector width
+
+          expectedWidth = expectedWidth - 4;
+        }
+      }
+    }
+
+    public PrimitiveColumnMetadata(PrimitiveColumnMetadata from) {
+      super(from);
+      expectedWidth = from.expectedWidth;
+    }
+
+    @Override
+    public AbstractColumnMetadata copy() {
+      return new PrimitiveColumnMetadata(this);
+    }
+
+    @Override
+    public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.PRIMITIVE; }
+
+    @Override
+    public TupleMetadata mapSchema() { return null; }
+
+    @Override
+    public boolean isMap() { return false; }
+
+    @Override
+    public int expectedWidth() { return expectedWidth; }
+
+    @Override
+    public void setExpectedWidth(int width) {
+      // The allocation utilities don't like a width of zero, so set to
+      // 1 as the minimum. Adjusted to avoid trivial errors if the caller
+      // makes an error.
+
+      if (isVariableWidth()) {
+        expectedWidth = Math.max(1, width);
+      }
+    }
+
+    @Override
+    public ColumnMetadata cloneEmpty() {
+      return new PrimitiveColumnMetadata(this);
+    }
+  }
+
+  /**
+   * Describes a map and repeated map. Both are tuples that have a tuple
+   * schema as part of the column definition.
+   */
+
+  public static class MapColumnMetadata extends AbstractColumnMetadata {
+    private TupleMetadata parentTuple;
+    private final TupleSchema mapSchema;
+
+    /**
+     * Build a new map column from the field provided
+     *
+     * @param schema materialized field description of the map
+     */
+
+    public MapColumnMetadata(MaterializedField schema) {
+      this(schema, null);
+    }
+
+    /**
+     * Build a map column metadata by cloning the type information (but not
+     * the children) of the materialized field provided. Use the hints
+     * provided.
+     *
+     * @param schema the schema to use
+     * @param hints metadata hints for this column
+     */
+
+    private MapColumnMetadata(MaterializedField schema, TupleSchema mapSchema) {
+      super(schema);
+      if (mapSchema == null) {
+        this.mapSchema = new TupleSchema();
+      } else {
+        this.mapSchema = mapSchema;
+      }
+      this.mapSchema.bind(this);
+    }
+
+    @Override
+    public AbstractColumnMetadata copy() {
+      return new MapColumnMetadata(schema, (TupleSchema) mapSchema.copy());
+    }
+
+    @Override
+    protected void bind(TupleSchema parentTuple) {
+      this.parentTuple = parentTuple;
+    }
+
+    @Override
+    public ColumnMetadata.StructureType structureType() { return ColumnMetadata.StructureType.TUPLE; }
+
+    @Override
+    public TupleMetadata mapSchema() { return mapSchema; }
+
+    @Override
+    public int expectedWidth() { return 0; }
+
+    @Override
+    public boolean isMap() { return true; }
+
+    public TupleMetadata parentTuple() { return parentTuple; }
+
+    public TupleSchema mapSchemaImpl() { return mapSchema; }
+
+    @Override
+    public ColumnMetadata cloneEmpty() {
+      return new MapColumnMetadata(schema().cloneEmpty(), null);
+    }
+  }
+
+  private MapColumnMetadata parentMap;
+  private final TupleNameSpace<ColumnMetadata> nameSpace = new TupleNameSpace<>();
+
+  public void bind(MapColumnMetadata parentMap) {
+    this.parentMap = parentMap;
+  }
+
+  public static TupleSchema fromFields(Iterable<MaterializedField> fields) {
+    TupleSchema tuple = new TupleSchema();
+    for (MaterializedField field : fields) {
+      tuple.add(field);
+    }
+    return tuple;
+  }
+
+  public TupleMetadata copy() {
+    TupleMetadata tuple = new TupleSchema();
+    for (ColumnMetadata md : this) {
+      tuple.addColumn(((AbstractColumnMetadata) md).copy());
+    }
+    return tuple;
+  }
+
+  /**
+   * Create a column metadata object that holds the given
+   * {@link MaterializedField}. The type of the object will be either a
+   * primitive or map column, depending on the field's type.
+   *
+   * @param field the materialized field to wrap
+   * @return the column metadata that wraps the field
+   */
+
+  public static AbstractColumnMetadata fromField(MaterializedField field) {
+    if (field.getType().getMinorType() == MinorType.MAP) {
+      return newMap(field);
+    } else {
+      return new PrimitiveColumnMetadata(field);
+    }
+  }
+
+  public static AbstractColumnMetadata fromView(MaterializedField field) {
+    if (field.getType().getMinorType() == MinorType.MAP) {
+      return new MapColumnMetadata(field, null);
+    } else {
+      return new PrimitiveColumnMetadata(field);
+    }
+  }
+
+  /**
+   * Create a tuple given the list of columns that make up the tuple.
+   * Creates nested maps as needed.
+   *
+   * @param columns list of columns that make up the tuple
+   * @return a tuple metadata object that contains the columns
+   */
+
+  public static TupleSchema fromColumns(List<ColumnMetadata> columns) {
+    TupleSchema tuple = new TupleSchema();
+    for (ColumnMetadata column : columns) {
+      tuple.add((AbstractColumnMetadata) column);
+    }
+    return tuple;
+  }
+
+  /**
+   * Create a column metadata object for a map column, given the
+   * {@link MaterializedField} that describes the column, and a list
+   * of column metadata objects that describe the columns in the map.
+   *
+   * @param field the materialized field that describes the map column
+   * @param schema metadata that describes the tuple of columns in
+   * the map
+   * @return a map column metadata for the map
+   */
+
+  public static MapColumnMetadata newMap(MaterializedField field, TupleSchema schema) {
+    return new MapColumnMetadata(field, schema);
+  }
+
+  public static MapColumnMetadata newMap(MaterializedField field) {
+    return new MapColumnMetadata(field, fromFields(field.getChildren()));
+  }
+
+  @Override
+  public ColumnMetadata add(MaterializedField field) {
+    AbstractColumnMetadata md = fromField(field);
+    add(md);
+    return md;
+  }
+
+  public ColumnMetadata addView(MaterializedField field) {
+    AbstractColumnMetadata md = fromView(field);
+    add(md);
+    return md;
+  }
+
+  /**
+   * Add a column metadata column created by the caller. Used for specialized
+   * cases beyond those handled by {@link #add(MaterializedField)}.
+   *
+   * @param md the custom column metadata which must have the correct
+   * index set (from {@link #size()}
+   */
+
+  public void add(AbstractColumnMetadata md) {
+    md.bind(this);
+    nameSpace.add(md.name(), md);
+    if (parentMap != null) {
+      parentMap.schema.addChild(md.schema());
+    }
+  }
+
+  @Override
+  public int addColumn(ColumnMetadata column) {
+    add((AbstractColumnMetadata) column);
+    return size() - 1;
+  }
+
+  @Override
+  public MaterializedField column(String name) {
+    ColumnMetadata md = metadata(name);
+    return md == null ? null : md.schema();
+  }
+
+  @Override
+  public ColumnMetadata metadata(String name) {
+    return nameSpace.get(name);
+  }
+
+  @Override
+  public int index(String name) {
+    return nameSpace.indexOf(name);
+  }
+
+  @Override
+  public MaterializedField column(int index) {
+    return metadata(index).schema();
+  }
+
+  @Override
+  public ColumnMetadata metadata(int index) {
+    return nameSpace.get(index);
+  }
+
+  @Override
+  public MapColumnMetadata parent() { return parentMap; }
+
+  @Override
+  public int size() { return nameSpace.count(); }
+
+  @Override
+  public boolean isEmpty() { return nameSpace.count( ) == 0; }
+
+  @Override
+  public Iterator<ColumnMetadata> iterator() {
+    return nameSpace.iterator();
+  }
+
+  @Override
+  public boolean isEquivalent(TupleMetadata other) {
+    TupleSchema otherSchema = (TupleSchema) other;
+    if (nameSpace.count() != otherSchema.nameSpace.count()) {
+      return false;
+    }
+    for (int i = 0; i < nameSpace.count(); i++) {
+      if (! nameSpace.get(i).isEquivalent(otherSchema.nameSpace.get(i))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public List<MaterializedField> toFieldList() {
+    List<MaterializedField> cols = new ArrayList<>();
+    for (ColumnMetadata md : nameSpace) {
+      cols.add(md.schema());
+    }
+    return cols;
+  }
+
+  public BatchSchema toBatchSchema(SelectionVectorMode svMode) {
+    return new BatchSchema(svMode, toFieldList());
+  }
+
+  @Override
+  public String fullName(int index) {
+    return fullName(metadata(index));
+  }
+
+  @Override
+  public String fullName(ColumnMetadata column) {
+    String quotedName = column.name();
+    if (quotedName.contains(".")) {
+      quotedName = "`" + quotedName + "`";
+    }
+    if (isRoot()) {
+      return column.name();
+    } else {
+      return fullName() + "." + quotedName;
+    }
+  }
+
+  public String fullName() {
+    if (isRoot()) {
+      return "<root>";
+    } else {
+      return parentMap.parentTuple().fullName(parentMap);
+    }
+  }
+
+  public boolean isRoot() { return parentMap == null; }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" ");
+    boolean first = true;
+    for (ColumnMetadata md : nameSpace) {
+      if (! first) {
+        buf.append(", ");
+      }
+      buf.append(md.toString());
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
index a38a7fe..42f3473 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
@@ -24,10 +24,16 @@ import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.record.DeadBuf;
 
 /**
- * A selection vector that fronts, at most, a
+ * A selection vector that fronts, at most, 64K values.
+ * The selection vector is used for two cases:
+ * <ol>
+ * <li>To create a list of values retained by a filter.</li>
+ * <li>To provide a redirection level for sorted
+ * batches.</li>
+ * </ol>
  */
+
 public class SelectionVector2 implements AutoCloseable {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector2.class);
 
   private final BufferAllocator allocator;
   private int recordCount;
@@ -39,9 +45,19 @@ public class SelectionVector2 implements AutoCloseable {
     this.allocator = allocator;
   }
 
+  /**
+   * Create a selection vector with the given buffer. The selection vector
+   * increments the buffer's reference count, talking ownership of the buffer.
+   *
+   * @param allocator allocator used to allocate the buffer
+   * @param buf the buffer containing the selection vector's data
+   * @param count the number of values in the selection vector
+   */
+
   public SelectionVector2(BufferAllocator allocator, DrillBuf buf, int count) {
     this.allocator = allocator;
     buffer = buf;
+    buffer.retain(1);
     recordCount = count;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
index cfb8645..a283924 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -27,15 +27,16 @@ import java.io.InputStream;
 import java.io.OutputStream;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.cache.VectorSerializer.Reader;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.SchemaBuilder;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -73,7 +74,7 @@ public class TestBatchSerialization extends DrillTest {
       if (i % 2 == 0) {
         RowSetUtilities.setFromInt(writer, 0, i);
       } else {
-        writer.column(0).setNull();
+        writer.scalar(0).setNull();
       }
       writer.save();
     }
@@ -125,9 +126,8 @@ public class TestBatchSerialization extends DrillTest {
 
     RowSet result;
     try (InputStream in = new BufferedInputStream(new FileInputStream(outFile))) {
-      result = fixture.wrap(
-        VectorSerializer.reader(fixture.allocator(), in)
-          .read());
+      Reader reader = VectorSerializer.reader(fixture.allocator(), in);
+      result = fixture.wrap(reader.read(), reader.sv2());
     }
 
     new RowSetComparison(expected)
@@ -163,17 +163,17 @@ public class TestBatchSerialization extends DrillTest {
 
   private SingleRowSet buildMapSet(BatchSchema schema) {
     return fixture.rowSetBuilder(schema)
-        .add(1, 100, "first")
-        .add(2, 200, "second")
-        .add(3, 300, "third")
+        .addRow(1, new Object[] {100, "first"})
+        .addRow(2, new Object[] {200, "second"})
+        .addRow(3, new Object[] {300, "third"})
         .build();
   }
 
   private SingleRowSet buildArraySet(BatchSchema schema) {
     return fixture.rowSetBuilder(schema)
-        .add(1, new String[] { "first, second, third" } )
-        .add(2, null)
-        .add(3, new String[] { "third, fourth, fifth" } )
+        .addRow(1, new String[] { "first, second, third" } )
+        .addRow(2, null)
+        .addRow(3, new String[] { "third, fourth, fifth" } )
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
index fa6e318..e7d0a97 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
@@ -80,16 +80,16 @@ public class TopNBatchTest extends PopUnitTestBase {
 
     try (RootAllocator allocator = new RootAllocator(100_000_000)) {
       expectedRowSet = new RowSetBuilder(allocator, batchSchema)
-        .add(110, 10)
-        .add(109, 9)
-        .add(108, 8)
-        .add(107, 7)
-        .add(106, 6)
-        .add(105, 5)
-        .add(104, 4)
-        .add(103, 3)
-        .add(102, 2)
-        .add(101, 1)
+        .addRow(110, 10)
+        .addRow(109, 9)
+        .addRow(108, 8)
+        .addRow(107, 7)
+        .addRow(106, 6)
+        .addRow(105, 5)
+        .addRow(104, 4)
+        .addRow(103, 3)
+        .addRow(102, 2)
+        .addRow(101, 1)
         .build();
 
       PriorityQueue queue;
@@ -121,10 +121,10 @@ public class TopNBatchTest extends PopUnitTestBase {
 
         for (int batchCounter = 0; batchCounter < numBatches; batchCounter++) {
           RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, batchSchema);
-          rowSetBuilder.add((batchCounter + bound), batchCounter);
+          rowSetBuilder.addRow((batchCounter + bound), batchCounter);
 
           for (int recordCounter = 0; recordCounter < numRecordsPerBatch; recordCounter++) {
-            rowSetBuilder.add(random.nextInt(bound), random.nextInt(bound));
+            rowSetBuilder.addRow(random.nextInt(bound), random.nextInt(bound));
           }
 
           VectorContainer vectorContainer = rowSetBuilder.build().container();
@@ -135,7 +135,7 @@ public class TopNBatchTest extends PopUnitTestBase {
         VectorContainer resultContainer = queue.getHyperBatch();
         resultContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
 
-        RowSet.HyperRowSet actualHyperSet = new HyperRowSetImpl(allocator, resultContainer, queue.getFinalSv4());
+        RowSet.HyperRowSet actualHyperSet = new HyperRowSetImpl(resultContainer, queue.getFinalSv4());
         new RowSetComparison(expectedRowSet).verify(actualHyperSet);
       } finally {
         if (expectedRowSet != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/validate/TestBatchValidator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/validate/TestBatchValidator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/validate/TestBatchValidator.java
index eafb4c8..202a0f1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/validate/TestBatchValidator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/validate/TestBatchValidator.java
@@ -68,10 +68,10 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add(10, 100)
-        .add(20, 120)
-        .add(30, null)
-        .add(40, 140)
+        .addRow(10, 100)
+        .addRow(20, 120)
+        .addRow(30, null)
+        .addRow(40, 140)
         .build();
 
     BatchValidator validator = new BatchValidator(batch.vectorAccessible(), true);
@@ -88,10 +88,10 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("col1.1", "col1.2")
-        .add("col2.1", "col2.2")
-        .add("col3.1", null)
-        .add("col4.1", "col4.2")
+        .addRow("col1.1", "col1.2")
+        .addRow("col2.1", "col2.2")
+        .addRow("col3.1", null)
+        .addRow("col4.1", "col4.2")
         .build();
 
     BatchValidator validator = new BatchValidator(batch.vectorAccessible(), true);
@@ -108,9 +108,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add(new int[] {}, new String[] {})
-        .add(new int[] {1, 2, 3}, new String[] {"fred", "barney", "wilma"})
-        .add(new int[] {4}, new String[] {"dino"})
+        .addRow(new int[] {}, new String[] {})
+        .addRow(new int[] {1, 2, 3}, new String[] {"fred", "barney", "wilma"})
+        .addRow(new int[] {4}, new String[] {"dino"})
         .build();
 
     BatchValidator validator = new BatchValidator(batch.vectorAccessible(), true);
@@ -126,9 +126,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("x")
-        .add("y")
-        .add("z")
+        .addRow("x")
+        .addRow("y")
+        .addRow("z")
         .build();
 
     // Here we are evil: stomp on the last offset to simulate corruption.
@@ -160,9 +160,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("x")
-        .add("y")
-        .add("z")
+        .addRow("x")
+        .addRow("y")
+        .addRow("z")
         .build();
 
     zapOffset(batch, 0, 1);
@@ -198,9 +198,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("xx")
-        .add("yy")
-        .add("zz")
+        .addRow("xx")
+        .addRow("yy")
+        .addRow("zz")
         .build();
 
     zapOffset(batch, 2, 1);
@@ -222,9 +222,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("xx")
-        .add("yy")
-        .add("zz")
+        .addRow("xx")
+        .addRow("yy")
+        .addRow("zz")
         .build();
 
     zapOffset(batch, 1, 10);
@@ -246,9 +246,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add("xx")
-        .add("yy")
-        .add("zz")
+        .addRow("xx")
+        .addRow("yy")
+        .addRow("zz")
         .build();
 
     zapOffset(batch, 3, 100_000);
@@ -270,9 +270,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add((Object) new String[] {})
-        .add((Object) new String[] {"fred", "barney", "wilma"})
-        .add((Object) new String[] {"dino"})
+        .addRow((Object) new String[] {})
+        .addRow((Object) new String[] {"fred", "barney", "wilma"})
+        .addRow((Object) new String[] {"dino"})
         .build();
 
     VectorAccessible va = batch.vectorAccessible();
@@ -298,9 +298,9 @@ public class TestBatchValidator /* TODO: extends SubOperatorTest */ {
         .build();
 
     SingleRowSet batch = fixture.rowSetBuilder(schema)
-        .add((Object) new String[] {})
-        .add((Object) new String[] {"fred", "barney", "wilma"})
-        .add((Object) new String[] {"dino"})
+        .addRow((Object) new String[] {})
+        .addRow((Object) new String[] {"fred", "barney", "wilma"})
+        .addRow((Object) new String[] {"dino"})
         .build();
 
     VectorAccessible va = batch.vectorAccessible();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
index c52f1a9..563d97e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
@@ -60,7 +60,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 0; i <= record_count; i += 2) {
-        rowSetBuilder.add(i);
+        rowSetBuilder.addRow(i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();
@@ -76,7 +76,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 1; i <= record_count; i += 2) {
-        rowSetBuilder.add((float) i);
+        rowSetBuilder.addRow((float) i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();
@@ -131,7 +131,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 0; i <= record_count; i += 2) {
-        rowSetBuilder.add(i);
+        rowSetBuilder.addRow(i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();
@@ -147,7 +147,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 1; i <= record_count; i += 2) {
-        rowSetBuilder.add(i);
+        rowSetBuilder.addRow(i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();
@@ -199,7 +199,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 0; i <= record_count; i += 2) {
-        rowSetBuilder.add(i, i);
+        rowSetBuilder.addRow(i, i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();
@@ -216,7 +216,7 @@ public class TestExternalSort extends BaseTestQuery {
       final RowSetBuilder rowSetBuilder = new RowSetBuilder(allocator, schema);
 
       for (int i = 1; i <= record_count; i += 2) {
-        rowSetBuilder.add(i, i);
+        rowSetBuilder.addRow(i, i);
       }
 
       final RowSet rowSet = rowSetBuilder.build();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
index c58abd6..cd408cb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -33,12 +33,12 @@ import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrap
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetComparison;
-import org.apache.drill.test.rowSet.RowSetSchema;
 import org.apache.drill.test.rowSet.SchemaBuilder;
 
 import com.google.common.collect.Lists;
@@ -93,7 +93,7 @@ public class SortTestUtilities {
     public void run() throws Exception {
       PriorityQueueCopierWrapper copier = makeCopier(fixture, sortOrder, nullOrder);
       List<BatchGroup> batches = new ArrayList<>();
-      RowSetSchema schema = null;
+      TupleMetadata schema = null;
       for (SingleRowSet rowSet : rowSets) {
         batches.add(new BatchGroup.InputBatch(rowSet.container(), rowSet.getSv2(),
                     fixture.allocator(), rowSet.size()));
@@ -103,7 +103,7 @@ public class SortTestUtilities {
       }
       int rowCount = outputRowCount();
       VectorContainer dest = new VectorContainer();
-      BatchMerger merger = copier.startMerge(schema.toBatchSchema(SelectionVectorMode.NONE),
+      BatchMerger merger = copier.startMerge(new BatchSchema(SelectionVectorMode.NONE, schema.toFieldList()),
                                              batches, dest, rowCount, null);
 
       verifyResults(merger, dest);
@@ -121,7 +121,7 @@ public class SortTestUtilities {
     protected void verifyResults(BatchMerger merger, VectorContainer dest) {
       for (RowSet expectedSet : expected) {
         assertTrue(merger.next());
-        RowSet rowSet = new DirectRowSet(fixture.allocator(), dest);
+        RowSet rowSet = DirectRowSet.fromContainer(dest);
         new RowSetComparison(expectedSet)
               .verifyAndClearAll(rowSet);
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
index f1c622f..5d438ee 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
@@ -29,15 +29,13 @@ import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrap
 import org.apache.drill.exec.physical.impl.xsort.managed.SortTestUtilities.CopierTester;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.SchemaBuilder;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -51,19 +49,7 @@ import org.junit.experimental.categories.Category;
  */
 
 @Category(OperatorTest.class)
-public class TestCopier extends DrillTest {
-
-  public static OperatorFixture fixture;
-
-  @BeforeClass
-  public static void setup() {
-    fixture = OperatorFixture.builder().build();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    fixture.close();
-  }
+public class TestCopier extends SubOperatorTest {
 
   @Test
   public void testEmptyInput() throws Exception {
@@ -101,12 +87,12 @@ public class TestCopier extends DrillTest {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     CopierTester tester = new CopierTester(fixture);
     tester.addInput(fixture.rowSetBuilder(schema)
-          .add(10, "10")
+          .addRow(10, "10")
           .withSv2()
           .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-          .add(10, "10")
+          .addRow(10, "10")
           .build());
     tester.run();
   }
@@ -116,17 +102,17 @@ public class TestCopier extends DrillTest {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     CopierTester tester = new CopierTester(fixture);
     tester.addInput(fixture.rowSetBuilder(schema)
-          .add(10, "10")
+          .addRow(10, "10")
           .withSv2()
           .build());
     tester.addInput(fixture.rowSetBuilder(schema)
-          .add(20, "20")
+          .addRow(20, "20")
           .withSv2()
           .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-          .add(10, "10")
-          .add(20, "20")
+          .addRow(10, "10")
+          .addRow(20, "20")
           .build());
     tester.run();
   }
@@ -137,7 +123,7 @@ public class TestCopier extends DrillTest {
     int value = first;
     for (int i = 0; i < count; i++, value += step) {
       RowSetUtilities.setFromInt(writer, 0, value);
-      writer.column(1).setString(Integer.toString(value));
+      writer.scalar(1).setString(Integer.toString(value));
       writer.save();
     }
     writer.done();
@@ -188,25 +174,25 @@ public class TestCopier extends DrillTest {
     tester.sortOrder = Ordering.ORDER_ASC;
     tester.nullOrder = Ordering.NULLS_LAST;
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(1, "1")
-        .add(4, "4")
-        .add(null, "null")
+        .addRow(1, "1")
+        .addRow(4, "4")
+        .addRow(null, "null")
         .withSv2()
         .build());
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(2, "2")
-        .add(3, "3")
-        .add(null, "null")
+        .addRow(2, "2")
+        .addRow(3, "3")
+        .addRow(null, "null")
         .withSv2()
         .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-        .add(1, "1")
-        .add(2, "2")
-        .add(3, "3")
-        .add(4, "4")
-        .add(null, "null")
-        .add(null, "null")
+        .addRow(1, "1")
+        .addRow(2, "2")
+        .addRow(3, "3")
+        .addRow(4, "4")
+        .addRow(null, "null")
+        .addRow(null, "null")
         .build());
 
     tester.run();
@@ -220,25 +206,25 @@ public class TestCopier extends DrillTest {
     tester.sortOrder = Ordering.ORDER_ASC;
     tester.nullOrder = Ordering.NULLS_FIRST;
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(1, "1")
-        .add(4, "4")
+        .addRow(null, "null")
+        .addRow(1, "1")
+        .addRow(4, "4")
         .withSv2()
         .build());
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(2, "2")
-        .add(3, "3")
+        .addRow(null, "null")
+        .addRow(2, "2")
+        .addRow(3, "3")
         .withSv2()
         .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(null, "null")
-        .add(1, "1")
-        .add(2, "2")
-        .add(3, "3")
-        .add(4, "4")
+        .addRow(null, "null")
+        .addRow(null, "null")
+        .addRow(1, "1")
+        .addRow(2, "2")
+        .addRow(3, "3")
+        .addRow(4, "4")
         .build());
 
     tester.run();
@@ -252,25 +238,25 @@ public class TestCopier extends DrillTest {
     tester.sortOrder = Ordering.ORDER_DESC;
     tester.nullOrder = Ordering.NULLS_LAST;
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(4, "4")
-        .add(1, "1")
-        .add(null, "null")
+        .addRow(4, "4")
+        .addRow(1, "1")
+        .addRow(null, "null")
         .withSv2()
         .build());
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(3, "3")
-        .add(2, "2")
-        .add(null, "null")
+        .addRow(3, "3")
+        .addRow(2, "2")
+        .addRow(null, "null")
         .withSv2()
         .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-        .add(4, "4")
-        .add(3, "3")
-        .add(2, "2")
-        .add(1, "1")
-        .add(null, "null")
-        .add(null, "null")
+        .addRow(4, "4")
+        .addRow(3, "3")
+        .addRow(2, "2")
+        .addRow(1, "1")
+        .addRow(null, "null")
+        .addRow(null, "null")
         .build());
 
     tester.run();
@@ -284,25 +270,25 @@ public class TestCopier extends DrillTest {
     tester.sortOrder = Ordering.ORDER_DESC;
     tester.nullOrder = Ordering.NULLS_FIRST;
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(4, "4")
-        .add(1, "1")
+        .addRow(null, "null")
+        .addRow(4, "4")
+        .addRow(1, "1")
         .withSv2()
         .build());
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(3, "3")
-        .add(2, "2")
+        .addRow(null, "null")
+        .addRow(3, "3")
+        .addRow(2, "2")
         .withSv2()
         .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-        .add(null, "null")
-        .add(null, "null")
-        .add(4, "4")
-        .add(3, "3")
-        .add(2, "2")
-        .add(1, "1")
+        .addRow(null, "null")
+        .addRow(null, "null")
+        .addRow(4, "4")
+        .addRow(3, "3")
+        .addRow(2, "2")
+        .addRow(1, "1")
         .build());
 
     tester.run();
@@ -362,22 +348,22 @@ public class TestCopier extends DrillTest {
 
     CopierTester tester = new CopierTester(fixture);
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(1, 10, 100)
-        .add(5, 50, 500)
+        .addRow(1, new Object[] {10, new Object[] {100}})
+        .addRow(5, new Object[] {50, new Object[] {500}})
         .withSv2()
         .build());
 
     tester.addInput(fixture.rowSetBuilder(schema)
-        .add(2, 20, 200)
-        .add(6, 60, 600)
+        .addRow(2, new Object[] {20, new Object[] {200}})
+        .addRow(6, new Object[] {60, new Object[] {600}})
         .withSv2()
         .build());
 
     tester.addOutput(fixture.rowSetBuilder(schema)
-        .add(1, 10, 100)
-        .add(2, 20, 200)
-        .add(5, 50, 500)
-        .add(6, 60, 600)
+        .addRow(1, new Object[] {10, new Object[] {100}})
+        .addRow(2, new Object[] {20, new Object[] {200}})
+        .addRow(5, new Object[] {50, new Object[] {500}})
+        .addRow(6, new Object[] {60, new Object[] {600}})
         .build());
 
     tester.run();

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestShortArrays.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestShortArrays.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestShortArrays.java
index ba5dfce..38e3698 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestShortArrays.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestShortArrays.java
@@ -61,9 +61,9 @@ public class TestShortArrays extends SubOperatorTest {
         .addArray("b", MinorType.INT)
         .build();
     RowSetBuilder builder = fixture.rowSetBuilder(schema)
-        .add(1, new int[] {10});
+        .addRow(1, new int[] {10});
     for (int i = 2; i <= 10; i++) {
-      builder.add(i, new int[] {});
+      builder.addRow(i, new int[] {});
     }
     RowSet rows = builder.build();
 
@@ -87,9 +87,9 @@ public class TestShortArrays extends SubOperatorTest {
 
     SingleRowSet empty = fixture.rowSet(schema);
     vi.allocateBatch(empty.container(), 100);
-    assertEquals(2, empty.vectors().length);
+    assertEquals(2, empty.container().getNumberOfColumns());
     @SuppressWarnings("resource")
-    ValueVector bVector = empty.vectors()[1];
+    ValueVector bVector = empty.container().getValueVector(1).getValueVector();
     assertTrue(bVector instanceof RepeatedIntVector);
     assertEquals(16, ((RepeatedIntVector) bVector).getDataVector().getValueCapacity());
 

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
index d83a765..93411d7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -46,8 +46,8 @@ import org.apache.drill.test.rowSet.HyperRowSetImpl;
 import org.apache.drill.test.rowSet.IndirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.apache.drill.test.rowSet.SchemaBuilder;
@@ -193,11 +193,11 @@ public class TestSortImpl extends DrillTest {
 
   private static RowSet toRowSet(OperatorFixture fixture, SortResults results, VectorContainer dest) {
     if (results.getSv4() != null) {
-      return new HyperRowSetImpl(fixture.allocator(), dest, results.getSv4());
+      return new HyperRowSetImpl(dest, results.getSv4());
     } else if (results.getSv2() != null) {
-      return new IndirectRowSet(fixture.allocator(), dest, results.getSv2());
+      return IndirectRowSet.fromSv2(dest, results.getSv2());
     } else {
-      return new DirectRowSet(fixture.allocator(), dest);
+      return DirectRowSet.fromContainer(dest);
     }
   }
 
@@ -242,10 +242,10 @@ public class TestSortImpl extends DrillTest {
       BatchSchema schema = SortTestUtilities.nonNullSchema();
       SortTestFixture sortTest = new SortTestFixture(fixture);
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
+          .addRow(1, "first")
           .build());
       sortTest.addOutput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
+          .addRow(1, "first")
           .build());
       sortTest.run();
     }
@@ -262,12 +262,12 @@ public class TestSortImpl extends DrillTest {
       BatchSchema schema = SortTestUtilities.nonNullSchema();
       SortTestFixture sortTest = new SortTestFixture(fixture);
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(2, "second")
-          .add(1, "first")
+          .addRow(2, "second")
+          .addRow(1, "first")
           .build());
       sortTest.addOutput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
-          .add(2, "second")
+          .addRow(1, "first")
+          .addRow(2, "second")
           .build());
       sortTest.run();
     }
@@ -285,14 +285,14 @@ public class TestSortImpl extends DrillTest {
       BatchSchema schema = SortTestUtilities.nonNullSchema();
       SortTestFixture sortTest = new SortTestFixture(fixture);
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(2, "second")
+          .addRow(2, "second")
           .build());
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
+          .addRow(1, "first")
           .build());
       sortTest.addOutput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
-          .add(2, "second")
+          .addRow(1, "first")
+          .addRow(2, "second")
           .build());
       sortTest.run();
     }
@@ -356,7 +356,7 @@ public class TestSortImpl extends DrillTest {
       RowSetBuilder builder = fixture.rowSetBuilder(schema);
       int end = Math.min(batchSize, targetCount - rowCount);
       for (int i = 0; i < end; i++) {
-        builder.add(currentValue, i + ", " + currentValue);
+        builder.addRow(currentValue, i + ", " + currentValue);
         currentValue = (currentValue + step) % targetCount;
         rowCount++;
       }
@@ -387,7 +387,7 @@ public class TestSortImpl extends DrillTest {
       RowSetReader reader = output.reader();
       while (reader.next()) {
         assertEquals("Value of " + batchCount + ":" + rowCount,
-            rowCount, reader.column(0).getInt());
+            rowCount, reader.scalar(0).getInt());
         rowCount++;
       }
     }
@@ -593,18 +593,18 @@ public class TestSortImpl extends DrillTest {
         }
       };
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(2, "second")
+          .addRow(2, "second")
           .build());
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(3, "third")
+          .addRow(3, "third")
           .build());
       sortTest.addInput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
+          .addRow(1, "first")
           .build());
       sortTest.addOutput(fixture.rowSetBuilder(schema)
-          .add(1, "first")
-          .add(2, "second")
-          .add(3, "third")
+          .addRow(1, "first")
+          .addRow(2, "second")
+          .addRow(3, "third")
           .build());
       sortTest.run();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
index 5f04da6..c24f1a6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
@@ -36,8 +36,8 @@ import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSet.RowSetReader;
-import org.apache.drill.test.rowSet.RowSet.RowSetWriter;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.RowSetComparison;
@@ -111,12 +111,12 @@ public class TestSorter extends DrillTest {
   public void testSingleRow() throws Exception {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     SingleRowSet rowSet = new RowSetBuilder(fixture.allocator(), schema)
-          .add(0, "0")
+          .addRow(0, "0")
           .withSv2()
           .build();
 
     SingleRowSet expected = new RowSetBuilder(fixture.allocator(), schema)
-        .add(0, "0")
+        .addRow(0, "0")
         .build();
     runSorterTest(rowSet, expected);
   }
@@ -127,14 +127,14 @@ public class TestSorter extends DrillTest {
   public void testTwoRows() throws Exception {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     SingleRowSet rowSet = new RowSetBuilder(fixture.allocator(), schema)
-        .add(1, "1")
-        .add(0, "0")
+        .addRow(1, "1")
+        .addRow(0, "0")
         .withSv2()
         .build();
 
     SingleRowSet expected = new RowSetBuilder(fixture.allocator(), schema)
-        .add(0, "0")
-        .add(1, "1")
+        .addRow(0, "0")
+        .addRow(1, "1")
         .build();
     runSorterTest(rowSet, expected);
   }
@@ -207,11 +207,11 @@ public class TestSorter extends DrillTest {
       for (int i = 0; i < items.length; i++) {
         DataItem item = items[i];
         if (nullable && item.isNull) {
-          writer.column(0).setNull();
+          writer.scalar(0).setNull();
         } else {
           RowSetUtilities.setFromInt(writer, 0, item.key);
         }
-        writer.column(1).setString(Integer.toString(item.value));
+        writer.scalar(1).setString(Integer.toString(item.value));
         writer.save();
       }
       writer.done();
@@ -221,7 +221,7 @@ public class TestSorter extends DrillTest {
     private void verify(RowSet actual) {
       DataItem expected[] = Arrays.copyOf(data, data.length);
       doSort(expected);
-      RowSet expectedRows = makeDataSet(actual.allocator(), actual.schema().batch(), expected);
+      RowSet expectedRows = makeDataSet(actual.allocator(), actual.batchSchema(), expected);
       doVerify(expected, expectedRows, actual);
     }
 
@@ -369,7 +369,7 @@ public class TestSorter extends DrillTest {
         int mo = rand.nextInt(12);
         int yr = rand.nextInt(10);
         Period period = makePeriod(yr, mo, day, hr, min, sec, ms);
-        builder.add(period);
+        builder.addRow(period);
       }
       return builder.build();
     }
@@ -383,7 +383,7 @@ public class TestSorter extends DrillTest {
       int prevMonths = 0;
       long prevMs = 0;
       while (reader.next()) {
-        Period period = reader.column(0).getPeriod().normalizedStandard();
+        Period period = reader.scalar(0).getPeriod().normalizedStandard();
         int years = period.getYears();
         assertTrue(prevYears <= years);
         if (prevYears != years) {
@@ -586,16 +586,16 @@ public class TestSorter extends DrillTest {
         .build();
 
     SingleRowSet input = fixture.rowSetBuilder(schema)
-        .add(3, "third")
-        .add(1, "first")
-        .add(2, "second")
+        .addRow(3, "third")
+        .addRow(1, "first")
+        .addRow(2, "second")
         .withSv2()
         .build();
 
     SingleRowSet output = fixture.rowSetBuilder(schema)
-        .add(1, "first")
-        .add(2, "second")
-        .add(3, "third")
+        .addRow(1, "first")
+        .addRow(2, "second")
+        .addRow(3, "third")
         .build();
     Sort popConfig = makeSortConfig("map.key", Ordering.ORDER_ASC, Ordering.NULLS_LAST);
     runSorterTest(popConfig, input, output);

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderLimits.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderLimits.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderLimits.java
new file mode 100644
index 0000000..f9f5128
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderLimits.java
@@ -0,0 +1,224 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.*;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.physical.rowSet.impl.ResultSetLoaderImpl.ResultSetOptions;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Tests of the row limit functionality of the result set loader. The
+ * row limit is set up front and has a default value. Because Drill must
+ * discover data structure as it reads, the result set loader also allows changing
+ * the row limit between batches (perhaps Drill discovers that rows are much
+ * narrower or wider than expected.)
+ * <p>
+ * The tests here are independent of the tests for vector allocation (which does,
+ * in fact, depend on the row count) and vector overflow (which an occur when
+ * the row limit turns out to be too large.)
+ */
+
+public class TestResultSetLoaderLimits extends SubOperatorTest {
+
+  /**
+   * Verify that the writer stops when reaching the row limit.
+   * In this case there is no look-ahead row.
+   */
+
+  @Test
+  public void testRowLimit() {
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator());
+    assertEquals(ResultSetLoaderImpl.DEFAULT_ROW_COUNT, rsLoader.targetRowCount());
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("s", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    byte value[] = new byte[200];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+    assertEquals(ResultSetLoaderImpl.DEFAULT_ROW_COUNT, count);
+    assertEquals(count, rootWriter.rowCount());
+
+    rsLoader.harvest().clear();
+
+    // Do it again, a different way.
+
+    count = 0;
+    rsLoader.startBatch();
+    assertEquals(0, rootWriter.rowCount());
+    while (rootWriter.start()) {
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+    assertEquals(ResultSetLoaderImpl.DEFAULT_ROW_COUNT, count);
+    assertEquals(count, rootWriter.rowCount());
+
+    rsLoader.harvest().clear();
+
+    rsLoader.close();
+  }
+
+  private static final int TEST_ROW_LIMIT = 1024;
+
+  /**
+   * Verify that the caller can set a row limit lower than the default.
+   */
+
+  @Test
+  public void testCustomRowLimit() {
+
+    // Try to set a default value larger than the hard limit. Value
+    // is truncated to the limit.
+
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT + 1)
+        .build();
+    assertEquals(ValueVector.MAX_ROW_COUNT, options.rowCountLimit);
+
+    // Just a bit of paranoia that we check against the vector limit,
+    // not any previous value...
+
+    options = new OptionBuilder()
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT + 1)
+        .setRowCountLimit(TEST_ROW_LIMIT)
+        .build();
+    assertEquals(TEST_ROW_LIMIT, options.rowCountLimit);
+
+    options = new OptionBuilder()
+        .setRowCountLimit(TEST_ROW_LIMIT)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT + 1)
+        .build();
+    assertEquals(ValueVector.MAX_ROW_COUNT, options.rowCountLimit);
+
+    // Can't set the limit lower than 1
+
+    options = new OptionBuilder()
+        .setRowCountLimit(0)
+        .build();
+    assertEquals(1, options.rowCountLimit);
+
+    // Do load with a (valid) limit lower than the default.
+
+    options = new OptionBuilder()
+        .setRowCountLimit(TEST_ROW_LIMIT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(TEST_ROW_LIMIT, rsLoader.targetRowCount());
+
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("s", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rsLoader.startBatch();
+    int count = fillToLimit(rootWriter);
+    assertEquals(TEST_ROW_LIMIT, count);
+    assertEquals(count, rootWriter.rowCount());
+
+    // Should fail to write beyond the row limit
+
+    assertFalse(rootWriter.start());
+    try {
+      rootWriter.save();
+      fail();
+    } catch (IllegalStateException e) {
+      // Expected
+    }
+
+    rsLoader.harvest().clear();
+    rsLoader.startBatch();
+    assertEquals(0, rootWriter.rowCount());
+
+    rsLoader.close();
+  }
+
+  private int fillToLimit(RowSetLoader rootWriter) {
+    byte value[] = new byte[200];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    while (! rootWriter.isFull()) {
+      rootWriter.start();
+      rootWriter.scalar(0).setBytes(value, value.length);
+      rootWriter.save();
+      count++;
+    }
+    return count;
+  }
+
+  /**
+   * Test that the row limit can change between batches.
+   */
+
+  @Test
+  public void testDynamicLimit() {
+
+    // Start with a small limit.
+
+    ResultSetOptions options = new OptionBuilder()
+        .setRowCountLimit(TEST_ROW_LIMIT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    assertEquals(TEST_ROW_LIMIT, rsLoader.targetRowCount());
+
+    RowSetLoader rootWriter = rsLoader.writer();
+    rootWriter.addColumn(SchemaBuilder.columnSchema("s", MinorType.VARCHAR, DataMode.REQUIRED));
+
+    rsLoader.startBatch();
+    int count = fillToLimit(rootWriter);
+    assertEquals(TEST_ROW_LIMIT, count);
+    assertEquals(count, rootWriter.rowCount());
+    rsLoader.harvest().clear();
+
+    // Reset the batch size larger and fill a second batch
+
+    int newLimit = 8000;
+    rsLoader.setTargetRowCount(newLimit);
+    rsLoader.startBatch();
+    count = fillToLimit(rootWriter);
+    assertEquals(newLimit, count);
+    assertEquals(count, rootWriter.rowCount());
+    rsLoader.harvest().clear();
+
+    // Put the limit back to a lower number.
+
+    newLimit = 1000;
+    rsLoader.setTargetRowCount(newLimit);
+    rsLoader.startBatch();
+    count = fillToLimit(rootWriter);
+    assertEquals(newLimit, count);
+    assertEquals(count, rootWriter.rowCount());
+    rsLoader.harvest().clear();
+
+    rsLoader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
new file mode 100644
index 0000000..115e52d
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/rowSet/impl/TestResultSetLoaderMapArray.java
@@ -0,0 +1,481 @@
+/*
+ * 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.physical.rowSet.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.ResultSetLoader;
+import org.apache.drill.exec.physical.rowSet.RowSetLoader;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ArrayReader;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ScalarElementReader;
+import org.apache.drill.exec.vector.accessor.ScalarReader;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleReader;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.test.SubOperatorTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.apache.drill.test.rowSet.RowSetReader;
+import org.apache.drill.test.rowSet.SchemaBuilder;
+import org.junit.Test;
+
+/**
+ * Test map array support in the result set loader.
+ * <p>
+ * The tests here should be considered in the "extra for experts"
+ * category: run and/or debug these tests only after the scalar
+ * tests work. Maps, and especially repeated maps, are very complex
+ * constructs not to be tackled lightly.
+ */
+
+public class TestResultSetLoaderMapArray extends SubOperatorTest {
+
+  @Test
+  public void testBasics() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m")
+          .add("c", MinorType.INT)
+          .add("d", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Verify structure and schema
+
+    TupleMetadata actualSchema = rootWriter.schema();
+    assertEquals(2, actualSchema.size());
+    assertTrue(actualSchema.metadata(1).isArray());
+    assertTrue(actualSchema.metadata(1).isMap());
+    assertEquals(2, actualSchema.metadata("m").mapSchema().size());
+    assertEquals(2, actualSchema.column("m").getChildren().size());
+
+    // Write a couple of rows with arrays.
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(10, new Object[] {
+          new Object[] {110, "d1.1"},
+          new Object[] {120, "d2.2"}})
+      .addRow(20, new Object[] {})
+      .addRow(30, new Object[] {
+          new Object[] {310, "d3.1"},
+          new Object[] {320, "d3.2"},
+          new Object[] {330, "d3.3"}})
+      ;
+
+    // Verify the first batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {
+            new Object[] {110, "d1.1"},
+            new Object[] {120, "d2.2"}})
+        .addRow(20, new Object[] {})
+        .addRow(30, new Object[] {
+            new Object[] {310, "d3.1"},
+            new Object[] {320, "d3.2"},
+            new Object[] {330, "d3.3"}})
+        .build();
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    // In the second, create a row, then add a map member.
+    // Should be back-filled to empty for the first row.
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(40, new Object[] {
+          new Object[] {410, "d4.1"},
+          new Object[] {420, "d4.2"}});
+
+    TupleWriter mapWriter = rootWriter.array("m").tuple();
+    mapWriter.addColumn(SchemaBuilder.columnSchema("e", MinorType.VARCHAR, DataMode.OPTIONAL));
+
+    rootWriter
+      .addRow(50, new Object[] {
+          new Object[] {510, "d5.1", "e5.1"},
+          new Object[] {520, "d5.2", null}})
+      .addRow(60, new Object[] {
+          new Object[] {610, "d6.1", "e6.1"},
+          new Object[] {620, "d6.2", null},
+          new Object[] {630, "d6.3", "e6.3"}})
+      ;
+
+    // Verify the second batch
+
+    actual = fixture.wrap(rsLoader.harvest());
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m")
+          .add("c", MinorType.INT)
+          .add("d", MinorType.VARCHAR)
+          .addNullable("e", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(40, new Object[] {
+            new Object[] {410, "d4.1", null},
+            new Object[] {420, "d4.2", null}})
+        .addRow(50, new Object[] {
+            new Object[] {510, "d5.1", "e5.1"},
+            new Object[] {520, "d5.2", null}})
+        .addRow(60, new Object[] {
+            new Object[] {610, "d6.1", "e6.1"},
+            new Object[] {620, "d6.2", null},
+            new Object[] {630, "d6.3", "e6.3"}})
+        .build();
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  @Test
+  public void testNestedArray() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m")
+          .add("c", MinorType.INT)
+          .addArray("d", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Write a couple of rows with arrays within arrays.
+    // (And, of course, the Varchar is actually an array of
+    // bytes, so that's three array levels.)
+
+    rsLoader.startBatch();
+    rootWriter
+      .addRow(10, new Object[] {
+          new Object[] {110, new String[] {"d1.1.1", "d1.1.2"}},
+          new Object[] {120, new String[] {"d1.2.1", "d1.2.2"}}})
+      .addRow(20, new Object[] {})
+      .addRow(30, new Object[] {
+          new Object[] {310, new String[] {"d3.1.1", "d3.2.2"}},
+          new Object[] {320, new String[] {}},
+          new Object[] {330, new String[] {"d3.3.1", "d1.2.2"}}})
+      ;
+
+    // Verify the batch
+
+    RowSet actual = fixture.wrap(rsLoader.harvest());
+    SingleRowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(10, new Object[] {
+            new Object[] {110, new String[] {"d1.1.1", "d1.1.2"}},
+            new Object[] {120, new String[] {"d1.2.1", "d1.2.2"}}})
+        .addRow(20, new Object[] {})
+        .addRow(30, new Object[] {
+            new Object[] {310, new String[] {"d3.1.1", "d3.2.2"}},
+            new Object[] {320, new String[] {}},
+            new Object[] {330, new String[] {"d3.3.1", "d1.2.2"}}})
+        .build();
+    new RowSetComparison(expected).verifyAndClearAll(actual);
+
+    rsLoader.close();
+  }
+
+  /**
+   * Test a doubly-nested arrays of maps.
+   */
+
+  @Test
+  public void testDoubleNestedArray() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m1")
+          .add("b", MinorType.INT)
+          .addMapArray("m2")
+            .add("c", MinorType.INT)
+            .addArray("d", MinorType.VARCHAR)
+            .buildMap()
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+    rsLoader.startBatch();
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    ArrayWriter a1Writer = rootWriter.array("m1");
+    TupleWriter m1Writer = a1Writer.tuple();
+    ScalarWriter bWriter = m1Writer.scalar("b");
+    ArrayWriter a2Writer = m1Writer.array("m2");
+    TupleWriter m2Writer = a2Writer.tuple();
+    ScalarWriter cWriter = m2Writer.scalar("c");
+    ScalarWriter dWriter = m2Writer.array("d").scalar();
+
+    for (int i = 0; i < 5; i++) {
+      rootWriter.start();
+      aWriter.setInt(i);
+      for (int j = 0; j < 4; j++) {
+        int a1Key = i + 10 + j;
+        bWriter.setInt(a1Key);
+        for (int k = 0; k < 3; k++) {
+          int a2Key = a1Key * 10 + k;
+          cWriter.setInt(a2Key);
+          for (int l = 0; l < 2; l++) {
+            dWriter.setString("d-" + (a2Key * 10 + l));
+          }
+          a2Writer.save();
+        }
+        a1Writer.save();
+      }
+      rootWriter.save();
+    }
+
+    RowSet results = fixture.wrap(rsLoader.harvest());
+    RowSetReader reader = results.reader();
+
+    ScalarReader aReader = reader.scalar("a");
+    ArrayReader a1Reader = reader.array("m1");
+    TupleReader m1Reader = a1Reader.tuple();
+    ScalarReader bReader = m1Reader.scalar("b");
+    ArrayReader a2Reader = m1Reader.array("m2");
+    TupleReader m2Reader = a2Reader.tuple();
+    ScalarReader cReader = m2Reader.scalar("c");
+    ScalarElementReader dReader = m2Reader.elements("d");
+
+    for (int i = 0; i < 5; i++) {
+      reader.next();
+      assertEquals(i, aReader.getInt());
+      for (int j = 0; j < 4; j++) {
+        a1Reader.setPosn(j);
+        int a1Key = i + 10 + j;
+        assertEquals(a1Key, bReader.getInt());
+        for (int k = 0; k < 3; k++) {
+          a2Reader.setPosn(k);
+          int a2Key = a1Key * 10 + k;
+          assertEquals(a2Key, cReader.getInt());
+          for (int l = 0; l < 2; l++) {
+            assertEquals("d-" + (a2Key * 10 + l), dReader.getString(l));
+          }
+        }
+      }
+    }
+    rsLoader.close();
+  }
+
+  /**
+   * Version of the {#link TestResultSetLoaderProtocol#testOverwriteRow()} test
+   * that uses nested columns inside an array of maps. Here we must call
+   * <tt>start()</tt> to reset the array back to the initial start position after
+   * each "discard."
+   */
+
+  @Test
+  public void testOverwriteRow() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addMapArray("m")
+          .add("b", MinorType.INT)
+          .add("c", MinorType.VARCHAR)
+        .buildMap()
+      .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    // Can't use the shortcut to populate rows when doing overwrites.
+
+    ScalarWriter aWriter = rootWriter.scalar("a");
+    ArrayWriter maWriter = rootWriter.array("m");
+    TupleWriter mWriter = maWriter.tuple();
+    ScalarWriter bWriter = mWriter.scalar("b");
+    ScalarWriter cWriter = mWriter.scalar("c");
+
+    // Write 100,000 rows, overwriting 99% of them. This will cause vector
+    // overflow and data corruption if overwrite does not work; but will happily
+    // produce the correct result if everything works as it should.
+
+    byte value[] = new byte[512];
+    Arrays.fill(value, (byte) 'X');
+    int count = 0;
+    rsLoader.startBatch();
+    while (count < 10_000) {
+      rootWriter.start();
+      count++;
+      aWriter.setInt(count);
+      for (int i = 0; i < 10; i++) {
+        bWriter.setInt(count * 10 + i);
+        cWriter.setBytes(value, value.length);
+        maWriter.save();
+      }
+      if (count % 100 == 0) {
+        rootWriter.save();
+      }
+    }
+
+    // Verify using a reader.
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(count / 100, result.rowCount());
+    RowSetReader reader = result.reader();
+    ArrayReader maReader = reader.array("m");
+    TupleReader mReader = maReader.tuple();
+    int rowId = 1;
+    while (reader.next()) {
+      assertEquals(rowId * 100, reader.scalar("a").getInt());
+      assertEquals(10, maReader.size());
+      for (int i = 0; i < 10; i++) {
+        maReader.setPosn(i);
+        assertEquals(rowId * 1000 + i, mReader.scalar("b").getInt());
+        assertTrue(Arrays.equals(value, mReader.scalar("c").getBytes()));
+      }
+      rowId++;
+    }
+
+    result.clear();
+    rsLoader.close();
+  }
+
+  /**
+   * Check that the "fill-empties" logic descends down into
+   * a repeated map.
+   */
+
+  @Test
+  public void testOmittedValues() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("id", MinorType.INT)
+        .addMapArray("m")
+          .addNullable("a", MinorType.INT)
+          .addNullable("b", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    int mapSkip = 5;
+    int entrySkip = 3;
+    int rowCount = 1000;
+    int entryCount = 10;
+
+    rsLoader.startBatch();
+    ArrayWriter maWriter = rootWriter.array("m");
+    TupleWriter mWriter = maWriter.tuple();
+    for (int i = 0; i < rowCount; i++) {
+      rootWriter.start();
+      rootWriter.scalar(0).setInt(i);
+      if (i % mapSkip != 0) {
+        for (int j = 0; j < entryCount; j++) {
+          if (j % entrySkip != 0) {
+            mWriter.scalar(0).setInt(i * entryCount + j);
+            mWriter.scalar(1).setString("b-" + i + "." + j);
+          }
+          maWriter.save();
+        }
+      }
+      rootWriter.save();
+    }
+
+    RowSet result = fixture.wrap(rsLoader.harvest());
+    assertEquals(rowCount, result.rowCount());
+    RowSetReader reader = result.reader();
+    ArrayReader maReader = reader.array("m");
+    TupleReader mReader = maReader.tuple();
+    for (int i = 0; i < rowCount; i++) {
+      assertTrue(reader.next());
+      assertEquals(i, reader.scalar(0).getInt());
+      if (i % mapSkip == 0) {
+        assertEquals(0, maReader.size());
+        continue;
+      }
+      assertEquals(entryCount, maReader.size());
+      for (int j = 0; j < entryCount; j++) {
+        maReader.setPosn(j);
+        if (j % entrySkip == 0) {
+          assertTrue(mReader.scalar(0).isNull());
+          assertTrue(mReader.scalar(1).isNull());
+        } else {
+          assertFalse(mReader.scalar(0).isNull());
+          assertFalse(mReader.scalar(1).isNull());
+          assertEquals(i * entryCount + j, mReader.scalar(0).getInt());
+          assertEquals("b-" + i + "." + j, mReader.scalar(1).getString());
+        }
+      }
+    }
+    result.clear();
+    rsLoader.close();
+  }
+
+  /**
+   * Test that memory is released if the loader is closed with an active
+   * batch (that is, before the batch is harvested.)
+   */
+
+  @Test
+  public void testCloseWithoutHarvest() {
+    TupleMetadata schema = new SchemaBuilder()
+        .addMapArray("m")
+          .add("a", MinorType.INT)
+          .add("b", MinorType.VARCHAR)
+          .buildMap()
+        .buildSchema();
+    ResultSetLoaderImpl.ResultSetOptions options = new OptionBuilder()
+        .setSchema(schema)
+        .setRowCountLimit(ValueVector.MAX_ROW_COUNT)
+        .build();
+    ResultSetLoader rsLoader = new ResultSetLoaderImpl(fixture.allocator(), options);
+    RowSetLoader rootWriter = rsLoader.writer();
+
+    ArrayWriter maWriter = rootWriter.array("m");
+    TupleWriter mWriter = maWriter.tuple();
+    rsLoader.startBatch();
+    for (int i = 0; i < 40; i++) {
+      rootWriter.start();
+      for (int j = 0; j < 3; j++) {
+        mWriter.scalar("a").setInt(i);
+        mWriter.scalar("b").setString("b-" + i);
+        maWriter.save();
+      }
+      rootWriter.save();
+    }
+
+    // Don't harvest the batch. Allocator will complain if the
+    // loader does not release memory.
+
+    rsLoader.close();
+  }
+}


[02/15] drill git commit: DRILL-5657: Size-aware vector writer structure

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
new file mode 100644
index 0000000..1fd12f2
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/AbstractTupleWriter.java
@@ -0,0 +1,450 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ObjectType;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * 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.
+ * <p>
+ * A tuple maintains an internal state needed to handle dynamic column additions.
+ * The state identifies the amount of "catch up" needed to get the new column into
+ * the same state as the existing columns. The state is also handy for understanding
+ * the tuple lifecycle. This lifecycle works for all three cases of:
+ * <ul>
+ * <li>Top-level tuple (row).</li>
+ * <li>Nested tuple (map).</li>
+ * <li>Array of tuples (repeated map).</li>
+ * </ul>
+ *
+ * Specifically, the transitions, for batch, row and array events, are:
+ *
+ * <table border=1>
+ * <tr><th>Public API</th><th>Tuple Event</th><th>State Transition</th>
+ *     <th>Child Event</th></tr>
+ * <tr><td>(Start state)</td>
+ *     <td>&mdash;</td>
+ *     <td>IDLE</td>
+ *     <td>&mdash;</td></tr>
+ * <tr><td>startBatch()</td>
+ *     <td>startWrite()</td>
+ *     <td>IDLE &rarr; IN_WRITE</td>
+ *     <td>startWrite()</td></tr>
+ * <tr><td>start() (new row)</td>
+ *     <td>startRow()</td>
+ *     <td>IN_WRITE &rarr; IN_ROW</td>
+ *     <td>startRow()</td></tr>
+ * <tr><td>start() (without save)</td>
+ *     <td>restartRow()</td>
+ *     <td>IN_ROW &rarr; IN_ROW</td>
+ *     <td>restartRow()</td></tr>
+ * <tr><td>save() (array)</td>
+ *     <td>saveValue()</td>
+ *     <td>IN_ROW &rarr; IN_ROW</td>
+ *     <td>saveValue()</td></tr>
+ * <tr><td rowspan=2>save() (row)</td>
+ *     <td>saveValue()</td>
+ *     <td>IN_ROW &rarr; IN_ROW</td>
+ *     <td>saveValue()</td></tr>
+ * <tr><td>saveRow()</td>
+ *     <td>IN_ROW &rarr; IN_WRITE</td>
+ *     <td>saveRow()</td></tr>
+ * <tr><td rowspan=2>end batch</td>
+ *     <td>&mdash;</td>
+ *     <td>IN_ROW &rarr; IDLE</td>
+ *     <td>endWrite()</td></tr>
+ * <tr><td>&mdash;</td>
+ *     <td>IN_WRITE &rarr; IDLE</td>
+ *     <td>endWrite()</td></tr>
+ * </table>
+ *
+ * Notes:
+ * <ul>
+ * <li>For the top-level tuple, a special case occurs with ending a batch. (The
+ *     method for doing so differs depending on implementation.) If a row is active,
+ *     then that row's values are discarded. Then, the batch is ended.</li>
+ * </ul>
+ */
+
+public abstract class AbstractTupleWriter implements TupleWriter, WriterEvents {
+
+  /**
+   * Generic object wrapper for the tuple writer.
+   */
+
+  public static class TupleObjectWriter extends AbstractObjectWriter {
+
+    private AbstractTupleWriter tupleWriter;
+
+    public TupleObjectWriter(ColumnMetadata schema, AbstractTupleWriter tupleWriter) {
+      super(schema);
+      this.tupleWriter = tupleWriter;
+    }
+
+    @Override
+    public ObjectType type() { return ObjectType.TUPLE; }
+
+    @Override
+    public void set(Object value) { tupleWriter.setObject(value); }
+
+    @Override
+    public TupleWriter tuple() { return tupleWriter; }
+
+    @Override
+    public WriterEvents events() { return tupleWriter; }
+
+    @Override
+    public void bindListener(TupleWriterListener listener) {
+      tupleWriter.bindListener(listener);
+    }
+
+    @Override
+    public void dump(HierarchicalFormatter format) {
+      format
+        .startObject(this)
+        .attribute("tupleWriter");
+      tupleWriter.dump(format);
+      format.endObject();
+    }
+  }
+
+  /**
+   * Tracks the write state of the tuple to allow applying the correct
+   * operations to newly-added columns to synchronize them with the rest
+   * of the tuple.
+   */
+
+  public enum State {
+    /**
+     * No write is in progress. Nothing need be done to newly-added
+     * writers.
+     */
+    IDLE,
+
+    /**
+     * <tt>startWrite()</tt> has been called to start a write operation
+     * (start a batch), but <tt>startValue()</tt> has not yet been called
+     * to start a row (or value within an array). <tt>startWrite()</tt> must
+     * be called on newly added columns.
+     */
+
+    IN_WRITE,
+
+    /**
+     * Both <tt>startWrite()</tt> and <tt>startValue()</tt> has been called on
+     * the tuple to prepare for writing values, and both must be called on
+     * newly-added vectors.
+     */
+
+    IN_ROW
+  }
+
+  protected final TupleMetadata schema;
+  protected final List<AbstractObjectWriter> writers;
+  protected ColumnWriterIndex vectorIndex;
+  protected ColumnWriterIndex childIndex;
+  protected TupleWriterListener listener;
+  protected State state = State.IDLE;
+
+  protected AbstractTupleWriter(TupleMetadata schema, List<AbstractObjectWriter> writers) {
+    this.schema = schema;
+    this.writers = writers;
+  }
+
+  protected AbstractTupleWriter(TupleMetadata schema) {
+    this(schema, new ArrayList<AbstractObjectWriter>());
+  }
+
+  protected void bindIndex(ColumnWriterIndex index, ColumnWriterIndex childIndex) {
+    vectorIndex = index;
+    this.childIndex = childIndex;
+
+    for (int i = 0; i < writers.size(); i++) {
+      writers.get(i).events().bindIndex(childIndex);
+    }
+  }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) {
+    bindIndex(index, index);
+  }
+
+  @Override
+  public ColumnWriterIndex writerIndex() { return vectorIndex; }
+
+  /**
+   * Add a column writer to an existing tuple writer. Used for implementations
+   * that support "live" schema evolution: column discovery while writing.
+   * The corresponding metadata must already have been added to the schema.
+   *
+   * @param colWriter the column writer to add
+   */
+
+  public int addColumnWriter(AbstractObjectWriter colWriter) {
+    assert writers.size() == schema.size();
+    int colIndex = schema.addColumn(colWriter.schema());
+    writers.add(colWriter);
+    colWriter.events().bindIndex(childIndex);
+    if (state != State.IDLE) {
+      colWriter.events().startWrite();
+      if (state == State.IN_ROW) {
+        colWriter.events().startRow();
+      }
+    }
+    return colIndex;
+  }
+
+  @Override
+  public int addColumn(ColumnMetadata column) {
+    if (listener == null) {
+      throw new UnsupportedOperationException("addColumn");
+    }
+    AbstractObjectWriter colWriter = (AbstractObjectWriter) listener.addColumn(this, column);
+    return addColumnWriter(colWriter);
+  }
+
+  @Override
+  public int addColumn(MaterializedField field) {
+    if (listener == null) {
+      throw new UnsupportedOperationException("addColumn");
+    }
+    AbstractObjectWriter colWriter = (AbstractObjectWriter) listener.addColumn(this, field);
+    return addColumnWriter(colWriter);
+  }
+
+  @Override
+  public TupleMetadata schema() { return schema; }
+
+  @Override
+  public int size() { return schema().size(); }
+
+  @Override
+  public void startWrite() {
+    assert state == State.IDLE;
+    state = State.IN_WRITE;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().startWrite();
+    }
+  }
+
+  @Override
+  public void startRow() {
+    // Must be in a write. Can start a row only once.
+    // To restart, call restartRow() instead.
+
+    assert state == State.IN_WRITE;
+    state = State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().startRow();
+    }
+  }
+
+  @Override
+  public void endArrayValue() {
+    assert state == State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().endArrayValue();
+    }
+  }
+
+  @Override
+  public void restartRow() {
+
+    // Rewind is normally called only when a value is active: it resets
+    // pointers to allow rewriting the value. However, if this tuple
+    // is nested in an array, then the array entry could have been
+    // saved (state here is IN_WRITE), but the row as a whole has
+    // not been saved. Thus, we must also allow a rewind() while in
+    // the IN_WRITE state to set the pointers back to the start of
+    // the current row.
+
+    assert state == State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().restartRow();
+    }
+  }
+
+  @Override
+  public void saveRow() {
+    assert state == State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().saveRow();
+    }
+    state = State.IN_WRITE;
+  }
+
+  @Override
+  public void preRollover() {
+
+    // Rollover can only happen while a row is in progress.
+
+    assert state == State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().preRollover();
+    }
+  }
+
+  @Override
+  public void postRollover() {
+
+    // Rollover can only happen while a row is in progress.
+
+    assert state == State.IN_ROW;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().postRollover();
+    }
+  }
+
+  @Override
+  public void endWrite() {
+    assert state != State.IDLE;
+    for (int i = 0; i < writers.size();  i++) {
+      writers.get(i).events().endWrite();
+    }
+    state = State.IDLE;
+  }
+
+  @Override
+  public ObjectWriter column(int colIndex) {
+    return writers.get(colIndex);
+  }
+
+  @Override
+  public ObjectWriter column(String colName) {
+    int index = schema.index(colName);
+    if (index == -1) {
+      throw new UndefinedColumnException(colName);
+    }
+    return writers.get(index);
+  }
+
+  @Override
+  public void set(int colIndex, Object value) {
+    ObjectWriter colWriter = column(colIndex);
+    switch (colWriter.type()) {
+    case ARRAY:
+      colWriter.array().setObject(value);
+      break;
+    case SCALAR:
+      colWriter.scalar().setObject(value);
+      break;
+    case TUPLE:
+      colWriter.tuple().setObject(value);
+      break;
+    default:
+      throw new IllegalStateException("Unexpected object type: " + colWriter.type());
+    }
+  }
+
+  @Override
+  public void setTuple(Object ...values) {
+    setObject(values);
+  }
+
+  @Override
+  public void setObject(Object value) {
+    Object values[] = (Object[]) value;
+    if (values.length != schema.size()) {
+      throw new IllegalArgumentException(
+          "Map has " + schema.size() +
+          " columns, but value array has " +
+          values.length + " values.");
+    }
+    for (int i = 0; i < values.length; i++) {
+      set(i, values[i]);
+    }
+  }
+
+  @Override
+  public ScalarWriter scalar(int colIndex) {
+    return column(colIndex).scalar();
+  }
+
+  @Override
+  public ScalarWriter scalar(String colName) {
+    return column(colName).scalar();
+  }
+
+  @Override
+  public TupleWriter tuple(int colIndex) {
+    return column(colIndex).tuple();
+  }
+
+  @Override
+  public TupleWriter tuple(String colName) {
+    return column(colName).tuple();
+  }
+
+  @Override
+  public ArrayWriter array(int colIndex) {
+    return column(colIndex).array();
+  }
+
+  @Override
+  public ArrayWriter array(String colName) {
+    return column(colName).array();
+  }
+
+  @Override
+  public ObjectType type(int colIndex) {
+    return column(colIndex).type();
+  }
+
+  @Override
+  public ObjectType type(String colName) {
+    return column(colName).type();
+  }
+
+  @Override
+  public int lastWriteIndex() {
+    return vectorIndex.vectorIndex();
+  }
+
+  @Override
+  public void bindListener(TupleWriterListener listener) {
+    this.listener = listener;
+  }
+
+  public void dump(HierarchicalFormatter format) {
+    format
+      .startObject(this)
+      .attribute("vectorIndex", vectorIndex)
+      .attribute("state", state)
+      .attributeArray("writers");
+    for (int i = 0; i < writers.size(); i++) {
+      format.element(i);
+      writers.get(i).dump(format);
+    }
+    format
+      .endArray()
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseScalarWriter.java
new file mode 100644
index 0000000..4793277
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseScalarWriter.java
@@ -0,0 +1,272 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.joda.time.Period;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * 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).
+ * <p>
+ * The only tricky part to this class is understanding the
+ * state of the write indexes as the write proceeds. There are
+ * two pointers to consider:
+ * <ul>
+ * <li>lastWriteIndex: The position in the vector at which the
+ * client last asked us to write data. This index is maintained
+ * in this class because it depends only on the actions of this
+ * class.</li>
+ * <li>vectorIndex: The position in the vector at which we will
+ * write if the client chooses to write a value at this time.
+ * The vector index is shared by all columns at the same repeat
+ * level. It is incremented as the client steps through the write
+ * and is observed in this class each time a write occurs.</i>
+ * </ul>
+ * A repeat level is defined as any of the following:
+ * <ul>
+ * <li>The set of top-level scalar columns, or those within a
+ * top-level, non-repeated map, or nested to any depth within
+ * non-repeated maps rooted at the top level.</li>
+ * <li>The values for a single scalar array.</li>
+ * <li>The set of scalar columns within a repeated map, or
+ * nested within non-repeated maps within a repeated map.</li>
+ * </ul>
+ * Items at a repeat level index together and share a vector
+ * index. However, the columns within a repeat level
+ * <i>do not</i> share a last write index: some can lag further
+ * behind than others.
+ * <p>
+ * Let's illustrate the states. Let's focus on one column and
+ * illustrate the three states that can occur during write:
+ * <ul>
+ * <li><b>Behind</b>: the last write index is more than one position behind
+ * the vector index. Zero-filling will be needed to catch up to
+ * the vector index.</li>
+ * <li><b>Written</b>: the last write index is the same as the vector
+ * index because the client wrote data at this position (and previous
+ * values were back-filled with nulls, empties or zeros.)</li>
+ * <li><b>Unwritten</b>: the last write index is one behind the vector
+ * index. This occurs when the column was written, then the client
+ * moved to the next row or array position.</li>
+ * <li><b>Restarted</b>: The current row is abandoned (perhaps filtered
+ * out) and is to be rewritten. The last write position moves
+ * back one position. Note that, the Restarted state is
+ * indistinguishable from the unwritten state: the only real
+ * difference is that the current slot (pointed to by the
+ * vector index) contains the previous written value that must
+ * be overwritten or back-filled. But, this is fine, because we
+ * assume that unwritten values are garbage anyway.</li>
+ * </ul>
+ * To illustrate:<pre><code>
+ *      Behind      Written    Unwritten    Restarted
+ *       |X|          |X|         |X|          |X|
+ *   lw >|X|          |X|         |X|          |X|
+ *       | |          |0|         |0|     lw > |0|
+ *    v >| |  lw, v > |X|    lw > |X|      v > |X|
+ *                            v > | |
+ * </code></pre>
+ * The illustrated state transitions are:
+ * <ul>
+ * <li>Suppose the state starts in Behind.<ul>
+ *   <li>If the client writes a value, then the empty slot is
+ *       back-filled and the state moves to Written.</li>
+ *   <li>If the client does not write a value, the state stays
+ *       at Behind, and the gap of unfilled values grows.</li></ul></li>
+ * <li>When in the Written state:<ul>
+ *   <li>If the client saves the current row or array position,
+ *       the vector index increments and we move to the Unwritten
+ *       state.</li>
+ *   <li>If the client abandons the row, the last write position
+ *       moves back one to recreate the unwritten state. We've
+ *       shown this state separately above just to illustrate
+ *       the two transitions from Written.</li></ul></li>
+ * <li>When in the Unwritten (or Restarted) states:<ul>
+ *   <li>If the client writes a value, then the writer moves back to the
+ *       Written state.</li>
+ *   <li>If the client skips the value, then the vector index increments
+ *       again, leaving a gap, and the writer moves to the
+ *       Behind state.</li></ul>
+ * </ul>
+ * <p>
+ * We've already noted that the Restarted state is identical to
+ * the Unwritten state (and was discussed just to make the flow a bit
+ * clearer.) The astute reader will have noticed that the Behind state is
+ * the same as the Unwritten state if we define the combined state as
+ * when the last write position is behind the vector index.
+ * <p>
+ * Further, if
+ * one simply treats the gap between last write and the vector indexes
+ * as the amount (which may be zero) to back-fill, then there is just
+ * one state. This is, in fact, how the code works: it always writes
+ * to the vector index (and can do so multiple times for a single row),
+ * back-filling as necessary.
+ * <p>
+ * The states, then, are more for our use in understanding the algorithm.
+ * They are also very useful when working through the logic of performing
+ * a roll-over when a vector overflows.
+ */
+
+public abstract class BaseScalarWriter extends AbstractScalarWriter {
+
+  public static final int MIN_BUFFER_SIZE = 256;
+
+  /**
+   * Indicates the position in the vector to write. Set via an object so that
+   * all writers (within the same subtree) can agree on the write position.
+   * For example, all top-level, simple columns see the same row index.
+   * All columns within a repeated map see the same (inner) index, etc.
+   */
+
+  protected ColumnWriterIndex vectorIndex;
+
+  /**
+   * Listener invoked if the vector overflows. If not provided, then the writer
+   * does not support vector overflow.
+   */
+
+  protected ColumnWriterListener listener;
+
+  protected DrillBuf drillBuf;
+
+  /**
+   * Capacity, in values, of the currently allocated buffer that backs
+   * the vector. Updated each time the buffer changes. The capacity is in
+   * values (rather than bytes) to streamline the per-write logic.
+   */
+
+  protected int capacity;
+
+  @Override
+  public void bindIndex(ColumnWriterIndex vectorIndex) {
+    this.vectorIndex = vectorIndex;
+  }
+
+  @Override
+  public ColumnWriterIndex writerIndex() { return vectorIndex; }
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) {
+    this.listener = listener;
+  }
+
+  /**
+   * All change of buffer comes through this function to allow capturing
+   * the buffer address and capacity. Only two ways to set the buffer:
+   * by binding to a vector in bindVector(), or by resizing the vector
+   * in writeIndex().
+   */
+
+  protected abstract void setBuffer();
+
+  protected void realloc(int size) {
+    vector().reallocRaw(size);
+    setBuffer();
+  }
+
+  /**
+   * The vector is about to grow. Give the listener a chance to
+   * veto the growth and opt for overflow instead.
+   *
+   * @param delta the new amount of memory to allocate
+   * @return true if the vector can be grown, false if an
+   * overflow should be triggered
+   */
+
+  protected boolean canExpand(int delta) {
+    if (listener == null) {
+      return true;
+    } else {
+      return listener.canExpand(this, delta);
+    }
+  }
+
+  /**
+   * Handle vector overflow. If this is an array, then there is a slim chance
+   * we may need to grow the vector immediately after overflow. Since a double
+   * overflow is not allowed, this recursive call won't continue forever.
+   */
+
+  protected void overflowed() {
+    if (listener == null) {
+      throw new IndexOutOfBoundsException("Overflow not supported");
+    } else {
+      listener.overflowed(this);
+    }
+  }
+
+  public abstract void skipNulls();
+
+  @Override
+  public void setNull() {
+    throw new UnsupportedOperationException("Vector is not nullable");
+  }
+
+  @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, int len) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setDecimal(BigDecimal value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void setPeriod(Period value) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.extend();
+    super.dump(format);
+    format
+      .attribute("vectorIndex", vectorIndex)
+      .attributeIdentity("listener", listener)
+      .attribute("capacity", capacity)
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseVarWidthWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseVarWidthWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseVarWidthWriter.java
new file mode 100644
index 0000000..e54625e
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/BaseVarWidthWriter.java
@@ -0,0 +1,157 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Base class for variable-width (VarChar, VarBinary, etc.) writers.
+ * Handles the additional complexity that such writers work with
+ * both an offset vector and a data vector. The offset vector is
+ * written using a specialized offset vector writer. The last write
+ * index is defined as the the last write position in the offset
+ * vector; not the last write position in the variable-width
+ * vector.
+ * <p>
+ * Most and value events are forwarded to the offset vector.
+ */
+
+public abstract class BaseVarWidthWriter extends BaseScalarWriter {
+  protected final OffsetVectorWriter offsetsWriter;
+
+  public BaseVarWidthWriter(UInt4Vector offsetVector) {
+    offsetsWriter = new OffsetVectorWriter(offsetVector);
+  }
+
+  @Override
+  public void bindIndex(final ColumnWriterIndex index) {
+    offsetsWriter.bindIndex(index);
+    super.bindIndex(index);
+  }
+
+  @Override
+  public void startWrite() {
+    setBuffer();
+    offsetsWriter.startWrite();
+  }
+
+  @Override
+  public void startRow() { offsetsWriter.startRow(); }
+
+  protected final int writeIndex(final int width) {
+
+    // This is performance critical code; every operation counts.
+    // Please be thoughtful when changing the code.
+
+    int writeOffset = offsetsWriter.nextOffset();
+    if (writeOffset + width < capacity) {
+      return writeOffset;
+    }
+    resize(writeOffset + width);
+    return offsetsWriter.nextOffset();
+  }
+
+  @Override
+  protected final void setBuffer() {
+    drillBuf = vector().getBuffer();
+    capacity = drillBuf.capacity();
+  }
+
+  private void resize(int size) {
+    if (size <= capacity) {
+      return;
+    }
+
+    // Since some vectors start off as 0 length, set a
+    // minimum size to avoid silly thrashing on early rows.
+
+    if (size < MIN_BUFFER_SIZE) {
+      size = MIN_BUFFER_SIZE;
+    }
+
+    // Grow the vector -- or overflow if the growth would make the batch
+    // consume too much memory. The idea is that we grow vectors as they
+    // fit the available memory budget, then we fill those vectors until
+    // one of them needs more space. At that point we trigger overflow to
+    // a new set of vectors. Internal fragmentation will result, but this
+    // approach (along with proper initial vector sizing), minimizes that
+    // fragmentation.
+
+    size = BaseAllocator.nextPowerOfTwo(size);
+
+    // Two cases: grow this vector or allocate a new one.
+
+    if (size <= ValueVector.MAX_BUFFER_SIZE && canExpand(size - capacity)) {
+
+      // Optimized form of reAlloc() which does not zero memory, does not do
+      // bounds checks (since they were already done above). The write index
+      // and offset remain unchanged.
+
+      realloc(size);
+    } else {
+
+      // Allocate a new vector, or throw an exception if overflow is not
+      // supported. If overflow is supported, the callback will call
+      // endWrite(), which will set the final writer index for the current
+      // vector. Then, bindVector() will be called to provide the new vector.
+      // The write index changes with the new vector.
+
+      overflowed();
+    }
+  }
+
+  @Override
+  public void skipNulls() { }
+
+  @Override
+  public void restartRow() { offsetsWriter.restartRow(); }
+
+  @Override
+  public int lastWriteIndex() { return offsetsWriter.lastWriteIndex(); }
+
+  @Override
+  public final void preRollover() {
+    vector().getBuffer().writerIndex(offsetsWriter.rowStartOffset());
+    offsetsWriter.preRollover();
+  }
+
+  @Override
+  public void postRollover() {
+    setBuffer();
+    offsetsWriter.postRollover();
+  }
+
+  @Override
+  public final void endWrite() {
+    vector().getBuffer().writerIndex(offsetsWriter.nextOffset());
+    offsetsWriter.endWrite();
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.extend();
+    super.dump(format);
+    format.attribute("offsetsWriter");
+    offsetsWriter.dump(format);
+    format.endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
new file mode 100644
index 0000000..5a1187a
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ColumnWriterFactory.java
@@ -0,0 +1,196 @@
+/*
+ * 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;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.ArrayObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter.ScalarObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractTupleWriter.TupleObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.MapWriter.ArrayMapWriter;
+import org.apache.drill.exec.vector.accessor.writer.MapWriter.DummyArrayMapWriter;
+import org.apache.drill.exec.vector.accessor.writer.MapWriter.DummyMapWriter;
+import org.apache.drill.exec.vector.accessor.writer.MapWriter.SingleMapWriter;
+import org.apache.drill.exec.vector.accessor.writer.dummy.DummyArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.dummy.DummyScalarWriter;
+import org.apache.drill.exec.vector.complex.AbstractMapVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.exec.vector.complex.RepeatedMapVector;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+
+/**
+ * Gather generated writer classes into a set of class tables to allow rapid
+ * run-time creation of writers. Builds the writer and its object writer
+ * wrapper which binds the vector to the writer.
+ */
+
+@SuppressWarnings("unchecked")
+public class ColumnWriterFactory {
+
+  private static final int typeCount = MinorType.values().length;
+  private static final Class<? extends BaseScalarWriter> requiredWriters[] = new Class[typeCount];
+
+  static {
+    ColumnAccessors.defineRequiredWriters(requiredWriters);
+  }
+
+  public static AbstractObjectWriter buildColumnWriter(ColumnMetadata schema, ValueVector vector) {
+    if (vector == null) {
+      return buildDummyColumnWriter(schema);
+    }
+
+    // Build a writer for a materialized column.
+
+    assert schema.type() == vector.getField().getType().getMinorType();
+    assert schema.mode() == vector.getField().getType().getMode();
+
+    switch (schema.type()) {
+    case GENERIC_OBJECT:
+    case LATE:
+    case NULL:
+    case LIST:
+    case MAP:
+      throw new UnsupportedOperationException(schema.type().toString());
+    default:
+      switch (schema.mode()) {
+      case OPTIONAL:
+        NullableVector nullableVector = (NullableVector) vector;
+        return NullableScalarWriter.build(schema, nullableVector,
+                newWriter(nullableVector.getValuesVector()));
+      case REQUIRED:
+        return new ScalarObjectWriter(schema, newWriter(vector));
+      case REPEATED:
+        RepeatedValueVector repeatedVector = (RepeatedValueVector) vector;
+        return ScalarArrayWriter.build(schema, repeatedVector,
+                newWriter(repeatedVector.getDataVector()));
+      default:
+        throw new UnsupportedOperationException(schema.mode().toString());
+      }
+    }
+  }
+
+  /**
+   * Build a writer for a non-projected column.
+   * @param schema schema of the column
+   * @return a "dummy" writer for the column
+   */
+
+  public static AbstractObjectWriter buildDummyColumnWriter(ColumnMetadata schema) {
+    switch (schema.type()) {
+    case GENERIC_OBJECT:
+    case LATE:
+    case NULL:
+    case LIST:
+    case MAP:
+      throw new UnsupportedOperationException(schema.type().toString());
+    default:
+      ScalarObjectWriter scalarWriter = new ScalarObjectWriter(schema,
+          new DummyScalarWriter());
+      switch (schema.mode()) {
+      case OPTIONAL:
+      case REQUIRED:
+        return scalarWriter;
+      case REPEATED:
+        return new ArrayObjectWriter(schema,
+            new DummyArrayWriter(
+              scalarWriter));
+      default:
+        throw new UnsupportedOperationException(schema.mode().toString());
+      }
+    }
+  }
+
+  public static TupleObjectWriter buildMap(ColumnMetadata schema, MapVector vector,
+                                        List<AbstractObjectWriter> writers) {
+    MapWriter mapWriter;
+    if (schema.isProjected()) {
+      mapWriter = new SingleMapWriter(schema, vector, writers);
+    } else {
+      mapWriter = new DummyMapWriter(schema, writers);
+    }
+    return new TupleObjectWriter(schema, mapWriter);
+  }
+
+  public static ArrayObjectWriter buildMapArray(ColumnMetadata schema,
+                                        UInt4Vector offsetVector,
+                                        List<AbstractObjectWriter> writers) {
+    MapWriter mapWriter;
+    if (schema.isProjected()) {
+      mapWriter = new ArrayMapWriter(schema, writers);
+    } else {
+      mapWriter = new DummyArrayMapWriter(schema, writers);
+    }
+    TupleObjectWriter mapArray = new TupleObjectWriter(schema, mapWriter);
+    AbstractArrayWriter arrayWriter;
+    if (schema.isProjected()) {
+      arrayWriter = new ObjectArrayWriter(
+          offsetVector,
+          mapArray);
+    } else  {
+      arrayWriter = new DummyArrayWriter(mapArray);
+    }
+    return new ArrayObjectWriter(schema, arrayWriter);
+  }
+
+  public static AbstractObjectWriter buildMapWriter(ColumnMetadata schema,
+      AbstractMapVector vector,
+      List<AbstractObjectWriter> writers) {
+    assert (vector != null) == schema.isProjected();
+    if (! schema.isArray()) {
+      return buildMap(schema, (MapVector) vector, writers);
+    } else if (vector == null) {
+      return buildMapArray(schema,
+          null, writers);
+    } else {
+      return buildMapArray(schema,
+          ((RepeatedMapVector) vector).getOffsetVector(),
+          writers);
+    }
+  }
+
+  public static AbstractObjectWriter buildMapWriter(ColumnMetadata schema, AbstractMapVector vector) {
+    assert schema.mapSchema().size() == 0;
+    return buildMapWriter(schema, vector, new ArrayList<AbstractObjectWriter>());
+  }
+
+  public static BaseScalarWriter newWriter(ValueVector vector) {
+    MajorType major = vector.getField().getType();
+    MinorType type = major.getMinorType();
+    try {
+      Class<? extends BaseScalarWriter> accessorClass = requiredWriters[type.ordinal()];
+      if (accessorClass == null) {
+        throw new UnsupportedOperationException(type.toString());
+      }
+      Constructor<? extends BaseScalarWriter> ctor = accessorClass.getConstructor(ValueVector.class);
+      return ctor.newInstance(vector);
+    } catch (InstantiationException | IllegalAccessException | NoSuchMethodException |
+             SecurityException | IllegalArgumentException | InvocationTargetException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
new file mode 100644
index 0000000..8aec301
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/MapWriter.java
@@ -0,0 +1,155 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+/**
+ * Writer for a Drill Map type. Maps are actually tuples, just like rows.
+ */
+
+public abstract class MapWriter extends AbstractTupleWriter {
+
+  /**
+   * Wrap the outer index to avoid incrementing the array index
+   * on the call to <tt>nextElement().</tt> For maps, the increment
+   * is done at the map level, not the column level.
+   */
+
+  private static class MemberWriterIndex implements ColumnWriterIndex {
+    private ColumnWriterIndex baseIndex;
+
+    private MemberWriterIndex(ColumnWriterIndex baseIndex) {
+      this.baseIndex = baseIndex;
+    }
+
+    @Override public int rowStartIndex() { return baseIndex.rowStartIndex(); }
+    @Override public int vectorIndex() { return baseIndex.vectorIndex(); }
+    @Override public void nextElement() { }
+    @Override public void rollover() { }
+    @Override public ColumnWriterIndex outerIndex() {
+      return baseIndex.outerIndex();
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder()
+        .append("[")
+        .append(getClass().getSimpleName())
+        .append(" baseIndex = ")
+        .append(baseIndex.toString())
+        .append("]")
+        .toString();
+    }
+  }
+
+  /**
+   * Writer for a single (non-array) map. Clients don't really "write" maps;
+   * rather, this writer is a holder for the columns within the map, and those
+   * columns are what is written.
+   */
+
+  protected static class SingleMapWriter extends MapWriter {
+    private final MapVector mapVector;
+
+    protected SingleMapWriter(ColumnMetadata schema, MapVector vector, List<AbstractObjectWriter> writers) {
+      super(schema, writers);
+      mapVector = vector;
+    }
+
+    @Override
+    public void endWrite() {
+      super.endWrite();
+
+      // Special form of set value count: used only for
+      // this class to avoid setting the value count of children.
+      // Setting these counts was already done. Doing it again
+      // will corrupt nullable vectors because the writers don't
+      // set the "lastSet" field of nullable vector accessors,
+      // and the initial value of -1 will cause all values to
+      // be overwritten.
+      //
+      // Note that the map vector can be null if there is no actual
+      // map vector represented by this writer.
+
+      if (mapVector != null) {
+        mapVector.setMapValueCount(vectorIndex.vectorIndex());
+      }
+    }
+  }
+
+  /**
+   * Writer for a an array of maps. A single array index coordinates writes
+   * to the constituent member vectors so that, say, the values for (row 10,
+   * element 5) all occur to the same position in the columns within the map.
+   * Since the map is an array, it has an associated offset vector, which the
+   * parent array writer is responsible for maintaining.
+   */
+
+  protected static class ArrayMapWriter extends MapWriter {
+
+    protected ArrayMapWriter(ColumnMetadata schema, List<AbstractObjectWriter> writers) {
+      super(schema, writers);
+    }
+
+    @Override
+    public void bindIndex(ColumnWriterIndex index) {
+
+      // This is a repeated map, so the provided index is an array element
+      // index. Convert this to an index that will not increment the element
+      // index on each write so that a map with three members, say, won't
+      // increment the index for each member. Rather, the index must be
+      // incremented at the array level.
+
+      bindIndex(index, new MemberWriterIndex(index));
+    }
+
+    // In endWrite(), do not call setValueCount on the map vector.
+    // Doing so will zero-fill the composite vectors because
+    // the internal map state does not track the writer state.
+    // Instead, the code in this structure has set the value
+    // count for each composite vector individually.
+  }
+
+  protected static class DummyMapWriter extends MapWriter {
+
+    protected DummyMapWriter(ColumnMetadata schema,
+        List<AbstractObjectWriter> writers) {
+      super(schema, writers);
+    }
+  }
+
+  protected static class DummyArrayMapWriter extends MapWriter {
+
+    protected DummyArrayMapWriter(ColumnMetadata schema,
+        List<AbstractObjectWriter> writers) {
+      super(schema, writers);
+    }
+  }
+
+  protected final ColumnMetadata mapColumnSchema;
+
+  protected MapWriter(ColumnMetadata schema, List<AbstractObjectWriter> writers) {
+    super(schema.mapSchema(), writers);
+    mapColumnSchema = schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
new file mode 100644
index 0000000..6da2b50
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/NullableScalarWriter.java
@@ -0,0 +1,190 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.NullableVector;
+import org.apache.drill.exec.vector.accessor.ColumnAccessors.UInt1ColumnWriter;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.joda.time.Period;
+
+public class NullableScalarWriter extends AbstractScalarWriter {
+
+  private final UInt1ColumnWriter isSetWriter;
+  private final BaseScalarWriter baseWriter;
+
+  public NullableScalarWriter(NullableVector nullableVector, BaseScalarWriter baseWriter) {
+    isSetWriter = new UInt1ColumnWriter(nullableVector.getBitsVector());
+    this.baseWriter = baseWriter;
+  }
+
+  public static ScalarObjectWriter build(ColumnMetadata schema,
+      NullableVector nullableVector, BaseScalarWriter baseWriter) {
+    return new ScalarObjectWriter(schema,
+        new NullableScalarWriter(nullableVector, baseWriter));
+  }
+
+  public BaseScalarWriter bitsWriter() { return isSetWriter; }
+  public BaseScalarWriter baseWriter() { return baseWriter; }
+
+  @Override
+  public BaseDataValueVector vector() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) {
+    isSetWriter.bindIndex(index);
+    baseWriter.bindIndex(index);
+  }
+
+  @Override
+  public ColumnWriterIndex writerIndex() { return baseWriter.writerIndex(); }
+
+  @Override
+  public ValueType valueType() {
+    return baseWriter.valueType();
+  }
+
+  @Override
+  public void restartRow() {
+    isSetWriter.restartRow();
+    baseWriter.restartRow();
+  }
+
+  @Override
+  public void setNull() {
+    isSetWriter.setInt(0);
+    baseWriter.skipNulls();
+  }
+
+  @Override
+  public void setInt(int value) {
+    baseWriter.setInt(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setLong(long value) {
+    baseWriter.setLong(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setDouble(double value) {
+    baseWriter.setDouble(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setString(String value) {
+    // String may overflow. Set bits after
+    // overflow since bits vector does not have
+    // overflow handling separate from the nullable
+    // vector as a whole.
+
+    baseWriter.setString(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setBytes(byte[] value, int len) {
+    baseWriter.setBytes(value, len);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setDecimal(BigDecimal value) {
+    baseWriter.setDecimal(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void setPeriod(Period value) {
+    baseWriter.setPeriod(value);
+    isSetWriter.setInt(1);
+  }
+
+  @Override
+  public void preRollover() {
+    isSetWriter.preRollover();
+    baseWriter.preRollover();
+  }
+
+  @Override
+  public void postRollover() {
+    isSetWriter.postRollover();
+    baseWriter.postRollover();
+  }
+
+  @Override
+  public int lastWriteIndex() {
+    return baseWriter.lastWriteIndex();
+  }
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) {
+    baseWriter.bindListener(listener);
+  }
+
+  @Override
+  public void startWrite() {
+    isSetWriter.startWrite();
+    baseWriter.startWrite();
+  }
+
+  @Override
+  public void startRow() {
+    // Skip calls for performance: they do nothing for
+    // scalar writers -- the only kind supported here.
+//    isSetWriter.startRow();
+    baseWriter.startRow();
+  }
+
+  @Override
+  public void endArrayValue() {
+    // Skip calls for performance: they do nothing for
+    // scalar writers -- the only kind supported here.
+//    isSetWriter.saveValue();
+    baseWriter.endArrayValue();
+  }
+
+  @Override
+  public void endWrite() {
+    isSetWriter.endWrite();
+    // Avoid back-filling null values.
+    baseWriter.skipNulls();
+    baseWriter.endWrite();
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.extend();
+    super.dump(format);
+    format.attribute("isSetWriter");
+    isSetWriter.dump(format);
+    format.attribute("baseWriter");
+    baseWriter.dump(format);
+    format.endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ObjectArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ObjectArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ObjectArrayWriter.java
new file mode 100644
index 0000000..3554a3b
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ObjectArrayWriter.java
@@ -0,0 +1,143 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.BaseArrayWriter;
+
+/**
+ * Writer for an array of either a map or another array. Here, the contents
+ * are a structure and need explicit saves. State transitions in addition to the
+ * base class are:
+ *
+ * <table border=1>
+ * <tr><th>Public API</th><th>Array Event</th>
+ *     <th>Offset Event</th><th>Element Event</th></tr>
+ * <tr><td>save() (array)</td>
+ *     <td>saveValue()</td>
+ *     <td>saveValue()</td>
+ *     <td>saveValue()</td></tr>
+ * </table>
+ *
+ * This class is use for arrays of maps (and for arrays of arrays). When used
+ * with a map, then we have a single offset vector pointing into a group of
+ * arrays. Consider the simple case of a map of three scalars. Here, we have
+ * a hybrid of the states discussed for the {@link BaseScalarWriter} and those
+ * discussed for {@link OffsetVectorWriter}. That is, the offset vector
+ * points into one map element. The individual elements can we Behind,
+ * Written or Unwritten, depending on the specific actions taken by the
+ * client.
+ * <p>
+ * For example:<pre><code>
+ *   Offset Vector      Vector A     Vector B    Vector C       Index
+ *       |    |   + - >   |X| < lwa    |Y|         |Z|            8
+ *  lw > |  8 | - +       | |          |Y|         |Z|            9
+ *   v > | 10 | - - - >   | |          |Y|         |Z|           10
+ *       |    |           | |          |Y| < lwb   |Z|           11
+ *       |    |      v' > | |          | |         |Z| < lwc     12
+ * </code></pre>
+ * In the above:
+ * <ul>
+ * <li>The last write index, lw, for the current row points to the
+ *     previous start position. (Recall that finishing the row writes the
+ *     end position into the entry for the <i>next</i> row.</li>
+ * <li>The top-level vector index, v, points to start position of
+ *     the current row, which is offset 10 in all three data vectors.</li>
+ * <li>The current array write position, v', is for the third element
+ *     of the array that starts at position 10.</li>
+ * <li>Since the row is active, the end position of the row has not yet
+ *     been written, and so is blank in the offset vector.</li>
+ * <li>The previous row had a two-element map array written, starting
+ *     at offset 8 and ending at offset 9 (inclusive), identified as
+ *     writing the next start offset (exclusive) into the following
+ *     offset array slot.</li>
+ * <li>Column A has not had data written since the first element of the
+ *     previous row. It is currently in the Behind state with the last
+ *     write position for A, lwa, pointing to the last write.</li>
+ * <li>Column B is in the Unwritten state. A value was written for
+ *     previous element in the map array, but not for the current element.
+ *     We see this by the fact that the last write position for B, lwb,
+ *     is one behind v'.</li>
+ * <li>Column C has been written for the current array element and is
+ *     in the Written state, with the last write position, lwc, pointing
+ *     to the same location as v'.</li>
+ * </ul>
+ * Suppose we now write to Vector A and end the row:<pre><code>
+ *   Offset Vector      Vector A     Vector B    Vector C       Index
+ *       |    |   + - >   |X|          |Y|         |Z|            8
+ *       |  8 | - +       |0|          |Y|         |Z|            9
+ *  lw > | 10 | - - - >   |0|          |Y|         |Z|           10
+ *   v > | 13 | - +       |0|          |Y| < lwb   |Z|           11
+ *       |    |   |       |X| < lwa    | |         |Z| < lwc     12
+ *       |    |   + - >   | |          | |         | | < v'      13
+ * </code></pre>
+ * Here:
+ * <ul>
+ * <li>Vector A has been back-filled and the last write index advanced.</li>
+ * <li>Vector B is now in the Behind state. Vectors A and B are in the
+ *     Unwritten state.</li>
+ * <li>The end position has been written to the offset vector, the
+ *     offset vector last write position has been advance, and the
+ *     top-level vector offset has advanced.</li>
+ * </ul>
+ * All this happens automatically as part of the indexing mechanisms.
+ * The key reason to understand this flow is to understand what happens
+ * in vector overflow: unlike an array of scalars, in which the data
+ * vector can never be in the Behind state, when we have an array of
+ * maps then each vector can be in an of the scalar writer state.
+ */
+
+public class ObjectArrayWriter extends BaseArrayWriter {
+
+  protected ObjectArrayWriter(UInt4Vector offsetVector, AbstractObjectWriter elementWriter) {
+    super(offsetVector, elementWriter);
+  }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) {
+    elementIndex = new ArrayElementWriterIndex();
+    super.bindIndex(index);
+  }
+
+  @Override
+  public void save() {
+    elementObjWriter.events().endArrayValue();
+    elementIndex.next();
+  }
+
+  @Override
+  public void set(Object... values) {
+    setObject(values);
+  }
+
+  @Override
+  public void setObject(Object array) {
+    Object values[] = (Object[]) array;
+    for (int i = 0; i < values.length; i++) {
+      elementObjWriter.set(values[i]);
+      save();
+    }
+  }
+
+  @Override
+  public int lastWriteIndex() {
+    // Undefined for arrays
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/OffsetVectorWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/OffsetVectorWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/OffsetVectorWriter.java
new file mode 100644
index 0000000..d5f9b30
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/OffsetVectorWriter.java
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.UInt4Vector;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.exec.vector.accessor.impl.HierarchicalFormatter;
+
+/**
+ * Specialized column writer for the (hidden) offset vector used
+ * with variable-length or repeated vectors. See comments in the
+ * <tt>ColumnAccessors.java</tt> template file for more details.
+ * <p>
+ * Note that the <tt>lastWriteIndex</tt> tracked here corresponds
+ * to the data values; it is one less than the actual offset vector
+ * last write index due to the nature of offset vector layouts. The selection
+ * of last write index basis makes roll-over processing easier as only this
+ * writer need know about the +1 translation required for writing.
+ * <p>
+ * The states illustrated in the base class apply here as well,
+ * remembering that the end offset for a row (or array position)
+ * is written one ahead of the vector index.
+ * <p>
+ * The vector index does create an interesting dynamic for the child
+ * writers. From the child writer's perspective, the states described in
+ * the super class are the only states of interest. Here we want to
+ * take the perspective of the parent.
+ * <p>
+ * The offset vector is an implementation of a repeat level. A repeat
+ * level can occur for a single array, or for a collection of columns
+ * within a repeated map. (A repeat level also occurs for variable-width
+ * fields, but this is a bit harder to see, so let's ignore that for
+ * now.)
+ * <p>
+ * The key point to realize is that each repeat level introduces an
+ * isolation level in terms of indexing. That is, empty values in the
+ * outer level have no affect on indexing in the inner level. In fact,
+ * the nature of a repeated outer level means that there are no empties
+ * in the inner level.
+ * <p>
+ * To illustrate:<pre><code>
+ *       Offset Vector          Data Vector   Indexes
+ *  lw, v > | 10 |   - - - - - >   | X |        10
+ *          | 12 |   - - +         | X | < lw'  11
+ *          |    |       + - - >   |   | < v'   12
+ * </code></pre>
+ * In the above, the client has just written an array of two elements
+ * at the current write position. The data starts at offset 10 in
+ * the data vector, and the next write will be at 12. The end offset
+ * is written one ahead of the vector index.
+ * <p>
+ * From the data vector's perspective, its last-write (lw') reflects
+ * the last element written. If this is an array of scalars, then the
+ * write index is automatically incremented, as illustrated by v'.
+ * (For map arrays, the index must be incremented by calling
+ * <tt>save()</tt> on the map array writer.)
+ * <p>
+ * Suppose the client now skips some arrays:<pre><code>
+ *       Offset Vector          Data Vector
+ *     lw > | 10 |   - - - - - >   | X |        10
+ *          | 12 |   - - +         | X | < lw'  11
+ *          |    |       + - - >   |   | < v'   12
+ *          |    |                 |   |        13
+ *      v > |    |                 |   |        14
+ * </code></pre>
+ * The last write position does not move and there are gaps in the
+ * offset vector. The vector index points to the current row. Note
+ * that the data vector last write and vector indexes do not change,
+ * this reflects the fact that the the data vector's vector index
+ * (v') matches the tail offset
+ * <p>
+ * The
+ * client now writes a three-element vector:<pre><code>
+ *       Offset Vector          Data Vector
+ *          | 10 |   - - - - - >   | X |        10
+ *          | 12 |   - - +         | X |        11
+ *          | 12 |   - - + - - >   | Y |        12
+ *          | 12 |   - - +         | Y |        13
+ *  lw, v > | 12 |   - - +         | Y | < lw'  14
+ *          | 15 |   - - - - - >   |   | < v'   15
+ * </code></pre>
+ * Quite a bit just happened. The empty offset slots were back-filled
+ * with the last write offset in the data vector. The client wrote
+ * three values, which advanced the last write and vector indexes
+ * in the data vector. And, the last write index in the offset
+ * vector also moved to reflect the update of the offset vector.
+ * Note that as a result, multiple positions in the offset vector
+ * point to the same location in the data vector. This is fine; we
+ * compute the number of entries as the difference between two successive
+ * offset vector positions, so the empty positions have become 0-length
+ * arrays.
+ * <p>
+ * Note that, for an array of scalars, when overflow occurs,
+ * we need only worry about two
+ * states in the data vector. Either data has been written for the
+ * row (as in the third example above), and so must be moved to the
+ * roll-over vector, or no data has been written and no move is
+ * needed. We never have to worry about missing values because the
+ * cannot occur in the data vector.
+ * <p>
+ * See {@link ObjectArrayWriter} for information about arrays of
+ * maps (arrays of multiple columns.)
+ */
+
+public class OffsetVectorWriter extends AbstractFixedWidthWriter {
+
+  private static final int VALUE_WIDTH = UInt4Vector.VALUE_WIDTH;
+
+  private UInt4Vector vector;
+
+  /**
+   * Offset of the first value for the current row. Used during
+   * overflow or if the row is restarted.
+   */
+
+  private int rowStartOffset;
+
+  /**
+   * Cached value of the end offset for the current value. Used
+   * primarily for variable-width columns to allow the column to be
+   * rewritten multiple times within the same row. The start offset
+   * value is updated with the end offset only when the value is
+   * committed in {@link @endValue()}.
+   */
+
+  private int nextOffset;
+
+  public OffsetVectorWriter(UInt4Vector vector) {
+    this.vector = vector;
+  }
+
+  @Override public BaseDataValueVector vector() { return vector; }
+  @Override public int width() { return VALUE_WIDTH; }
+
+  @Override
+  protected void realloc(int size) {
+    vector.reallocRaw(size);
+    setBuffer();
+  }
+
+  @Override
+  public ValueType valueType() { return ValueType.INTEGER; }
+
+  @Override
+  public void startWrite() {
+    super.startWrite();
+    nextOffset = 0;
+    rowStartOffset = 0;
+
+    // Special handling for first value. Alloc vector if needed.
+    // Offset vectors require a 0 at position 0. The (end) offset
+    // for row 0 starts at position 1, which is handled in
+    // writeOffset() below.
+
+    if (capacity * VALUE_WIDTH < MIN_BUFFER_SIZE) {
+      realloc(MIN_BUFFER_SIZE);
+    }
+    vector.getBuffer().unsafePutInt(0, 0);
+  }
+
+  public int nextOffset() { return nextOffset; }
+  public int rowStartOffset() { return rowStartOffset; }
+
+  @Override
+  public void startRow() { rowStartOffset = nextOffset; }
+
+  /**
+   * Return the write offset, which is one greater than the index reported
+   * by the vector index.
+   *
+   * @return the offset in which to write the current offset of the end
+   * of the current data value
+   */
+
+  protected final int writeIndex() {
+
+    // "Fast path" for the normal case of no fills, no overflow.
+    // This is the only bounds check we want to do for the entire
+    // set operation.
+
+    // This is performance critical code; every operation counts.
+    // Please be thoughtful when changing the code.
+
+    final int valueIndex = vectorIndex.vectorIndex();
+    int writeIndex = valueIndex + 1;
+    if (lastWriteIndex < valueIndex - 1 || writeIndex >= capacity) {
+      writeIndex = prepareWrite(writeIndex);
+    }
+
+    // Track the last write location for zero-fill use next time around.
+    // Recall, it is the value index, which is one less than the (end)
+    // offset index.
+
+    lastWriteIndex = writeIndex - 1;
+    return writeIndex;
+  }
+
+  protected int prepareWrite(int writeIndex) {
+
+    // Either empties must be filled or the vector is full.
+
+    resize(writeIndex);
+
+    // Call to resize may cause rollover, so reset write index
+    // afterwards.
+
+    writeIndex = vectorIndex.vectorIndex() + 1;
+
+    // Fill empties to the write position.
+
+    fillEmpties(writeIndex);
+    return writeIndex;
+  }
+
+  @Override
+  protected final void fillEmpties(final int writeIndex) {
+    while (lastWriteIndex < writeIndex - 1) {
+      drillBuf.unsafePutInt((++lastWriteIndex + 1) * VALUE_WIDTH, nextOffset);
+    }
+  }
+
+  public final void setNextOffset(final int newOffset) {
+    final int writeIndex = writeIndex();
+    drillBuf.unsafePutInt(writeIndex * VALUE_WIDTH, newOffset);
+    nextOffset = newOffset;
+  }
+
+  @Override
+  public void skipNulls() {
+
+    // Nothing to do. Fill empties logic will fill in missing
+    // offsets.
+  }
+
+  @Override
+  public void restartRow() {
+    nextOffset = rowStartOffset;
+    super.restartRow();
+  }
+
+  @Override
+  public void preRollover() {
+    setValueCount(vectorIndex.rowStartIndex() + 1);
+  }
+
+  @Override
+  public void postRollover() {
+    final int newNext = nextOffset - rowStartOffset;
+    super.postRollover();
+    nextOffset = newNext;
+  }
+
+  @Override
+  public final void endWrite() {
+    setValueCount(vectorIndex.vectorIndex() + 1);
+  }
+
+  @Override
+  public void dump(HierarchicalFormatter format) {
+    format.extend();
+    super.dump(format);
+    format
+      .attribute("lastWriteIndex", lastWriteIndex)
+      .attribute("nextOffset", nextOffset)
+      .endObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
new file mode 100644
index 0000000..95f8f29
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/ScalarArrayWriter.java
@@ -0,0 +1,229 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.record.ColumnMetadata;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.BaseArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter.ScalarObjectWriter;
+import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+import org.joda.time.Period;
+
+/**
+ * Writer for a column that holds an array of scalars. This writer manages
+ * the array itself. A type-specific child writer manages the elements within
+ * the array. The overall row index (usually) provides the index into
+ * the offset vector. An array-specific element index provides the index
+ * into elements.
+ * <p>
+ * This class manages the offset vector directly. Doing so saves one read and
+ * one write to direct memory per element value.
+ * <p>
+ * Provides generic write methods for testing and other times when
+ * convenience is more important than speed.
+ * <p>
+ * The scalar writer for array-valued columns 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 class ScalarArrayWriter extends BaseArrayWriter {
+
+  /**
+   * For scalar arrays, incrementing the element index and
+   * committing the current value is done automatically since
+   * there is exactly one value per array element.
+   */
+
+  public class ScalarElementWriterIndex extends ArrayElementWriterIndex {
+
+    @Override
+    public final void nextElement() { next(); }
+  }
+
+  private final BaseScalarWriter elementWriter;
+
+  public ScalarArrayWriter(ColumnMetadata schema,
+      RepeatedValueVector vector, BaseScalarWriter elementWriter) {
+    super(vector.getOffsetVector(),
+        new ScalarObjectWriter(schema, elementWriter));
+    this.elementWriter = elementWriter;
+  }
+
+  public static ArrayObjectWriter build(ColumnMetadata schema,
+      RepeatedValueVector repeatedVector, BaseScalarWriter elementWriter) {
+    return new ArrayObjectWriter(schema,
+        new ScalarArrayWriter(schema, repeatedVector, elementWriter));
+  }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) {
+    elementIndex = new ScalarElementWriterIndex();
+    super.bindIndex(index);
+    elementWriter.bindIndex(elementIndex);
+  }
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) {
+    elementWriter.bindListener(listener);
+  }
+
+  @Override
+  public void save() {
+    // No-op: done when writing each scalar value
+  }
+
+  @Override
+  public void set(Object... values) {
+    for (Object value : values) {
+      entry().set(value);
+    }
+  }
+
+  @Override
+  public void setObject(Object array) {
+    if (array == null) {
+      // Assume null means a 0-element array since Drill does
+      // not support null for the whole array.
+
+      return;
+    }
+    String objClass = array.getClass().getName();
+    if (! objClass.startsWith("[")) {
+      throw new IllegalArgumentException("Argument must be an array");
+    }
+
+    // Figure out type
+
+    char second = objClass.charAt(1);
+    switch ( second ) {
+    case  '[':
+      // bytes is represented as an array of byte arrays.
+
+      char third = objClass.charAt(2);
+      switch (third) {
+      case 'B':
+        setBytesArray((byte[][]) array);
+        break;
+      default:
+        throw new IllegalArgumentException( "Unknown Java array type: " + objClass );
+      }
+      break;
+    case  'S':
+      setShortArray((short[]) array );
+      break;
+    case  'I':
+      setIntArray((int[]) array );
+      break;
+    case  'J':
+      setLongArray((long[]) array );
+      break;
+    case  'F':
+      setFloatArray((float[]) array );
+      break;
+    case  'D':
+      setDoubleArray((double[]) array );
+      break;
+    case  'Z':
+      setBooleanArray((boolean[]) array );
+      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())) {
+        setStringArray((String[]) array );
+      } else if (memberClassName.equals(Period.class.getName())) {
+        setPeriodArray((Period[]) array );
+      } else if (memberClassName.equals(BigDecimal.class.getName())) {
+        setBigDecimalArray((BigDecimal[]) array );
+      } else {
+        throw new IllegalArgumentException( "Unknown Java array type: " + memberClassName );
+      }
+      break;
+    default:
+      throw new IllegalArgumentException( "Unknown Java array type: " + objClass );
+    }
+  }
+
+  public void setBooleanArray(boolean[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setInt(value[i] ? 1 : 0);
+    }
+  }
+
+  public void setBytesArray(byte[][] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setBytes(value[i], value[i].length);
+    }
+  }
+
+  public void setShortArray(short[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setInt(value[i]);
+    }
+  }
+
+  public void setIntArray(int[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setInt(value[i]);
+    }
+  }
+
+  public void setLongArray(long[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setLong(value[i]);
+    }
+  }
+
+  public void setFloatArray(float[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setDouble(value[i]);
+    }
+  }
+
+  public void setDoubleArray(double[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setDouble(value[i]);
+    }
+  }
+
+  public void setStringArray(String[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setString(value[i]);
+    }
+  }
+
+  public void setPeriodArray(Period[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setPeriod(value[i]);
+    }
+  }
+
+  public void setBigDecimalArray(BigDecimal[] value) {
+    for (int i = 0; i < value.length; i++) {
+      elementWriter.setDecimal(value[i]);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/WriterEvents.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/WriterEvents.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/WriterEvents.java
new file mode 100644
index 0000000..7566f28
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/WriterEvents.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.writer;
+
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+
+/**
+ * Internal interface used to control the behavior
+ * of writers. Consumers of writers never use this method; it is
+ * instead used by the code that implements writers.
+ * <p>
+ * Most methods here represents events in a state machine. The top-level
+ * writer provides a set of public methods which trigger one or more of
+ * these internal events. The events draw some fine distinctions between
+ * top-level values and those nested within arrays. See each kind of
+ * writer for the details.
+ * <p>
+ * The events also ensure symmetry between top-level and nested tuples,
+ * especially those nested within an array. That is, an event cannot change
+ * meaning depending on whether the tuple is top-level or nested within an
+ * array. Instead, the order of calls, or selectively making or not making
+ * calls, can change.
+ */
+
+public interface WriterEvents {
+
+  /**
+   * Bind the writer to a writer index.
+   *
+   * @param index the writer index (top level or nested for
+   * arrays)
+   */
+
+  void bindIndex(ColumnWriterIndex index);
+
+  ColumnWriterIndex writerIndex();
+
+  /**
+   * Start a write (batch) operation. Performs any vector initialization
+   * required at the start of a batch (especially for offset vectors.)
+   */
+
+  void startWrite();
+
+  /**
+   * Start a new row. To be called only when a row is not active. To
+   * restart a row, call {@link #restartRow()} instead.
+   */
+
+  void startRow();
+
+  /**
+   * End a value. Similar to {@link saveRow()}, but the save of a value
+   * is conditional on saving the row. This version is primarily of use
+   * in tuples nested inside arrays: it saves each tuple within the array,
+   * advancing to a new position in the array. The update of the array's
+   * offset vector based on the cumulative value saves is done when
+   * saving the row.
+   */
+
+  void endArrayValue();
+
+  /**
+   * During a writer to a row, rewind the the current index position to
+   * restart the row.
+   * Done when abandoning the current row, such as when filtering out
+   * a row at read time.
+   */
+
+  void restartRow();
+
+  /**
+   * Saves a row. Commits offset vector locations and advances each to
+   * the next position. Can be called only when a row is active.
+   */
+
+  void saveRow();
+
+  /**
+   * End a batch: finalize any vector values.
+   */
+
+  void endWrite();
+
+  /**
+   * The vectors backing this vector are about to roll over. Finish
+   * the current batch up to, but not including, the current row.
+   */
+
+  void preRollover();
+
+  /**
+   * The vectors backing this writer rolled over. This means that data
+   * for the current row has been rolled over into a new vector. Offsets
+   * and indexes should be shifted based on the understanding that data
+   * for the current row now resides at the start of a new vector instead
+   * of its previous location elsewhere in an old vector.
+   */
+
+  void postRollover();
+
+  /**
+   * Return the last write position in the vector. This may be the
+   * same as the writer index position (if the vector was written at
+   * that point), or an earlier point. In either case, this value
+   * points to the last valid value in the vector.
+   *
+   * @return index of the last valid value in the vector
+   */
+
+  int lastWriteIndex();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyArrayWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyArrayWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyArrayWriter.java
new file mode 100644
index 0000000..7c9f8ba
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyArrayWriter.java
@@ -0,0 +1,96 @@
+/*
+ * 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.dummy;
+
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ScalarWriter.ColumnWriterListener;
+import org.apache.drill.exec.vector.accessor.TupleWriter.TupleWriterListener;
+import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter;
+import org.apache.drill.exec.vector.accessor.writer.AbstractObjectWriter;
+import org.apache.drill.exec.vector.accessor.writer.OffsetVectorWriter;
+
+/**
+ * Dummy scalar array writer that allows a client to write values into
+ * the array, but discards all of them. Provides no implementations of
+ * any methods, all are simply ignored.
+ * <p>
+ * Experience may suggest that some methods must return non-dummy
+ * values, such as the number of items in the array. That can be added
+ * as needed.
+ */
+public class DummyArrayWriter extends AbstractArrayWriter {
+
+  public DummyArrayWriter(
+      AbstractObjectWriter elementWriter) {
+    super(elementWriter);
+  }
+
+  @Override
+  public int size() { return 0; }
+
+  @Override
+  public void save() { }
+
+  @Override
+  public void set(Object... values) { }
+
+  @Override
+  public void setObject(Object array) { }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) { }
+
+  @Override
+  public ColumnWriterIndex writerIndex() { return null; }
+
+  @Override
+  public void startWrite() { }
+
+  @Override
+  public void startRow() { }
+
+  @Override
+  public void endArrayValue() { }
+
+  @Override
+  public void restartRow() { }
+
+  @Override
+  public void saveRow() { }
+
+  @Override
+  public void endWrite() { }
+
+  @Override
+  public void preRollover() { }
+
+  @Override
+  public void postRollover() { }
+
+  @Override
+  public int lastWriteIndex() { return 0; }
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) { }
+
+  @Override
+  public void bindListener(TupleWriterListener listener) { }
+
+  @Override
+  public OffsetVectorWriter offsetWriter() { return null; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/40de8ca4/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyScalarWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyScalarWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyScalarWriter.java
new file mode 100644
index 0000000..e8272d6
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/accessor/writer/dummy/DummyScalarWriter.java
@@ -0,0 +1,89 @@
+/*
+ * 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.dummy;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.BaseDataValueVector;
+import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
+import org.apache.drill.exec.vector.accessor.ValueType;
+import org.apache.drill.exec.vector.accessor.writer.AbstractScalarWriter;
+import org.joda.time.Period;
+
+/**
+ * Represents a non-projected column. The writer accepts data, but
+ * discards it. The writer does not participate in writer events,
+ * nor is it backed by a real vector, index or type.
+ */
+
+public class DummyScalarWriter extends AbstractScalarWriter {
+
+  @Override
+  public void bindListener(ColumnWriterListener listener) { }
+
+  @Override
+  public ValueType valueType() { return null; }
+
+  @Override
+  public void setNull() { }
+
+  @Override
+  public void setInt(int value) { }
+
+  @Override
+  public void setLong(long value) { }
+
+  @Override
+  public void setDouble(double value) { }
+
+  @Override
+  public void setString(String value) { }
+
+  @Override
+  public void setBytes(byte[] value, int len) { }
+
+  @Override
+  public void setDecimal(BigDecimal value) { }
+
+  @Override
+  public void setPeriod(Period value) { }
+
+  @Override
+  public void bindIndex(ColumnWriterIndex index) { }
+
+  @Override
+  public ColumnWriterIndex writerIndex() { return null; }
+
+  @Override
+  public void restartRow() { }
+
+  @Override
+  public void endWrite() { }
+
+  @Override
+  public void preRollover() { }
+
+  @Override
+  public void postRollover() { }
+
+  @Override
+  public int lastWriteIndex() { return 0; }
+
+  @Override
+  public BaseDataValueVector vector() { return null; }
+}