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/07/03 19:49:35 UTC

[2/3] drill git commit: DRILL-5517: Size-aware set methods in value vectors

DRILL-5517: Size-aware set methods in value vectors

Please see DRILL-5517 for an explanation.

Also includes a workaround for DRILL-5529.
Implements a setEmpties method for repeated and non-nullable
variable-width types in support of the revised column accessors.

Unit test included. Without the setEmpties call, the tests fail with
vector corruption. With the call, things work properly.

closes #840


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

Branch: refs/heads/master
Commit: 92c9304f77a515d8f6747514bd295613738f611e
Parents: 6446e56
Author: Paul Rogers <pr...@maprtech.com>
Authored: Tue May 16 13:20:32 2017 -0700
Committer: Paul Rogers <pr...@maprtech.com>
Committed: Mon Jul 3 11:39:16 2017 -0700

----------------------------------------------------------------------
 .../exec/record/vector/TestValueVector.java     |   7 +-
 .../apache/drill/vector/TestFillEmpties.java    | 242 ++++++++
 .../apache/drill/vector/TestVectorLimits.java   | 484 ++++++++++++++++
 .../org/apache/drill/vector/package-info.java   |  22 +
 .../src/main/java/io/netty/buffer/DrillBuf.java |  19 +
 .../netty/buffer/UnsafeDirectLittleEndian.java  |  75 ++-
 .../codegen/templates/FixedValueVectors.java    | 577 +++++++++++++------
 .../codegen/templates/NullableValueVectors.java | 163 +++++-
 .../codegen/templates/RepeatedValueVectors.java | 132 ++++-
 .../src/main/codegen/templates/UnionVector.java |  13 +-
 .../templates/VariableLengthVectors.java        | 218 ++++++-
 .../drill/exec/vector/BaseDataValueVector.java  |  21 +-
 .../drill/exec/vector/BaseValueVector.java      |  13 +-
 .../org/apache/drill/exec/vector/BitVector.java |  85 ++-
 .../drill/exec/vector/FixedWidthVector.java     |   5 +-
 .../apache/drill/exec/vector/ObjectVector.java  |  39 +-
 .../apache/drill/exec/vector/ValueVector.java   |  60 +-
 .../exec/vector/VectorOverflowException.java    |  30 +
 .../apache/drill/exec/vector/VectorUtils.java   |  63 ++
 .../apache/drill/exec/vector/ZeroVector.java    |  71 +--
 .../vector/complex/BaseRepeatedValueVector.java |  18 +-
 .../drill/exec/vector/complex/ListVector.java   |   8 +-
 .../drill/exec/vector/complex/MapVector.java    |  14 +-
 .../exec/vector/complex/RepeatedListVector.java |  19 +-
 .../exec/vector/complex/RepeatedMapVector.java  |  19 +-
 .../vector/complex/impl/AbstractBaseReader.java |  12 +-
 26 files changed, 2059 insertions(+), 370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/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 f6044e1..5191f5f 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
@@ -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
@@ -25,7 +25,6 @@ import java.nio.charset.Charset;
 
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.types.MinorType;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecTest;
@@ -47,7 +46,6 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
@@ -97,6 +95,7 @@ public class TestValueVector extends ExecTest {
   @Test(expected = OversizedAllocationException.class)
   public void testFixedVectorReallocation() {
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+    @SuppressWarnings("resource")
     final UInt4Vector vector = new UInt4Vector(field, allocator);
     // edge case 1: buffer size = max value capacity
     final int expectedValueCapacity = BaseValueVector.MAX_ALLOCATION_SIZE / 4;
@@ -149,10 +148,10 @@ public class TestValueVector extends ExecTest {
     }
   }
 
-
   @Test(expected = OversizedAllocationException.class)
   public void testVariableVectorReallocation() {
     final MaterializedField field = MaterializedField.create(EMPTY_SCHEMA_PATH, UInt4Holder.TYPE);
+    @SuppressWarnings("resource")
     final VarCharVector vector = new VarCharVector(field, allocator);
     // edge case 1: value count = MAX_VALUE_ALLOCATION
     final int expectedAllocationInBytes = BaseValueVector.MAX_ALLOCATION_SIZE;

http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/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
new file mode 100644
index 0000000..266bff2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/vector/TestFillEmpties.java
@@ -0,0 +1,242 @@
+/*
+ * 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.assertEquals;
+import static org.junit.Assert.fail;
+
+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.junit.Test;
+
+import io.netty.buffer.DrillBuf;
+
+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);
+  }
+
+  @Test
+  public void testNullableVarChar() {
+    @SuppressWarnings("resource")
+    NullableVarCharVector vector = new NullableVarCharVector(makeField("a", MinorType.VARCHAR, DataMode.OPTIONAL), fixture.allocator());
+    vector.allocateNew( );
+
+    // Create "foo", null, "bar", but omit the null.
+
+    NullableVarCharVector.Mutator mutator = vector.getMutator();
+    byte[] value = makeValue( "foo" );
+    mutator.setSafe(0, value, 0, value.length);
+
+    value = makeValue("bar");
+    mutator.setSafe(2, value, 0, value.length);
+
+    visualize(vector, 3);
+    verifyOffsets(vector.getValuesVector().getOffsetVector(), new int[] {0, 3, 3, 6});
+    vector.close();
+  }
+
+  @Test
+  public void testVarChar() {
+    @SuppressWarnings("resource")
+    VarCharVector vector = new VarCharVector(makeField("a", MinorType.VARCHAR, DataMode.REQUIRED), fixture.allocator());
+    vector.allocateNew( );
+
+    // Create "foo", null, "bar", but omit the null.
+
+    VarCharVector.Mutator mutator = vector.getMutator();
+    byte[] value = makeValue( "foo" );
+    mutator.setSafe(0, value, 0, value.length);
+
+    // Work around: test fails without this. But, only the new column writers
+    // call this method.
+
+    try {
+      mutator.fillEmptiesBounded(0, 2);
+    } catch (VectorOverflowException e) {
+      fail();
+    }
+    value = makeValue("bar");
+    mutator.setSafe(2, value, 0, value.length);
+
+    visualize(vector, 3);
+    verifyOffsets(vector.getOffsetVector(), new int[] {0, 3, 3, 6});
+    vector.close();
+  }
+
+  @Test
+  public void testInt() {
+    @SuppressWarnings("resource")
+    IntVector vector = new IntVector(makeField("a", MinorType.INT, DataMode.REQUIRED), fixture.allocator());
+    vector.allocateNew( );
+
+    // Create 1, 0, 2, but omit the 0.
+
+    IntVector.Mutator mutator = vector.getMutator();
+    mutator.setSafe(0, 1);
+
+    mutator.setSafe(2, 3);
+
+    visualize(vector, 3);
+    vector.close();
+  }
+
+  @Test
+  public void testRepeatedVarChar() {
+    @SuppressWarnings("resource")
+    RepeatedVarCharVector vector = new RepeatedVarCharVector(makeField("a", MinorType.VARCHAR, DataMode.REPEATED), fixture.allocator());
+    vector.allocateNew( );
+
+    // Create "foo", null, "bar", but omit the null.
+
+    RepeatedVarCharVector.Mutator mutator = vector.getMutator();
+    mutator.startNewValue(0);
+    byte[] value = makeValue( "a" );
+    mutator.addSafe(0, value, 0, value.length);
+    value = makeValue( "b" );
+    mutator.addSafe(0, value, 0, value.length);
+
+    // 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.startNewValue(2);
+    value = makeValue( "c" );
+    mutator.addSafe(2, value, 0, value.length);
+    value = makeValue( "d" );
+    mutator.addSafe(2, value, 0, value.length);
+
+    visualize(vector, 3);
+    verifyOffsets(vector.getOffsetVector(), new int[] {0, 2, 2, 4});
+    verifyOffsets(vector.getDataVector().getOffsetVector(), new int[] {0, 1, 2, 3, 4});
+    vector.close();
+  }
+
+  private void visualize(RepeatedVarCharVector vector, int valueCount) {
+    visualize("Array Offsets", vector.getOffsetVector(), valueCount + 1);
+    visualize(vector.getDataVector(), vector.getOffsetVector().getAccessor().get(valueCount));
+  }
+
+  private void visualize(IntVector vector, int valueCount) {
+    System.out.print("Values: [");
+    IntVector.Accessor accessor = vector.getAccessor();
+    for (int i = 0; i < valueCount; i++) {
+      if (i > 0) { System.out.print(" "); }
+      System.out.print(accessor.get(i));
+    }
+    System.out.println("]");
+  }
+
+  private void visualize(NullableVarCharVector vector, int valueCount) {
+    visualize("Is-set", vector.getAccessor(), valueCount);
+    visualize(vector.getValuesVector(), valueCount);
+  }
+
+  private void visualize(VarCharVector vector, int valueCount) {
+    visualize("Offsets", vector.getOffsetVector(), valueCount + 1);
+    visualize("Data", vector.getBuffer(), vector.getOffsetVector().getAccessor().get(valueCount));
+  }
+
+  private void visualize(String label, UInt4Vector offsetVector,
+      int valueCount) {
+    System.out.print(label + ": [");
+    UInt4Vector.Accessor accessor = offsetVector.getAccessor();
+    for (int i = 0; i < valueCount; i++) {
+      if (i > 0) { System.out.print(" "); }
+      System.out.print(accessor.get(i));
+    }
+    System.out.println("]");
+  }
+
+  private void visualize(String label, DrillBuf buffer, int valueCount) {
+    System.out.print(label + ": [");
+    for (int i = 0; i < valueCount; i++) {
+      if (i > 0) { System.out.print(" "); }
+      System.out.print((char) buffer.getByte(i));
+    }
+    System.out.println("]");
+  }
+
+  private void visualize(String label, BaseDataValueVector.BaseAccessor accessor, int valueCount) {
+    System.out.print(label + ": [");
+    for (int i = 0; i < valueCount; i++) {
+      if (i > 0) { System.out.print(" "); }
+      System.out.print(accessor.isNull(i) ? 0 : 1);
+    }
+    System.out.println("]");
+  }
+
+  private void verifyOffsets(UInt4Vector offsetVector, int[] expected) {
+    UInt4Vector.Accessor accessor = offsetVector.getAccessor();
+    for (int i = 0; i < expected.length; i++) {
+      assertEquals(expected[i], accessor.get(i));
+    }
+  }
+
+  /**
+   * Create a test value. Works only for the ASCII subset of characters, obviously.
+   * @param string
+   * @return
+   */
+  private byte[] makeValue(String string) {
+    byte value[] = new byte[string.length()];
+    for (int i = 0; i < value.length; i++) {
+      value[i] = (byte) string.charAt(i);
+    }
+    return value;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/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
new file mode 100644
index 0000000..86bd206
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/vector/TestVectorLimits.java
@@ -0,0 +1,484 @@
+/*
+ * 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.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;
+
+/**
+ * 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.
+ */
+
+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/92c9304f/exec/java-exec/src/test/java/org/apache/drill/vector/package-info.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/vector/package-info.java b/exec/java-exec/src/test/java/org/apache/drill/vector/package-info.java
new file mode 100644
index 0000000..c858814
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/vector/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Tests for value vectors. Is in this module to allow use of
+ * the test tools which are available only in this module.
+ */
+package org.apache.drill.vector;

http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/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 3793f25..5139086 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
@@ -748,6 +748,25 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     return this;
   }
 
+  // Clone of UDLE's setBytes(), but with bounds checking done as a boolean,
+  // not assertion.
+
+  public boolean setBytesBounded(int index, byte[] src, int srcIndex, int length) {
+    // Must do here because Drill's UDLE is not ref counted.
+    // Done as an assert to avoid production overhead: if this is going
+    // to fail, it will do so spectacularly in tests, due to a programming error.
+    assert refCnt() > 0;
+    return udle.setBytesBounded(index, src, srcIndex, length);
+  }
+
+  // As above, but for direct memory.
+
+  public boolean setBytesBounded(int index, DrillBuf src, int srcIndex, int length) {
+    // See above.
+    assert refCnt() > 0;
+    return udle.setBytesBounded(index, src.udle, srcIndex, length);
+  }
+
   @Override
   public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
     udle.setBytes(index + offset, src, srcIndex, length);

http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java b/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
index 6495d5d..c91944f 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
@@ -18,15 +18,17 @@
 
 package io.netty.buffer;
 
-import io.netty.util.internal.PlatformDependent;
-
 import java.nio.ByteOrder;
 import java.util.concurrent.atomic.AtomicLong;
 
+import io.netty.util.internal.PlatformDependent;
+
 /**
- * The underlying class we use for little-endian access to memory. Is used underneath DrillBufs to abstract away the
- * Netty classes and underlying Netty memory management.
+ * The underlying class we use for little-endian access to memory. Is used
+ * underneath DrillBufs to abstract away the Netty classes and underlying Netty
+ * memory management.
  */
+
 public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
   private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
   private static final AtomicLong ID_GENERATOR = new AtomicLong(0);
@@ -49,7 +51,6 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
 
   UnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong bufferCount, AtomicLong bufferSize) {
     this(buf, true, bufferCount, bufferSize);
-
   }
 
   private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake, AtomicLong bufferCount, AtomicLong bufferSize) {
@@ -67,21 +68,21 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
     this.wrapped = buf;
     this.memoryAddress = buf.memoryAddress();
   }
-    private long addr(int index) {
-        return memoryAddress + index;
-    }
 
-    @Override
-    public long getLong(int index) {
-//        wrapped.checkIndex(index, 8);
-        long v = PlatformDependent.getLong(addr(index));
-        return v;
-    }
+  private long addr(int index) {
+    return memoryAddress + index;
+  }
 
-    @Override
-    public float getFloat(int index) {
-        return Float.intBitsToFloat(getInt(index));
-    }
+  @Override
+  public long getLong(int index) {
+//  wrapped.checkIndex(index, 8);
+    return PlatformDependent.getLong(addr(index));
+  }
+
+  @Override
+  public float getFloat(int index) {
+    return Float.intBitsToFloat(getInt(index));
+  }
 
   @Override
   public ByteBuf slice() {
@@ -174,6 +175,43 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
     return this;
   }
 
+  // Clone of the super class checkIndex, but this version returns a boolean rather
+  // than throwing an exception.
+
+  protected boolean hasCapacity(int index, int fieldLength) {
+    assert fieldLength >= 0;
+    return (! (index < 0 || index > capacity() - fieldLength));
+  }
+
+  /**
+   * Write bytes into the buffer at the given index, if space is available.
+   * @param index location to write
+   * @param src bytes to write
+   * @param srcIndex start of data in the source array
+   * @param length length of the data to write
+   * @return true if the value was written, false if the value was not
+   * written because the value would overflow the buffer
+   */
+
+  public boolean setBytesBounded(int index, byte[] src, int srcIndex, int length) {
+    if (! hasCapacity(index, length)) {
+      return false;
+    }
+    PlatformDependent.copyMemory(src, srcIndex, addr(index), length);
+    return true;
+  }
+
+  // Version of the super class setBytes(), but with bounds checking done as a boolean,
+  // not assertion. This version requires a direct source buffer.
+
+  public boolean setBytesBounded(int index, UnsafeDirectLittleEndian src, int srcIndex, int length) {
+    if (! hasCapacity(index, length)) {
+      return false;
+    }
+    PlatformDependent.copyMemory(src.memoryAddress() + srcIndex, addr(index), length);
+    return true;
+  }
+
   @Override
   public ByteBuf writeShort(int value) {
     wrapped.ensureWritable(2);
@@ -266,5 +304,4 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
     assert isAssertEnabled = true;
     ASSERT_ENABLED = isAssertEnabled;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/92c9304f/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 23188ce..1e83a4f 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -31,22 +31,60 @@ 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 randomly accessed.
- *   The width of each element is ${type.width} byte(s)
- *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
+ * ${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 equivalent Java primitive is '${minor.javaType!type.javaType}'.</li>
+ * </ul>
  *
  * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
  */
-public final class ${minor.class}Vector extends BaseDataValueVector implements FixedWidthVector{
+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);
 
+  /**
+   * Width of each fixed-width value.
+   */
+
+  public static final int VALUE_WIDTH = ${type.width};
+
+  /**
+   * Maximum number of values that this fixed-width vector can hold
+   * and stay below the maximum vector size limit. This is the limit
+   * enforced when the vector is used to hold values in a repeated
+   * vector.
+   */
+
+  public static final int MAX_VALUE_COUNT = MAX_BUFFER_SIZE / VALUE_WIDTH;
+
+  /**
+   * Maximum number of values that this fixed-width vector can hold
+   * and stay below the maximum vector size limit and/or stay below
+   * the maximum row count. This is the limit enforced when the
+   * vector is used to hold scalar (required or nullable) values.
+   * <p>
+   * Note: <tt>MAX_ROW_COUNT</tt> is defined in the parent <tt>ValueVector</tt>
+   * class as the maximum number of rows in a record batch (64K). Use this
+   * in place of the <tt>Character.MAX_SIZE</tt> value previously used.
+   */
+
+  public static final int MAX_SCALAR_COUNT = Math.min(MAX_ROW_COUNT, MAX_VALUE_COUNT);
+
+  /**
+   * Actual maximum vector size, in bytes, given the number of fixed-width
+   * values that either fit in the maximum overall vector size, or that
+   * is no larger than the maximum vector item count.
+   */
+
+  public static final int NET_MAX_SCALAR_SIZE = VALUE_WIDTH * MAX_SCALAR_COUNT;
+
   private final FieldReader reader = new ${minor.class}ReaderImpl(${minor.class}Vector.this);
   private final Accessor accessor = new Accessor();
   private final Mutator mutator = new Mutator();
 
-  private int allocationSizeInBytes = INITIAL_VALUE_ALLOCATION * ${type.width};
+  private int allocationSizeInBytes = Math.min(INITIAL_VALUE_ALLOCATION * VALUE_WIDTH, MAX_BUFFER_SIZE);
   private int allocationMonitor = 0;
 
   public ${minor.class}Vector(MaterializedField field, BufferAllocator allocator) {
@@ -54,45 +92,41 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   @Override
-  public FieldReader getReader(){
-    return reader;
-  }
+  public FieldReader getReader() { return reader; }
 
   @Override
   public int getBufferSizeFor(final int valueCount) {
     if (valueCount == 0) {
       return 0;
     }
-    return valueCount * ${type.width};
+    return valueCount * VALUE_WIDTH;
   }
 
   @Override
   public int getValueCapacity(){
-    return data.capacity() / ${type.width};
+    return data.capacity() / VALUE_WIDTH;
   }
 
   @Override
-  public Accessor getAccessor(){
-    return accessor;
-  }
+  public Accessor getAccessor() { return accessor; }
 
   @Override
-  public Mutator getMutator(){
-    return mutator;
-  }
+  public Mutator getMutator() { return mutator; }
 
   @Override
   public void setInitialCapacity(final int valueCount) {
-    final long size = 1L * valueCount * ${type.width};
+    final long size = (long) valueCount * VALUE_WIDTH;
+    // TODO: Replace this with MAX_BUFFER_SIZE once all
+    // code is aware of the maximum vector size.
     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;
   }
 
   @Override
   public void allocateNew() {
-    if(!allocateNewSafe()){
+    if (!allocateNewSafe()){
       throw new OutOfMemoryException("Failure while allocating buffer.");
     }
   }
@@ -123,11 +157,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
    * Note that the maximum number of values a vector can allocate is Integer.MAX_VALUE / value width.
    *
    * @param valueCount
-   * @throws org.apache.drill.exec.memory.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) {
-    allocateBytes(valueCount * ${type.width});
+    allocateBytes(valueCount * VALUE_WIDTH);
   }
 
   @Override
@@ -139,6 +173,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   private void allocateBytes(final long size) {
+    // TODO: Replace this with MAX_BUFFER_SIZE once all
+    // code is aware of the maximum vector size.
     if (size > MAX_ALLOCATION_SIZE) {
       throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
     }
@@ -150,13 +186,15 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     allocationSizeInBytes = curSize;
   }
 
-/**
- * Allocate new buffer with double capacity, and copy data into the new buffer. Replace vector's buffer with new buffer, and release old one
- *
- * @throws org.apache.drill.exec.memory.OutOfMemoryException if it can't allocate the new buffer
- */
+  /**
+   * Allocate new buffer with double capacity, and copy data into the new buffer. Replace vector's buffer with new buffer, and release old one
+   *
+   * @throws org.apache.drill.exec.memory.OutOfMemoryException if it can't allocate the new buffer
+   */
   public void reAlloc() {
     final long newAllocationSize = allocationSizeInBytes * 2L;
+    // TODO: Replace this with MAX_BUFFER_SIZE once all
+    // code is aware of the maximum vector size.
     if (newAllocationSize > MAX_ALLOCATION_SIZE)  {
       throw new OversizedAllocationException("Unable to expand the buffer. Max allowed buffer size is reached.");
     }
@@ -185,7 +223,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     Preconditions.checkArgument(this.field.getPath().equals(metadata.getNamePart().getName()), "The field %s doesn't match the provided metadata %s.", this.field, metadata);
     final int actualLength = metadata.getBufferLength();
     final int valueCount = metadata.getValueCount();
-    final int expectedLength = valueCount * ${type.width};
+    final int expectedLength = valueCount * VALUE_WIDTH;
     assert actualLength == expectedLength : String.format("Expected to load %d bytes but actually loaded %d bytes", expectedLength, actualLength);
 
     clear();
@@ -220,8 +258,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   public void splitAndTransferTo(int startIndex, int length, ${minor.class}Vector target) {
-    final int startPoint = startIndex * ${type.width};
-    final int sliceLength = length * ${type.width};
+    final int startPoint = startIndex * VALUE_WIDTH;
+    final int sliceLength = length * VALUE_WIDTH;
     target.clear();
     target.data = data.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer;
     target.data.writerIndex(sliceLength);
@@ -229,7 +267,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
   @Override
   public int getPayloadByteCount() {
-    return getAccessor().getValueCount() * ${type.width};
+    return getAccessor().getValueCount() * VALUE_WIDTH;
   }
 
   private class TransferImpl implements TransferPair{
@@ -266,10 +304,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
   public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){
     <#if (type.width > 8)>
-    from.data.getBytes(fromIndex * ${type.width}, data, thisIndex * ${type.width}, ${type.width});
+    from.data.getBytes(fromIndex * VALUE_WIDTH, data, thisIndex * VALUE_WIDTH, VALUE_WIDTH);
     <#else> <#-- type.width <= 8 -->
-    data.set${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width},
-        from.data.get${(minor.javaType!type.javaType)?cap_first}(fromIndex * ${type.width})
+    data.set${(minor.javaType!type.javaType)?cap_first}(thisIndex * VALUE_WIDTH,
+        from.data.get${(minor.javaType!type.javaType)?cap_first}(fromIndex * VALUE_WIDTH)
     );
     </#if> <#-- type.width -->
   }
@@ -281,6 +319,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     copyFrom(fromIndex, thisIndex, from);
   }
 
+  @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});
+  }
+
   public void decrementAllocationMonitor() {
     if (allocationMonitor > 0) {
       allocationMonitor = 0;
@@ -295,7 +338,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   public final class Accessor extends BaseDataValueVector.BaseAccessor {
     @Override
     public int getValueCount() {
-      return data.writerIndex() / ${type.width};
+      return data.writerIndex() / VALUE_WIDTH;
     }
 
     @Override
@@ -305,20 +348,19 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     <#if (type.width > 8)>
     public ${minor.javaType!type.javaType} get(int index) {
-      return data.slice(index * ${type.width}, ${type.width});
+      return data.slice(index * VALUE_WIDTH, VALUE_WIDTH);
     }
 
     <#if (minor.class == "Interval")>
     public void get(int index, ${minor.class}Holder holder){
-
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       holder.months = data.getInt(offsetIndex);
       holder.days = data.getInt(offsetIndex + ${minor.daysOffset});
       holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
     }
 
     public void get(int index, Nullable${minor.class}Holder holder){
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       holder.isSet = 1;
       holder.months = data.getInt(offsetIndex);
       holder.days = data.getInt(offsetIndex + ${minor.daysOffset});
@@ -327,7 +369,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     @Override
     public ${friendlyType} getObject(int index) {
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       final int months  = data.getInt(offsetIndex);
       final int days    = data.getInt(offsetIndex + ${minor.daysOffset});
       final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
@@ -337,7 +379,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     public StringBuilder getAsStringBuilder(int index) {
 
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
 
       int months  = data.getInt(offsetIndex);
       final int days    = data.getInt(offsetIndex + ${minor.daysOffset});
@@ -372,14 +414,13 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     <#elseif (minor.class == "IntervalDay")>
     public void get(int index, ${minor.class}Holder holder){
-
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       holder.days = data.getInt(offsetIndex);
       holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
     }
 
     public void get(int index, Nullable${minor.class}Holder holder){
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       holder.isSet = 1;
       holder.days = data.getInt(offsetIndex);
       holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
@@ -387,7 +428,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     @Override
     public ${friendlyType} getObject(int index) {
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
       final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
       final int  days   = data.getInt(offsetIndex);
       final Period p = new Period();
@@ -395,7 +436,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
     public StringBuilder getAsStringBuilder(int index) {
-      final int offsetIndex = index * ${type.width};
+      final int offsetIndex = index * VALUE_WIDTH;
 
       int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
       final int  days   = data.getInt(offsetIndex);
@@ -419,10 +460,9 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
               append(millis));
     }
 
-    <#elseif (minor.class == "Decimal28Sparse") || (minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")>
-
+    <#elseif minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense">
     public void get(int index, ${minor.class}Holder holder) {
-      holder.start = index * ${type.width};
+      holder.start = index * VALUE_WIDTH;
       holder.buffer = data;
       holder.scale = getField().getScale();
       holder.precision = getField().getPrecision();
@@ -430,7 +470,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     public void get(int index, Nullable${minor.class}Holder holder) {
       holder.isSet = 1;
-      holder.start = index * ${type.width};
+      holder.start = index * VALUE_WIDTH;
       holder.buffer = data;
       holder.scale = getField().getScale();
       holder.precision = getField().getPrecision();
@@ -440,65 +480,61 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public ${friendlyType} getObject(int index) {
       <#if (minor.class == "Decimal28Sparse") || (minor.class == "Decimal38Sparse")>
       // Get the BigDecimal object
-      return DecimalUtility.getBigDecimalFromSparse(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale());
+      return DecimalUtility.getBigDecimalFromSparse(data, index * VALUE_WIDTH, ${minor.nDecimalDigits}, getField().getScale());
       <#else>
-      return DecimalUtility.getBigDecimalFromDense(data, index * ${type.width}, ${minor.nDecimalDigits}, getField().getScale(), ${minor.maxPrecisionDigits}, ${type.width});
+      return DecimalUtility.getBigDecimalFromDense(data, index * VALUE_WIDTH, ${minor.nDecimalDigits}, getField().getScale(), ${minor.maxPrecisionDigits}, VALUE_WIDTH);
       </#if>
     }
 
     <#else>
     public void get(int index, ${minor.class}Holder holder){
       holder.buffer = data;
-      holder.start = index * ${type.width};
+      holder.start = index * VALUE_WIDTH;
     }
 
     public void get(int index, Nullable${minor.class}Holder holder){
       holder.isSet = 1;
       holder.buffer = data;
-      holder.start = index * ${type.width};
+      holder.start = index * VALUE_WIDTH;
     }
 
     @Override
     public ${friendlyType} getObject(int index) {
-      return data.slice(index * ${type.width}, ${type.width})
+      return data.slice(index * VALUE_WIDTH, VALUE_WIDTH)
     }
 
     </#if>
     <#else> <#-- type.width <= 8 -->
-
     public ${minor.javaType!type.javaType} get(int index) {
-      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+      return data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
 
     <#if type.width == 4>
     public long getTwoAsLong(int index) {
-      return data.getLong(index * ${type.width});
+      return data.getLong(index * VALUE_WIDTH);
     }
 
     </#if>
-
     <#if minor.class == "Date">
     @Override
     public ${friendlyType} getObject(int index) {
-        org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-        date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-        return date;
+      org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
+      date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
+      return date;
     }
 
     <#elseif minor.class == "TimeStamp">
     @Override
     public ${friendlyType} getObject(int index) {
-        org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-        date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-        return date;
+      org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
+      date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
+      return date;
     }
 
     <#elseif minor.class == "IntervalYear">
     @Override
     public ${friendlyType} getObject(int index) {
-
       final int value = get(index);
-
       final int years  = (value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
       final int months = (value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
       final Period p = new Period();
@@ -523,18 +559,16 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     <#elseif minor.class == "Time">
     @Override
     public DateTime getObject(int index) {
-
-        org.joda.time.DateTime time = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
-        time = time.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-        return time;
+      org.joda.time.DateTime time = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
+      time = time.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
+      return time;
     }
 
     <#elseif minor.class == "Decimal9" || minor.class == "Decimal18">
     @Override
     public ${friendlyType} getObject(int index) {
-
-        final BigInteger value = BigInteger.valueOf(((${type.boxedType})get(index)).${type.javaType}Value());
-        return new BigDecimal(value, getField().getScale());
+      final BigInteger value = BigInteger.valueOf(((${type.boxedType})get(index)).${type.javaType}Value());
+      return new BigDecimal(value, getField().getScale());
     }
 
     <#else>
@@ -546,191 +580,298 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public ${minor.javaType!type.javaType} getPrimitiveObject(int index) {
       return get(index);
     }
-    </#if>
 
+    </#if>
     public void get(int index, ${minor.class}Holder holder){
       <#if minor.class.startsWith("Decimal")>
       holder.scale = getField().getScale();
       holder.precision = getField().getPrecision();
       </#if>
 
-      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
 
     public void get(int index, Nullable${minor.class}Holder holder){
       holder.isSet = 1;
-      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
+      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
     </#if> <#-- type.width -->
- }
-
- /**
-  * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
-  * vector are accessed by position from the logical start of the vector.  Values should be pushed
-  * onto the vector sequentially, but may be randomly accessed.
-  *   The width of each element is ${type.width} byte(s)
-  *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
-  *
-  * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-  */
-  public 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.
-    *
-    * @param index   position of the bit to set
-    * @param value   value to set
-    */
+  }
+
+  /**
+   * ${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 equivalent Java primitive is '${minor.javaType!type.javaType}'</li>
+   * </ul>
+   *
+   * 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.
+     *
+     * @param index   position of the bit to set
+     * @param value   value to set
+     */
+
   <#if (type.width > 8)>
     public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.setBytes(index * ${type.width}, value, 0, ${type.width});
+      data.setBytes(index * VALUE_WIDTH, value, 0, VALUE_WIDTH);
     }
 
     public void setSafe(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
-      data.setBytes(index * ${type.width}, value, 0, ${type.width});
+      data.setBytes(index * VALUE_WIDTH, value, 0, VALUE_WIDTH);
     }
 
-  <#if (minor.class == "Interval")>
-    public void set(int index, int months, int days, int milliseconds){
-      final int offsetIndex = index * ${type.width};
-      data.setInt(offsetIndex, months);
-      data.setInt((offsetIndex + ${minor.daysOffset}), days);
-      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
+    /**
+     * 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);
     }
 
-    protected void set(int index, ${minor.class}Holder holder){
-      set(index, holder.months, holder.days, holder.milliseconds);
+    /**
+     * 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);
     }
 
-    protected void set(int index, Nullable${minor.class}Holder holder){
-      set(index, holder.months, holder.days, holder.milliseconds);
+    <#if minor.class == "Interval">
+    public void set(int index, int months, int days, int milliseconds) {
+      final int offsetIndex = index * VALUE_WIDTH;
+      data.setInt(offsetIndex, months);
+      data.setInt((offsetIndex + ${minor.daysOffset}), days);
+      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
     }
 
-    public void setSafe(int index, int months, int days, int milliseconds){
+    public void setSafe(int index, int months, int days, int milliseconds) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
       set(index, months, days, milliseconds);
     }
 
-    public void setSafe(int index, Nullable${minor.class}Holder holder){
+    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);
+    }
+
+    public void setSafe(int index, ${minor.class}Holder holder) {
       setSafe(index, holder.months, holder.days, holder.milliseconds);
     }
 
-    public void setSafe(int index, ${minor.class}Holder holder){
+    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);
+    }
+
+    public void setSafe(int index, Nullable${minor.class}Holder holder) {
       setSafe(index, holder.months, holder.days, holder.milliseconds);
     }
 
-  <#elseif (minor.class == "IntervalDay")>
-    public void set(int index, int days, int milliseconds){
-      final int offsetIndex = index * ${type.width};
-      data.setInt(offsetIndex, days);
-      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
+    public void setScalar(int index, Nullable${minor.class}Holder holder) throws VectorOverflowException {
+      setScalar(index, holder.months, holder.days, holder.milliseconds);
     }
 
-    protected void set(int index, ${minor.class}Holder holder){
-      set(index, holder.days, holder.milliseconds);
+    public void setArrayItem(int index, Nullable${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.days, holder.milliseconds);
+    <#elseif minor.class == "IntervalDay">
+    public void set(int index, int days, int milliseconds) {
+      final int offsetIndex = index * VALUE_WIDTH;
+      data.setInt(offsetIndex, days);
+      data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
     }
 
-    public void setSafe(int index, int days, int milliseconds){
+    public void setSafe(int index, int days, int milliseconds) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
       set(index, days, milliseconds);
     }
 
-    public void setSafe(int index, ${minor.class}Holder holder){
-      setSafe(index, holder.days, holder.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 setSafe(int index, Nullable${minor.class}Holder holder){
+    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);
+    }
+
+    public void setSafe(int index, ${minor.class}Holder holder) {
       setSafe(index, holder.days, holder.milliseconds);
     }
 
-  <#elseif (minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse") || (minor.class == "Decimal28Dense") || (minor.class == "Decimal38Dense")>
-    public void set(int index, ${minor.class}Holder holder){
-      set(index, holder.start, holder.buffer);
+    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
+      setScalar(index, holder.days, holder.milliseconds);
     }
 
-    void set(int index, Nullable${minor.class}Holder holder){
-      set(index, holder.start, holder.buffer);
+    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
+      setArrayItem(index, holder.days, holder.milliseconds);
     }
 
-    public void setSafe(int index,  Nullable${minor.class}Holder holder){
-      setSafe(index, holder.start, holder.buffer);
+    protected void set(int index, Nullable${minor.class}Holder holder) {
+      set(index, holder.days, holder.milliseconds);
     }
 
-    public void setSafe(int index,  ${minor.class}Holder holder){
-      setSafe(index, holder.start, holder.buffer);
+    public void setSafe(int index, Nullable${minor.class}Holder holder){
+      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 setSafe(int index, int start, DrillBuf buffer){
+    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()) {
         reAlloc();
       }
       set(index, start, buffer);
     }
 
-  <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
-    public void set(int index, BigDecimal value) {
-      DecimalUtility.getSparseFromBigDecimal(value, data, index * ${type.width},
-           field.getScale(), field.getPrecision(), ${minor.nDecimalDigits});
+    public void setScalar(int index, int start, DrillBuf buffer) throws VectorOverflowException {
+      if (index >= MAX_SCALAR_COUNT) {
+        throw new VectorOverflowException();
+      }
+      setSafe(index, start, buffer);
     }
 
-    public void setSafe(int index, BigDecimal value) {
-      while(index >= getValueCapacity()) {
-        reAlloc();
+    public void setArrayItem(int index, int start, DrillBuf buffer) throws VectorOverflowException {
+      if (index >= MAX_VALUE_COUNT) {
+        throw new VectorOverflowException();
       }
-      set(index, value);
+      setSafe(index, start, buffer);
     }
 
-  </#if>
-    public void set(int index, int start, DrillBuf buffer){
-      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
+    public void set(int index, ${minor.class}Holder holder) {
+      set(index, holder.start, holder.buffer);
     }
 
-  <#else>
-    protected void set(int index, ${minor.class}Holder holder){
-      set(index, holder.start, holder.buffer);
+    public void setSafe(int index, ${minor.class}Holder holder) {
+      setSafe(index, holder.start, holder.buffer);
     }
 
-    public void set(int index, Nullable${minor.class}Holder holder){
-      set(index, holder.start, holder.buffer);
+    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
+      setScalar(index, holder.start, holder.buffer);
     }
 
-    public void set(int index, int start, DrillBuf buffer){
-      data.setBytes(index * ${type.width}, buffer, start, ${type.width});
+    public void setArrayItem(int index, ${minor.class}Holder holder) throws VectorOverflowException {
+      setArrayItem(index, holder.start, holder.buffer);
     }
 
-    public void setSafe(int index, ${minor.class}Holder holder){
-      setSafe(index, holder.start, holder.buffer);
+    void set(int index, Nullable${minor.class}Holder holder) {
+      set(index, holder.start, holder.buffer);
     }
 
-    public void setSafe(int index, Nullable${minor.class}Holder holder){
+    public void setSafe(int index, Nullable${minor.class}Holder holder) {
       setSafe(index, holder.start, holder.buffer);
     }
 
-    public void setSafe(int index, int start, DrillBuf buffer){
+    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,
+           field.getScale(), field.getPrecision(), ${minor.nDecimalDigits});
+    }
+
+    public void setSafe(int index, BigDecimal value) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
-      set(index, holder);
+      set(index, value);
     }
 
-    public void set(int index, Nullable${minor.class}Holder holder){
-      data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
+    public void setScalar(int index, BigDecimal value) throws VectorOverflowException {
+      if (index >= MAX_SCALAR_COUNT) {
+        throw new VectorOverflowException();
+      }
+      setSafe(index, value);
     }
-   </#if>
 
+    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);
+    }
+
+    </#if>
     @Override
     public void generateTestData(int count) {
       setValueCount(count);
@@ -738,46 +879,105 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       final int valueCount = getAccessor().getValueCount();
       for(int i = 0; i < valueCount; i++, even = !even) {
         final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
-        for(int w = 0; w < ${type.width}; w++){
+        for(int w = 0; w < VALUE_WIDTH; w++){
           data.setByte(i + w, b);
         }
       }
     }
 
-   <#else> <#-- type.width <= 8 -->
+  <#else> <#-- type.width <= 8 -->
     public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
+      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()) {
+    public void setSafe(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
+      while(index >= getValueCapacity()) {
         reAlloc();
       }
       set(index, value);
     }
 
-    protected void set(int index, ${minor.class}Holder holder){
-      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
+    /**
+     * 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);
+    }
+
+    protected void set(int index, ${minor.class}Holder holder) {
+      data.set${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH, holder.value);
     }
 
-    public void setSafe(int index, ${minor.class}Holder holder){
+    public void setSafe(int index, ${minor.class}Holder holder) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
       set(index, holder);
     }
 
-    protected void set(int index, Nullable${minor.class}Holder holder){
-      data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
+    public void setScalar(int index, ${minor.class}Holder holder) throws VectorOverflowException {
+      if (index >= MAX_SCALAR_COUNT) {
+        throw new VectorOverflowException();
+      }
+      setSafe(index, holder);
     }
 
-    public void setSafe(int index, Nullable${minor.class}Holder 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);
+    }
+
+    public void setSafe(int index, Nullable${minor.class}Holder holder) {
       while(index >= getValueCapacity()) {
         reAlloc();
       }
       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);
@@ -806,10 +1006,34 @@ 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();
-      final int idx = (${type.width} * valueCount);
+      final int idx = (VALUE_WIDTH * valueCount);
       while(valueCount > getValueCapacity()) {
         reAlloc();
       }
@@ -819,11 +1043,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
         allocationMonitor = 0;
       }
       VectorTrimmer.trim(data, idx);
-      data.writerIndex(valueCount * ${type.width});
+      data.writerIndex(valueCount * VALUE_WIDTH);
     }
   }
 }
-
 </#if> <#-- type.major -->
 </#list>
 </#list>