You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/11/15 21:43:37 UTC

[arrow] 02/04: ARROW-1474:[JAVA] ValueVector hierarchy (Implementation Phase 2)

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git

commit 9ee838ac9538666ae6d7b0b1e680e80410503f7e
Author: siddharth <si...@dremio.com>
AuthorDate: Mon Oct 16 00:39:30 2017 -0700

    ARROW-1474:[JAVA] ValueVector hierarchy (Implementation Phase 2)
    
    Implementation of all scalar types and complex types with corresponding legacy
    versions.
    
    Closes #1203
---
 .../apache/arrow/tools/ArrowFileTestFixtures.java  |   4 +-
 .../org/apache/arrow/tools/EchoServerTest.java     |  84 +--
 .../src/main/codegen/templates/ComplexReaders.java |  24 +-
 .../src/main/codegen/templates/ComplexWriters.java |  51 +-
 .../main/codegen/templates/LegacyUnionVector.java  | 356 ++++++++++
 .../src/main/codegen/templates/MapWriters.java     |   4 +-
 .../codegen/templates/NullableValueVectors.java    | 787 +++------------------
 .../main/codegen/templates/UnionListWriter.java    |  11 +-
 .../src/main/codegen/templates/UnionVector.java    | 212 +++---
 .../src/main/codegen/templates/UnionWriter.java    |  26 +-
 .../arrow/vector/BaseNullableFixedWidthVector.java | 315 ++++++---
 .../vector/BaseNullableVariableWidthVector.java    | 412 ++++++++---
 .../org/apache/arrow/vector/BaseValueVector.java   |  33 +-
 .../org/apache/arrow/vector/BitVectorHelper.java   | 115 ++-
 ...bleIntVector.java => NullableBigIntVector.java} | 210 ++++--
 .../org/apache/arrow/vector/NullableBitVector.java | 502 +++++++++++++
 ...leIntVector.java => NullableDateDayVector.java} | 187 +++--
 .../arrow/vector/NullableDateMilliVector.java      | 408 +++++++++++
 .../apache/arrow/vector/NullableDecimalVector.java | 454 ++++++++++++
 ...bleIntVector.java => NullableFloat4Vector.java} | 207 ++++--
 ...bleIntVector.java => NullableFloat8Vector.java} | 209 ++++--
 .../org/apache/arrow/vector/NullableIntVector.java | 140 +++-
 .../arrow/vector/NullableIntervalDayVector.java    | 429 +++++++++++
 ...Vector.java => NullableIntervalYearVector.java} | 176 +++--
 .../arrow/vector/NullableSmallIntVector.java       | 431 +++++++++++
 ...IntVector.java => NullableTimeMicroVector.java} | 209 ++++--
 ...IntVector.java => NullableTimeMilliVector.java} | 198 ++++--
 ...eIntVector.java => NullableTimeNanoVector.java} | 209 ++++--
 ...leIntVector.java => NullableTimeSecVector.java} | 188 +++--
 .../vector/NullableTimeStampMicroTZVector.java     | 219 ++++++
 .../arrow/vector/NullableTimeStampMicroVector.java | 220 ++++++
 .../vector/NullableTimeStampMilliTZVector.java     | 218 ++++++
 .../arrow/vector/NullableTimeStampMilliVector.java | 218 ++++++
 .../vector/NullableTimeStampNanoTZVector.java      | 219 ++++++
 .../arrow/vector/NullableTimeStampNanoVector.java  | 219 ++++++
 .../arrow/vector/NullableTimeStampSecTZVector.java | 217 ++++++
 .../arrow/vector/NullableTimeStampSecVector.java   | 219 ++++++
 .../arrow/vector/NullableTimeStampVector.java      | 253 +++++++
 .../apache/arrow/vector/NullableTinyIntVector.java | 432 +++++++++++
 ...ableIntVector.java => NullableUInt1Vector.java} | 115 +--
 ...ableIntVector.java => NullableUInt2Vector.java} | 114 +--
 ...ableIntVector.java => NullableUInt4Vector.java} |  74 +-
 ...ableIntVector.java => NullableUInt8Vector.java} |  94 ++-
 ...harVector.java => NullableVarBinaryVector.java} | 157 ++--
 .../apache/arrow/vector/NullableVarCharVector.java | 100 ++-
 .../java/org/apache/arrow/vector/ValueVector.java  |   8 +
 .../org/apache/arrow/vector/VectorSchemaRoot.java  |   2 +-
 .../org/apache/arrow/vector/VectorUnloader.java    |  12 +-
 .../java/org/apache/arrow/vector/ZeroVector.java   |   6 +
 .../arrow/vector/complex/AbstractMapVector.java    |   4 +-
 .../vector/complex/BaseRepeatedValueVector.java    | 174 +++--
 .../arrow/vector/complex/FixedSizeListVector.java  | 238 ++++---
 .../complex/LegacyBaseRepeatedValueVector.java     | 240 +++++++
 .../vector/complex/LegacyFixedSizeListVector.java  | 298 ++++++++
 .../arrow/vector/complex/LegacyListVector.java     | 288 ++++++++
 .../arrow/vector/complex/LegacyMapVector.java      | 229 ++++++
 .../apache/arrow/vector/complex/ListVector.java    | 424 ++++++-----
 .../org/apache/arrow/vector/complex/MapVector.java |  96 +--
 .../arrow/vector/complex/NullableMapVector.java    | 299 +++++---
 .../vector/complex/impl/NullableMapReaderImpl.java |   2 +-
 .../vector/complex/impl/PromotableWriter.java      |   2 +-
 .../vector/complex/impl/SingleMapReaderImpl.java   |   2 +-
 .../complex/impl/UnionFixedSizeListReader.java     |   4 +-
 .../arrow/vector/complex/impl/UnionListReader.java |  12 +-
 .../arrow/vector/dictionary/DictionaryEncoder.java |  28 +-
 .../org/apache/arrow/vector/file/ArrowWriter.java  |   2 +-
 .../arrow/vector/file/json/JsonFileReader.java     | 341 ++++-----
 .../arrow/vector/file/json/JsonFileWriter.java     | 198 +++---
 .../org/apache/arrow/vector/util/Validator.java    |  10 +-
 .../org/apache/arrow/vector/TestBitVector.java     |  35 +-
 .../org/apache/arrow/vector/TestDecimalVector.java |  12 +-
 .../apache/arrow/vector/TestDictionaryVector.java  |   4 +-
 .../arrow/vector/TestFixedSizeListVector.java      |  33 +-
 .../org/apache/arrow/vector/TestListVector.java    | 196 +++--
 .../org/apache/arrow/vector/TestUnionVector.java   | 269 ++++---
 .../org/apache/arrow/vector/TestValueVector.java   | 222 +++---
 .../org/apache/arrow/vector/TestVectorReAlloc.java |   5 +-
 .../apache/arrow/vector/TestVectorUnloadLoad.java  |   6 +-
 .../vector/complex/impl/TestPromotableWriter.java  |  21 +-
 .../vector/complex/writer/TestComplexWriter.java   |   4 +-
 .../org/apache/arrow/vector/file/BaseFileTest.java | 139 ++--
 .../apache/arrow/vector/file/TestArrowFile.java    |  28 +-
 .../apache/arrow/vector/file/TestArrowStream.java  |   6 +-
 .../arrow/vector/file/TestArrowStreamPipe.java     |  14 +-
 .../arrow/vector/file/json/TestJSONFile.java       |   2 +-
 85 files changed, 10499 insertions(+), 3095 deletions(-)

diff --git a/java/tools/src/test/java/org/apache/arrow/tools/ArrowFileTestFixtures.java b/java/tools/src/test/java/org/apache/arrow/tools/ArrowFileTestFixtures.java
index 6d9a6c1..c56a5a3 100644
--- a/java/tools/src/test/java/org/apache/arrow/tools/ArrowFileTestFixtures.java
+++ b/java/tools/src/test/java/org/apache/arrow/tools/ArrowFileTestFixtures.java
@@ -78,8 +78,8 @@ public class ArrowFileTestFixtures {
   static void validateContent(int count, VectorSchemaRoot root) {
     Assert.assertEquals(count, root.getRowCount());
     for (int i = 0; i < count; i++) {
-      Assert.assertEquals(i, root.getVector("int").getAccessor().getObject(i));
-      Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getAccessor().getObject(i));
+      Assert.assertEquals(i, root.getVector("int").getObject(i));
+      Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getObject(i));
     }
   }
 
diff --git a/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java b/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
index ecac6d6..89714e4 100644
--- a/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
+++ b/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
@@ -104,10 +104,10 @@ public class EchoServerTest {
       for (int i = 0; i < batches; i++) {
         vector.allocateNew(16);
         for (int j = 0; j < 8; j++) {
-          vector.getMutator().set(j, j + i);
-          vector.getMutator().set(j + 8, 0, (byte) (j + i));
+          vector.set(j, j + i);
+          vector.set(j + 8, 0, (byte) (j + i));
         }
-        vector.getMutator().setValueCount(16);
+        vector.setValueCount(16);
         root.setRowCount(16);
         writer.writeBatch();
       }
@@ -120,10 +120,10 @@ public class EchoServerTest {
       for (int i = 0; i < batches; i++) {
         Assert.assertTrue(reader.loadNextBatch());
         assertEquals(16, reader.getVectorSchemaRoot().getRowCount());
-        assertEquals(16, readVector.getAccessor().getValueCount());
+        assertEquals(16, readVector.getValueCount());
         for (int j = 0; j < 8; j++) {
-          assertEquals(j + i, readVector.getAccessor().get(j));
-          assertTrue(readVector.getAccessor().isNull(j + 8));
+          assertEquals(j + i, readVector.get(j));
+          assertTrue(readVector.isNull(j + 8));
         }
       }
       Assert.assertFalse(reader.loadNextBatch());
@@ -169,20 +169,18 @@ public class EchoServerTest {
                  FieldType.nullable(VARCHAR.getType()),
                  allocator)) {
       writeVector.allocateNewSafe();
-      NullableIntVector.Mutator mutator = writeVector.getMutator();
-      mutator.set(0, 0);
-      mutator.set(1, 1);
-      mutator.set(3, 2);
-      mutator.set(4, 1);
-      mutator.set(5, 2);
-      mutator.setValueCount(6);
+      writeVector.set(0, 0);
+      writeVector.set(1, 1);
+      writeVector.set(3, 2);
+      writeVector.set(4, 1);
+      writeVector.set(5, 2);
+      writeVector.setValueCount(6);
 
       writeDictionaryVector.allocateNewSafe();
-      NullableVarCharVector.Mutator dictionaryMutator = writeDictionaryVector.getMutator();
-      dictionaryMutator.set(0, "foo".getBytes(StandardCharsets.UTF_8));
-      dictionaryMutator.set(1, "bar".getBytes(StandardCharsets.UTF_8));
-      dictionaryMutator.set(2, "baz".getBytes(StandardCharsets.UTF_8));
-      dictionaryMutator.setValueCount(3);
+      writeDictionaryVector.set(0, "foo".getBytes(StandardCharsets.UTF_8));
+      writeDictionaryVector.set(1, "bar".getBytes(StandardCharsets.UTF_8));
+      writeDictionaryVector.set(2, "baz".getBytes(StandardCharsets.UTF_8));
+      writeDictionaryVector.setValueCount(3);
 
       List<Field> fields = ImmutableList.of(writeVector.getField());
       List<FieldVector> vectors = ImmutableList.of((FieldVector) writeVector);
@@ -210,23 +208,21 @@ public class EchoServerTest {
         Assert.assertNotNull(readEncoding);
         Assert.assertEquals(1L, readEncoding.getId());
 
-        FieldVector.Accessor accessor = readVector.getAccessor();
-        Assert.assertEquals(6, accessor.getValueCount());
-        Assert.assertEquals(0, accessor.getObject(0));
-        Assert.assertEquals(1, accessor.getObject(1));
-        Assert.assertEquals(null, accessor.getObject(2));
-        Assert.assertEquals(2, accessor.getObject(3));
-        Assert.assertEquals(1, accessor.getObject(4));
-        Assert.assertEquals(2, accessor.getObject(5));
+        Assert.assertEquals(6, readVector.getValueCount());
+        Assert.assertEquals(0, readVector.getObject(0));
+        Assert.assertEquals(1, readVector.getObject(1));
+        Assert.assertEquals(null, readVector.getObject(2));
+        Assert.assertEquals(2, readVector.getObject(3));
+        Assert.assertEquals(1, readVector.getObject(4));
+        Assert.assertEquals(2, readVector.getObject(5));
 
         Dictionary dictionary = reader.lookup(1L);
         Assert.assertNotNull(dictionary);
-        NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary
-            .getVector()).getAccessor();
-        Assert.assertEquals(3, dictionaryAccessor.getValueCount());
-        Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-        Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
-        Assert.assertEquals(new Text("baz"), dictionaryAccessor.getObject(2));
+        NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary.getVector());
+        Assert.assertEquals(3, dictionaryVector.getValueCount());
+        Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
+        Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
+        Assert.assertEquals(new Text("baz"), dictionaryVector.getObject(2));
       }
     }
   }
@@ -243,9 +239,9 @@ public class EchoServerTest {
       // [['foo', 'bar'], ['foo'], ['bar']] -> [[0, 1], [0], [1]]
 
       writeDictionaryVector.allocateNew();
-      writeDictionaryVector.getMutator().set(0, "foo".getBytes(StandardCharsets.UTF_8));
-      writeDictionaryVector.getMutator().set(1, "bar".getBytes(StandardCharsets.UTF_8));
-      writeDictionaryVector.getMutator().setValueCount(2);
+      writeDictionaryVector.set(0, "foo".getBytes(StandardCharsets.UTF_8));
+      writeDictionaryVector.set(1, "bar".getBytes(StandardCharsets.UTF_8));
+      writeDictionaryVector.setValueCount(2);
 
       writeVector.addOrGetVector(new FieldType(true, MinorType.INT.getType(), writeEncoding, null));
       writeVector.allocateNew();
@@ -297,19 +293,17 @@ public class EchoServerTest {
         Assert.assertEquals(2L, encoding.getId());
         Assert.assertEquals(new Int(32, true), encoding.getIndexType());
 
-        ListVector.Accessor accessor = readVector.getAccessor();
-        Assert.assertEquals(3, accessor.getValueCount());
-        Assert.assertEquals(Arrays.asList(0, 1), accessor.getObject(0));
-        Assert.assertEquals(Arrays.asList(0), accessor.getObject(1));
-        Assert.assertEquals(Arrays.asList(1), accessor.getObject(2));
+        Assert.assertEquals(3, readVector.getValueCount());
+        Assert.assertEquals(Arrays.asList(0, 1), readVector.getObject(0));
+        Assert.assertEquals(Arrays.asList(0), readVector.getObject(1));
+        Assert.assertEquals(Arrays.asList(1), readVector.getObject(2));
 
         Dictionary readDictionary = reader.lookup(2L);
         Assert.assertNotNull(readDictionary);
-        NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector)
-            readDictionary.getVector()).getAccessor();
-        Assert.assertEquals(2, dictionaryAccessor.getValueCount());
-        Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-        Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
+        NullableVarCharVector dictionaryVector = ((NullableVarCharVector) readDictionary.getVector());
+        Assert.assertEquals(2, dictionaryVector.getValueCount());
+        Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
+        Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
       }
     }
   }
diff --git a/java/vector/src/main/codegen/templates/ComplexReaders.java b/java/vector/src/main/codegen/templates/ComplexReaders.java
index 7910649..60347a2 100644
--- a/java/vector/src/main/codegen/templates/ComplexReaders.java
+++ b/java/vector/src/main/codegen/templates/ComplexReaders.java
@@ -70,11 +70,7 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
   
   public boolean isSet(){
     <#if nullMode == "Nullable">
-      <#if minor.class != "Int" && minor.class != "VarChar">
-        return !vector.getAccessor().isNull(idx());
-      <#else>
         return !vector.isNull(idx());
-      </#if>
     <#else>
     return true;
     </#if>
@@ -97,19 +93,11 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
   </#if>
 
   public void read(Nullable${minor.class?cap_first}Holder h){
-    <#if minor.class != "Int" && minor.class != "VarChar">
-      vector.getAccessor().get(idx(), h);
-    <#else>
-      vector.get(idx(), h);
-    </#if>
+    vector.get(idx(), h);
   }
   
   public ${friendlyType} read${safeType}(){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-      return vector.getObject(idx());
-    <#else>
-      return vector.getAccessor().getObject(idx());
-    </#if>
+    return vector.getObject(idx());
   }
 
   <#if minor.class == "TimeStampSec" ||
@@ -118,7 +106,7 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
        minor.class == "TimeStampNano">
   @Override
   public ${minor.boxedType} read${minor.boxedType}(){
-    return vector.getAccessor().get(idx());
+    return vector.get(idx());
   }
   </#if>
   
@@ -127,11 +115,7 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
   }
   
   public Object readObject(){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-      return (Object)vector.getObject(idx());
-    <#else>
-      return vector.getAccessor().getObject(idx());
-    </#if>
+    return (Object)vector.getObject(idx());
   }
 }
 </#if>
diff --git a/java/vector/src/main/codegen/templates/ComplexWriters.java b/java/vector/src/main/codegen/templates/ComplexWriters.java
index 77f6594..406bbb3 100644
--- a/java/vector/src/main/codegen/templates/ComplexWriters.java
+++ b/java/vector/src/main/codegen/templates/ComplexWriters.java
@@ -39,16 +39,9 @@ package org.apache.arrow.vector.complex.impl;
 @SuppressWarnings("unused")
 public class ${eName}WriterImpl extends AbstractFieldWriter {
 
-  <#if minor.class != "Int" && minor.class != "VarChar">
-    private final Nullable${name}Vector.Mutator mutator;
-  </#if>
-
   final Nullable${name}Vector vector;
 
   public ${eName}WriterImpl(Nullable${name}Vector vector) {
-    <#if minor.class != "Int" && minor.class != "VarChar">
-      this.mutator = vector.getMutator();
-    </#if>
     this.vector = vector;
   }
 
@@ -108,51 +101,31 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
   <#else>
 
   public void write(${minor.class}Holder h) {
-    <#if minor.class != "Int" && minor.class != "VarChar">
-      mutator.setSafe(idx(), h);
-      vector.getMutator().setValueCount(idx()+1);
-    <#else>
-        vector.setSafe(idx(), h);
-        vector.setValueCount(idx()+1);
-    </#if>
+    vector.setSafe(idx(), h);
+    vector.setValueCount(idx()+1);
   }
 
   public void write(Nullable${minor.class}Holder h) {
-    <#if minor.class != "Int" && minor.class != "VarChar">
-      mutator.setSafe(idx(), h);
-      vector.getMutator().setValueCount(idx()+1);
-    <#else>
-      vector.setSafe(idx(), h);
-      vector.setValueCount(idx()+1);
-    </#if>
+    vector.setSafe(idx(), h);
+    vector.setValueCount(idx()+1);
   }
 
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
-    <#if minor.class != "Int" && minor.class != "VarChar">
-      mutator.setSafe(idx()<#if mode == "Nullable">, 1</#if><#list fields as field><#if field.include!true >, ${field.name}</#if></#list>);
-      vector.getMutator().setValueCount(idx()+1);
-    <#else>
-      vector.setSafe(idx()<#if mode == "Nullable">, 1</#if><#list fields as field><#if field.include!true >, ${field.name}</#if></#list>);
-      vector.setValueCount(idx()+1);
-    </#if>
+    vector.setSafe(idx()<#if mode == "Nullable">, 1</#if><#list fields as field><#if field.include!true >, ${field.name}</#if></#list>);
+    vector.setValueCount(idx()+1);
   }
-  <#if minor.class == "Decimal">
 
+  <#if minor.class == "Decimal">
   public void write${minor.class}(${friendlyType} value) {
-    mutator.setSafe(idx(), value);
-    vector.getMutator().setValueCount(idx()+1);
+    vector.setSafe(idx(), value);
+    vector.setValueCount(idx()+1);
   }
   </#if>
-  <#if mode == "Nullable">
 
+  <#if mode == "Nullable">
   public void writeNull() {
-    <#if minor.class != "Int" && minor.class != "VarChar">
-        mutator.setNull(idx());
-        vector.getMutator().setValueCount(idx()+1);
-    <#else>
-        vector.setNull(idx());
-        vector.setValueCount(idx()+1);
-    </#if>
+    vector.setNull(idx());
+    vector.setValueCount(idx()+1);
   }
   </#if>
   </#if>
diff --git a/java/vector/src/main/codegen/templates/LegacyUnionVector.java b/java/vector/src/main/codegen/templates/LegacyUnionVector.java
new file mode 100644
index 0000000..b9cb6df
--- /dev/null
+++ b/java/vector/src/main/codegen/templates/LegacyUnionVector.java
@@ -0,0 +1,356 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="/org/apache/arrow/vector/complex/LegacyUnionVector.java" />
+
+
+<#include "/@includes/license.ftl" />
+
+        package org.apache.arrow.vector.complex;
+
+<#include "/@includes/vv_imports.ftl" />
+        import com.google.common.collect.ImmutableList;
+        import java.util.ArrayList;
+        import java.util.Collections;
+        import java.util.Iterator;
+        import org.apache.arrow.vector.BaseDataValueVector;
+        import org.apache.arrow.vector.complex.impl.ComplexCopier;
+        import org.apache.arrow.vector.util.CallBack;
+        import org.apache.arrow.vector.schema.ArrowFieldNode;
+
+        import static org.apache.arrow.vector.types.UnionMode.Sparse;
+
+
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+@SuppressWarnings("unused")
+
+
+/**
+ * A vector which can hold values of different types. It does so by using a MapVector which contains a vector for each
+ * primitive type that is stored. MapVector is used in order to take advantage of its serialization/deserialization methods,
+ * as well as the addOrGet method.
+ *
+ * For performance reasons, UnionVector stores a cached reference to each subtype vector, to avoid having to do the map lookup
+ * each time the vector is accessed.
+ * Source code generated using FreeMarker template ${.template_name}
+ */
+public class LegacyUnionVector implements FieldVector {
+
+   private Accessor accessor = new Accessor();
+   private Mutator mutator = new Mutator();
+   private final UnionVector unionVector;
+
+   public LegacyUnionVector(String name, BufferAllocator allocator, CallBack callBack) {
+     unionVector = new UnionVector(name, allocator, callBack);
+   }
+
+   public BufferAllocator getAllocator() {
+      return unionVector.getAllocator();
+   }
+
+   @Override
+   public MinorType getMinorType() {
+      return MinorType.UNION;
+   }
+
+   @Override
+   public void initializeChildrenFromFields(List<Field> children) {
+      unionVector.initializeChildrenFromFields(children);
+   }
+
+   @Override
+   public List<FieldVector> getChildrenFromFields() {
+      return unionVector.getChildrenFromFields();
+   }
+
+   @Override
+   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
+      unionVector.loadFieldBuffers(fieldNode, ownBuffers);
+   }
+
+   @Override
+   public List<ArrowBuf> getFieldBuffers() {
+      return unionVector.getFieldBuffers();
+   }
+
+   @Override
+   public List<BufferBacked> getFieldInnerVectors() {
+      return unionVector.getFieldInnerVectors();
+   }
+
+   @Override
+   public long getValidityBufferAddress() {
+      return unionVector.getValidityBufferAddress();
+   }
+
+   @Override
+   public long getDataBufferAddress() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public long getOffsetBufferAddress() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getValidityBuffer() {
+      return unionVector.getValidityBuffer();
+   }
+
+   @Override
+   public ArrowBuf getDataBuffer() { throw new UnsupportedOperationException(); }
+
+   @Override
+   public ArrowBuf getOffsetBuffer() { throw new UnsupportedOperationException(); }
+
+   public NullableMapVector getMap() {
+      return unionVector.getMap();
+   }
+  <#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/>
+      <#assign lowerCaseName = name?lower_case/>
+      <#if !minor.typeParams?? >
+
+   private Nullable${name}Vector ${uncappedName}Vector;
+
+   public Nullable${name}Vector get${name}Vector() {
+      return unionVector.get${name}Vector();
+   }
+      </#if>
+    </#list>
+  </#list>
+
+   public ListVector getList() {
+      return unionVector.getList();
+   }
+
+   public int getTypeValue(int index) {
+      return unionVector.getTypeValue(index);
+   }
+
+   @Override
+   public void allocateNew() throws OutOfMemoryException {
+     unionVector.allocateNew();
+   }
+
+   @Override
+   public boolean allocateNewSafe() {
+     return unionVector.allocateNewSafe();
+   }
+
+   @Override
+   public void reAlloc() {
+      unionVector.reAlloc();
+   }
+
+   @Override
+   public void setInitialCapacity(int numRecords) {
+   }
+
+   @Override
+   public int getValueCapacity() {
+      return unionVector.getValueCapacity();
+   }
+
+   @Override
+   public void close() {
+     unionVector.close();
+   }
+
+   @Override
+   public void clear() {
+      unionVector.clear();
+   }
+
+   @Override
+   public Field getField() {
+      return unionVector.getField();
+   }
+
+   @Override
+   public TransferPair getTransferPair(BufferAllocator allocator) {
+      return unionVector.getTransferPair(allocator);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      return unionVector.getTransferPair(ref, allocator);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
+      return unionVector.getTransferPair(ref, allocator, callBack);
+   }
+
+   @Override
+   public TransferPair makeTransferPair(ValueVector target) {
+      return unionVector.makeTransferPair(((LegacyUnionVector)target).unionVector);
+   }
+
+   public void copyFrom(int inIndex, int outIndex, UnionVector from) {
+      unionVector.copyFrom(inIndex, outIndex, from);
+   }
+
+   public void copyFromSafe(int inIndex, int outIndex, UnionVector from) {
+      unionVector.copyFromSafe(inIndex, outIndex, from);
+   }
+
+   public FieldVector addVector(FieldVector v) {
+     return unionVector.addVector(v);
+   }
+
+   @Override
+   public Accessor getAccessor() {
+      return accessor;
+   }
+
+   @Override
+   public Mutator getMutator() {
+      return mutator;
+   }
+
+   @Override
+   public FieldReader getReader() {
+      return unionVector.getReader();
+   }
+
+   public FieldWriter getWriter() {
+      return unionVector.getWriter();
+   }
+
+   @Override
+   public int getBufferSize() {
+      return unionVector.getBufferSize();
+   }
+
+   @Override
+   public int getBufferSizeFor(final int valueCount) {
+     return unionVector.getBufferSizeFor(valueCount);
+   }
+
+   @Override
+   public ArrowBuf[] getBuffers(boolean clear) {
+     return unionVector.getBuffers(clear);
+   }
+
+   @Override
+   public Iterator<ValueVector> iterator() {
+      return unionVector.iterator();
+   }
+
+   public class Accessor extends BaseValueVector.BaseAccessor {
+
+      @Override
+      public Object getObject(int index) {
+        return unionVector.getObject(index);
+      }
+
+      public byte[] get(int index) {
+         return unionVector.get(index);
+      }
+
+      public void get(int index, ComplexHolder holder) {
+      }
+
+      public void get(int index, UnionHolder holder) {
+         unionVector.get(index, holder);
+      }
+
+      public int getNullCount() {
+         return unionVector.getNullCount();
+      }
+
+      @Override
+      public int getValueCount() {
+         return unionVector.getValueCount();
+      }
+
+      @Override
+      public boolean isNull(int index) {
+         return unionVector.isNull(index);
+      }
+
+      public int isSet(int index) {
+         return unionVector.isSet(index);
+      }
+   }
+
+   public class Mutator extends BaseValueVector.BaseMutator {
+
+      UnionWriter writer;
+
+      @Override
+      public void setValueCount(int valueCount) {
+         unionVector.setValueCount(valueCount);
+      }
+
+      public void setSafe(int index, UnionHolder holder) {
+        unionVector.setSafe(index, holder);
+      }
+    <#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/>
+        <#if !minor.typeParams?? >
+      public void setSafe(int index, Nullable${name}Holder holder) {
+         unionVector.setSafe(index, holder);
+      }
+
+        </#if>
+      </#list>
+    </#list>
+
+      public void setType(int index, MinorType type) {
+         unionVector.setType(index, type);
+      }
+
+      @Override
+      public void reset() { }
+
+      @Override
+      public void generateTestData(int values) { }
+   }
+
+   @Override
+   @Deprecated
+   public int getValueCount() { return getAccessor().getValueCount(); }
+
+   @Override
+   @Deprecated
+   public void setValueCount(int valueCount) { getMutator().setValueCount(valueCount);}
+
+   @Override
+   @Deprecated
+   public Object getObject(int index) { return getAccessor().getObject(index); }
+
+   @Override
+   @Deprecated
+   public int getNullCount() { return getAccessor().getNullCount(); }
+
+   @Override
+   @Deprecated
+   public boolean isNull(int index) { return getAccessor().isNull(index); }
+}
diff --git a/java/vector/src/main/codegen/templates/MapWriters.java b/java/vector/src/main/codegen/templates/MapWriters.java
index b89f914..6afd508 100644
--- a/java/vector/src/main/codegen/templates/MapWriters.java
+++ b/java/vector/src/main/codegen/templates/MapWriters.java
@@ -184,7 +184,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
   }
 
   public void setValueCount(int count) {
-    container.getMutator().setValueCount(count);
+    container.setValueCount(count);
   }
 
   @Override
@@ -199,7 +199,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
   public void start() {
     <#if mode == "Single">
     <#else>
-    container.getMutator().setIndexDefined(idx());
+    container.setIndexDefined(idx());
     </#if>
   }
 
diff --git a/java/vector/src/main/codegen/templates/NullableValueVectors.java b/java/vector/src/main/codegen/templates/NullableValueVectors.java
index 5d1f5a3..8e5b10b 100644
--- a/java/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/java/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -19,14 +19,8 @@
 <#list vv.types as type>
 <#list type.minor as minor>
 
-<#if minor.class == "Int" || minor.class == "VarChar">
 <#assign className = "LegacyNullable${minor.class}Vector" />
 <#assign valuesName = "Nullable${minor.class}Vector" />
-<#else>
-<#assign className = "Nullable${minor.class}Vector" />
-<#assign valuesName = "${minor.class}Vector" />
-</#if>
-
 <#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
 
 <@pp.changeOutputFile name="/org/apache/arrow/vector/${className}.java" />
@@ -50,35 +44,21 @@ import org.apache.arrow.flatbuf.Precision;
  * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
  */
 @SuppressWarnings("unused")
-<#if minor.class == "Int" || minor.class == "VarChar">
 @Deprecated
-</#if>
 public final class ${className} extends BaseValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector, FieldVector {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class);
 
 protected final static byte[] emptyByteArray = new byte[]{};
 
-  <#if minor.class != "Int" && minor.class != "VarChar">
-  private final FieldReader reader = new ${minor.class}ReaderImpl(${className}.this);
-  </#if>
-
   private final String bitsField = "$bits$";
   private final String valuesField = "$values$";
 
-  <#if minor.class != "Int" && minor.class != "VarChar">
-  private final Field field;
-  </#if>
-
   final BitVector bits = new BitVector(bitsField, allocator);
   final ${valuesName} values;
 
   private final Mutator mutator;
   private final Accessor accessor;
 
-  <#if minor.class != "Int" && minor.class != "VarChar">
-  private final List<BufferBacked> innerVectors;
-  </#if>
-
   <#if minor.typeParams??>
     <#assign typeParams = minor.typeParams?reverse>
     <#list typeParams as typeParam>
@@ -122,33 +102,12 @@ protected final static byte[] emptyByteArray = new byte[]{};
     </#if>
     this.mutator = new Mutator();
     this.accessor = new Accessor();
-    <#if minor.class != "Int" && minor.class != "VarChar">
-    this.field = new Field(name, fieldType, null);
-    innerVectors = Collections.unmodifiableList(Arrays.<BufferBacked>asList(
-        bits,
-        <#if type.major = "VarLen">
-        values.offsetVector,
-        </#if>
-        values
-    ));
-    </#if>
   }
 
-  <#if minor.class != "Int" && minor.class != "VarChar">
-  /* not needed for new vectors */
-  public BitVector getValidityVector() {
-    return bits;
-  }
-  </#if>
-
   @Override
   public List<BufferBacked> getFieldInnerVectors() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getFieldInnerVectors();
-    <#else>
-        return innerVectors;
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getFieldInnerVectors();
   }
 
   @Override
@@ -165,123 +124,61 @@ protected final static byte[] emptyByteArray = new byte[]{};
 
   @Override
   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.loadFieldBuffers(fieldNode, ownBuffers);
-    <#else>
-    <#if type.major = "VarLen">
-    // variable width values: truncate offset vector buffer to size (#1)
-    org.apache.arrow.vector.BaseDataValueVector.truncateBufferBasedOnSize(ownBuffers, 1,
-        values.offsetVector.getBufferSizeFor(
-        fieldNode.getLength() == 0? 0 : fieldNode.getLength() + 1));
-    mutator.lastSet = fieldNode.getLength() - 1;
-    <#else>
-    // fixed width values truncate value vector to size (#1)
-    org.apache.arrow.vector.BaseDataValueVector.truncateBufferBasedOnSize(ownBuffers, 1, values.getBufferSizeFor(fieldNode.getLength()));
-    </#if>
-    org.apache.arrow.vector.BaseDataValueVector.load(fieldNode, getFieldInnerVectors(), ownBuffers);
-    bits.valueCount = fieldNode.getLength();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.loadFieldBuffers(fieldNode, ownBuffers);
   }
 
   public List<ArrowBuf> getFieldBuffers() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getFieldBuffers();
-    <#else>
-        return org.apache.arrow.vector.BaseDataValueVector.unload(getFieldInnerVectors());
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getFieldBuffers();
   }
 
   @Override
   public Field getField() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getField();
-    <#else>
-      return field;
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getField();
   }
 
   @Override
   public MinorType getMinorType() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getMinorType();
-    <#else>
-        return MinorType.${minor.class?upper_case};
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getMinorType();
   }
 
   @Override
   public FieldReader getReader(){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getReader();
-    <#else>
-        return reader;
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getReader();
   }
 
   @Override
   public int getValueCapacity(){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getValueCapacity();
-    <#else>
-        return Math.min(bits.getValueCapacity(), values.getValueCapacity());
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getValueCapacity();
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getBuffers(clear);
-    <#else>
-    final ArrowBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), values.getBuffers(false), ArrowBuf.class);
-    if (clear) {
-      for (final ArrowBuf buffer:buffers) {
-        buffer.retain(1);
-      }
-      clear();
-    }
-    return buffers;
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getBuffers(clear);
   }
 
   @Override
   public void close() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.close();
-    <#else>
-        bits.close();
-        values.close();
-        super.close();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.close();
   }
 
   @Override
   public void clear() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.clear();
-    <#else>
-        bits.clear();
-        values.clear();
-        super.clear();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.clear();
   }
 
   @Override
   public int getBufferSize(){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getBufferSize();
-    <#else>
-        return values.getBufferSize() + bits.getBufferSize();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getBufferSize();
   }
 
   @Override
@@ -289,14 +186,8 @@ protected final static byte[] emptyByteArray = new byte[]{};
     if (valueCount == 0) {
       return 0;
     }
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getBufferSizeFor(valueCount);
-    <#else>
-        return values.getBufferSizeFor(valueCount)
-          + bits.getBufferSizeFor(valueCount);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getBufferSizeFor(valueCount);
   }
 
   public ArrowBuf getBuffer() {
@@ -309,93 +200,38 @@ protected final static byte[] emptyByteArray = new byte[]{};
 
   @Override
   public void setInitialCapacity(int numRecords) {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.setInitialCapacity(numRecords);
-    <#else>
-        bits.setInitialCapacity(numRecords);
-        values.setInitialCapacity(numRecords);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.setInitialCapacity(numRecords);
   }
 
   @Override
   public void allocateNew() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.allocateNew();
-    <#else>
-    if(!allocateNewSafe()){
-      throw new OutOfMemoryException("Failure while allocating buffer.");
-    }
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.allocateNew();
   }
 
   @Override
   public boolean allocateNewSafe() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.allocateNewSafe();
-    <#else>
-    /* Boolean to keep track if all the memory allocations were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    boolean success = false;
-    try {
-      success = values.allocateNewSafe() && bits.allocateNewSafe();
-    } finally {
-      if (!success) {
-        clear();
-      }
-    }
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-    return success;
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.allocateNewSafe();
   }
 
   @Override
   public void reAlloc() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.reAlloc();
-    <#else>
-        bits.reAlloc();
-        values.reAlloc();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.reAlloc();
   }
 
   public void reset() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.reset();
-    <#else>
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.reset();
   }
 
   <#if type.major == "VarLen">
   @Override
   public void allocateNew(int totalBytes, int valueCount) {
-    <#if minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        values.allocateNew(totalBytes, valueCount);
-    <#else>
-    try {
-      values.allocateNew(totalBytes, valueCount);
-      bits.allocateNew(valueCount);
-    } catch(RuntimeException e) {
-      clear();
-      throw e;
-    }
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.allocateNew(totalBytes, valueCount);
   }
 
   @Override
@@ -411,21 +247,8 @@ protected final static byte[] emptyByteArray = new byte[]{};
   <#else>
   @Override
   public void allocateNew(int valueCount) {
-    <#if minor.class == "Int">
-        /* DELEGATE TO NEW VECTOR */
-        values.allocateNew(valueCount);
-    <#else>
-    try {
-      values.allocateNew(valueCount);
-      bits.allocateNew(valueCount);
-    } catch(OutOfMemoryException e) {
-      clear();
-      throw e;
-    }
-    bits.zeroVector();
-    mutator.reset();
-    accessor.reset();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.allocateNew(valueCount);
   }
 
   /**
@@ -433,13 +256,8 @@ protected final static byte[] emptyByteArray = new byte[]{};
    */
   @Override
   public void zeroVector() {
-    <#if minor.class == "Int">
-        /* DELEGATE TO NEW VECTOR */
-        values.zeroVector();
-    <#else>
-        bits.zeroVector();
-        values.zeroVector();
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    values.zeroVector();
   }
   </#if>
 
@@ -447,53 +265,36 @@ protected final static byte[] emptyByteArray = new byte[]{};
 
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getTransferPair(ref, allocator, callBack);
-    <#else>
-        return getTransferPair(ref, allocator);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getTransferPair(ref, allocator, callBack);
   }
 
 
 
   @Override
   public TransferPair getTransferPair(BufferAllocator allocator){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getTransferPair(allocator);
-    <#else>
-        return new TransferImpl(name, allocator);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getTransferPair(allocator);
   }
 
 
 
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getTransferPair(ref, allocator);
-    <#else>
-        return new TransferImpl(ref, allocator);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getTransferPair(ref, allocator);
   }
 
 
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.makeTransferPair(to);
-    <#else>
-        return new TransferImpl((${className}) to);
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.makeTransferPair(to);
   }
 
 
 
-  <#if minor.class == "Int" || minor.class == "VarChar">
   public void transferTo(${valuesName} target) {
     /* DELEGATE TO NEW VECTOR */
     <#if type.major == "VarLen">
@@ -512,61 +313,6 @@ protected final static byte[] emptyByteArray = new byte[]{};
     </#if>
   }
 
-  <#else>
-  public void transferTo(${className} target){
-    bits.transferTo(target.bits);
-    values.transferTo(target.values);
-    <#if type.major == "VarLen">
-    target.mutator.lastSet = mutator.lastSet;
-    </#if>
-    clear();
-  }
-
-  public void splitAndTransferTo(int startIndex, int length, ${className} target) {
-    bits.splitAndTransferTo(startIndex, length, target.bits);
-    values.splitAndTransferTo(startIndex, length, target.values);
-    <#if type.major == "VarLen">
-    target.mutator.lastSet = length - 1;
-    </#if>
-  }
-  </#if>
-
-
-
-  <#if minor.class != "Int" && minor.class != "VarChar">
-  private class TransferImpl implements TransferPair {
-    ${className} to;
-
-    public TransferImpl(String ref, BufferAllocator allocator){
-      to = new ${className}(ref, field.getFieldType(), allocator);
-    }
-
-    public TransferImpl(${className} to){
-      this.to = to;
-    }
-
-    @Override
-    public ${className} getTo(){
-      return to;
-    }
-
-    @Override
-    public void transfer(){
-      transferTo(to);
-    }
-
-    @Override
-    public void splitAndTransfer(int startIndex, int length) {
-      splitAndTransferTo(startIndex, length, to);
-    }
-
-    @Override
-    public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, ${className}.this);
-    }
-  }
-  </#if>
-
 
 
   @Override
@@ -580,8 +326,6 @@ protected final static byte[] emptyByteArray = new byte[]{};
   }
 
 
-
-  <#if minor.class == "Int" || minor.class == "VarChar">
   public void copyFrom(int fromIndex, int thisIndex, ${valuesName} from) {
     /* DELEGATE TO NEW VECTOR */
     values.copyFrom(fromIndex, thisIndex, from);
@@ -591,107 +335,43 @@ protected final static byte[] emptyByteArray = new byte[]{};
     /* DELEGATE TO NEW VECTOR */
     values.copyFromSafe(fromIndex, thisIndex, from);
   }
-  <#else>
-  public void copyFrom(int fromIndex, int thisIndex, ${className} from) {
-    final Accessor fromAccessor = from.getAccessor();
-    if (!fromAccessor.isNull(fromIndex)) {
-      mutator.set(thisIndex, fromAccessor.get(fromIndex));
-    }
-    <#if type.major == "VarLen">mutator.lastSet = thisIndex;</#if>
-  }
-
-  public void copyFromSafe(int fromIndex, int thisIndex, ${valuesName} from){
-    <#if type.major == "VarLen">
-    mutator.fillEmpties(thisIndex);
-    </#if>
-    values.copyFromSafe(fromIndex, thisIndex, from);
-    bits.getMutator().setSafeToOne(thisIndex);
-    <#if type.major == "VarLen">mutator.lastSet = thisIndex;</#if>
-  }
-
-  public void copyFromSafe(int fromIndex, int thisIndex, ${className} from){
-    <#if type.major == "VarLen">
-    mutator.fillEmpties(thisIndex);
-    </#if>
-    bits.copyFromSafe(fromIndex, thisIndex, from.bits);
-    values.copyFromSafe(fromIndex, thisIndex, from.values);
-    <#if type.major == "VarLen">mutator.lastSet = thisIndex;</#if>
-  }
-  </#if>
 
   @Override
   public long getValidityBufferAddress() {
-    /* address of the databuffer associated with the bitVector */
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getValidityBufferAddress();
-    <#else>
-        return (bits.getDataBuffer().memoryAddress());
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getValidityBufferAddress();
   }
 
   @Override
   public long getDataBufferAddress() {
-    /* address of the dataBuffer associated with the valueVector */
-    <#if minor.class == "Int" || minor.class == "VarChar">
-          /* DELEGATE TO NEW VECTOR */
-          return values.getDataBufferAddress();
-    <#else>
-          return (bits.getDataBuffer().memoryAddress());
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getDataBufferAddress();
   }
 
   @Override
   public long getOffsetBufferAddress() {
-    /* address of the dataBuffer associated with the offsetVector
-     * this operation is not supported for fixed-width vector types.
-     */
-    <#if minor.class == "Int" || minor.class == "VarChar">
-          /* DELEGATE TO NEW VECTOR */
-          return values.getOffsetBufferAddress();
-    <#else>
-        <#if type.major != "VarLen">
-          throw new UnsupportedOperationException();
-        <#else>
-          return (values.getOffsetAddr());
-        </#if>
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getOffsetBufferAddress();
   }
 
   @Override
   public ArrowBuf getValidityBuffer() {
-    <#if minor.class == "Int" || minor.class == "VarChar">
-          /* DELEGATE TO NEW VECTOR */
-          return values.getValidityBuffer();
-    <#else>
-          return (bits.getDataBuffer());
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getValidityBuffer();
   }
 
   @Override
   public ArrowBuf getDataBuffer() {
-    /* dataBuffer associated with the valueVector */
     return (values.getDataBuffer());
   }
 
   @Override
   public ArrowBuf getOffsetBuffer() {
-    /* dataBuffer associated with the offsetVector of the valueVector */
-    <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getOffsetBuffer();
-    <#else>
-        <#if type.major != "VarLen">
-          throw new UnsupportedOperationException();
-        <#else>
-          return (values.getOffsetBuffer());
-        </#if>
-    </#if>
+    /* DELEGATE TO NEW VECTOR */
+    return values.getOffsetBuffer();
   }
 
   public final class Accessor extends BaseDataValueVector.BaseAccessor <#if type.major = "VarLen">implements VariableWidthVector.VariableWidthAccessor</#if> {
-    final BitVector.Accessor bAccessor = bits.getAccessor();
-    final ${valuesName}.Accessor vAccessor = values.getAccessor();
 
     /**
      * Get the element at the specified position.
@@ -699,119 +379,67 @@ protected final static byte[] emptyByteArray = new byte[]{};
      * @param  index   position of the value
      * @return value of the element, if not null
      */
-    <#if minor.class == "Int" || minor.class == "VarChar">
-      public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
-        /* DELEGATE TO NEW VECTOR */
-        return values.get(index);
-      }
-    <#else>
-
-      public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
-        if (isNull(index)) {
-          throw new IllegalStateException("Can't get a null value");
-        }
-        return vAccessor.get(index);
-      }
-    </#if>
+    public <#if type.major == "VarLen">byte[]<#else>${minor.javaType!type.javaType}</#if> get(int index) {
+      /* DELEGATE TO NEW VECTOR */
+      return values.get(index);
+    }
 
     @Override
     public boolean isNull(int index) {
-      <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.isNull(index);
-      <#else>
-        return isSet(index) == 0;
-      </#if>
+      /* DELEGATE TO NEW VECTOR */
+      return values.isNull(index);
     }
 
     public int isSet(int index){
-      <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.isSet(index);
-      <#else>
-        return bAccessor.get(index);
-      </#if>
+      /* DELEGATE TO NEW VECTOR */
+      return values.isSet(index);
     }
 
     <#if type.major == "VarLen">
     public long getStartEnd(int index){
-      <#if minor.class == "VarChar">
         /* DELEGATE TO NEW VECTOR */
         return values.getStartEnd(index);
-      <#else>
-        return vAccessor.getStartEnd(index);
-      </#if>
     }
 
     @Override
     public int getValueLength(int index) {
-      <#if minor.class == "VarChar">
         /* DELEGATE TO NEW VECTOR */
         return values.getValueLength(index);
-      <#else>
-        return values.getAccessor().getValueLength(index);
-      </#if>
     }
     </#if>
 
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void get(int index, Nullable${minor.class}Holder holder){
         /* DELEGATE TO NEW VECTOR */
         values.get(index, holder);
     }
-    <#else>
-    public void get(int index, Nullable${minor.class}Holder holder){
-      vAccessor.get(index, holder);
-      holder.isSet = bAccessor.get(index);
-    }
-    </#if>
 
-    <#if minor.class == "Int" || minor.class == "VarChar">
     @Override
     public ${friendlyType} getObject(int index) {
       /* DELEGATE TO NEW VECTOR */
       return values.getObject(index);
     }
-    <#else>
-    @Override
-    public ${friendlyType} getObject(int index) {
-      if (isNull(index)) {
-          return null;
-      }else{
-        return vAccessor.getObject(index);
-      }
-    }
-    </#if>
 
     <#if minor.class == "IntervalYear" || minor.class == "IntervalDay">
     public StringBuilder getAsStringBuilder(int index) {
-      if (isNull(index)) {
-          return null;
-      }else{
-        return vAccessor.getAsStringBuilder(index);
-      }
+       /* DELEGATE TO NEW VECTOR */
+       return values.getAsStringBuilder(index);
     }
     </#if>
 
     @Override
     public int getValueCount(){
-      <#if minor.class == "Int" || minor.class == "VarChar">
-        /* DELEGATE TO NEW VECTOR */
-        return values.getValueCount();
-      <#else>
-        return bits.getAccessor().getValueCount();
-      </#if>
+      /* DELEGATE TO NEW VECTOR */
+      return values.getValueCount();
     }
 
-    public void reset(){}
+    public void reset() { }
   }
 
   public final class Mutator extends BaseDataValueVector.BaseMutator implements NullableVectorDefinitionSetter<#if type.major = "VarLen">, VariableWidthVector.VariableWidthMutator</#if> {
     private int setCount;
     <#if type.major = "VarLen"> private int lastSet = -1;</#if>
 
-    private Mutator(){
-    }
+    private Mutator() { }
 
     public ${valuesName} getVectorWithValues() {
       return values;
@@ -819,12 +447,9 @@ protected final static byte[] emptyByteArray = new byte[]{};
 
 
     @Override
-    public void setIndexDefined(int index){
-      <#if minor.class == "Int" || minor.class == "VarChar">
+    public void setIndexDefined(int index) {
+      /* DELEGATE TO NEW VECTOR */
       values.setIndexDefined(index);
-      <#else>
-      bits.getMutator().setToOne(index);
-      </#if>
     }
 
 
@@ -835,32 +460,14 @@ protected final static byte[] emptyByteArray = new byte[]{};
      * @param index   position of the bit to set
      * @param value   array of bytes (or int if smaller than 4 bytes) to write
      */
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
        /* DELEGATE TO NEW VECTOR */
        values.set(index, value);
     }
-    <#else>
-    public void set(int index, <#if type.major == "VarLen">byte[]<#elseif (type.width < 4)>int<#else>${minor.javaType!type.javaType}</#if> value) {
-      setCount++;
-      final ${valuesName}.Mutator valuesMutator = values.getMutator();
-      final BitVector.Mutator bitsMutator = bits.getMutator();
-      <#if type.major == "VarLen">
-      for (int i = lastSet + 1; i < index; i++) {
-        valuesMutator.set(i, emptyByteArray);
-      }
-      </#if>
-      bitsMutator.setToOne(index);
-      valuesMutator.set(index, value);
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-    </#if>
 
 
 
     <#if type.major == "VarLen">
-    <#if minor.class == "VarChar">
     public void fillEmpties(int index) {
       /* DELEGATE TO NEW VECTOR */
       values.fillEmpties(index);
@@ -871,203 +478,55 @@ protected final static byte[] emptyByteArray = new byte[]{};
       /* DELEGATE TO NEW VECTOR */
       values.setValueLengthSafe(index, length);
     }
-
-    <#else>
-    public void fillEmpties(int index){
-      final ${valuesName}.Mutator valuesMutator = values.getMutator();
-      for (int i = lastSet + 1; i < index; i++) {
-        valuesMutator.setSafe(i, emptyByteArray);
-      }
-      while(index > bits.getValueCapacity()) {
-        bits.reAlloc();
-      }
-      lastSet = index - 1;
-    }
-
-    @Override
-    public void setValueLengthSafe(int index, int length) {
-      values.getMutator().setValueLengthSafe(index, length);
-      lastSet = index;
-    }
-    </#if>
     </#if>
 
 
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void setSafe(int index, byte[] value, int start, int length) {
        /* DELEGATE TO NEW VECTOR */
       values.setSafe(index, value, start, length);
     }
-    <#else>
-    public void setSafe(int index, byte[] value, int start, int length) {
-      <#if type.major != "VarLen">
-      throw new UnsupportedOperationException();
-      <#else>
-      fillEmpties(index);
-
-      bits.getMutator().setSafeToOne(index);
-      values.getMutator().setSafe(index, value, start, length);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-      </#if>
-    }
-    </#if>
 
 
-
-    <#if minor.class == "VarChar">
     public void setSafe(int index, ByteBuffer value, int start, int length) {
        /* DELEGATE TO NEW VECTOR */
        values.setSafe(index, value, start, length);
     }
-    <#else>
-    public void setSafe(int index, ByteBuffer value, int start, int length) {
-      <#if type.major != "VarLen">
-      throw new UnsupportedOperationException();
-      <#else>
-      fillEmpties(index);
-
-      bits.getMutator().setSafeToOne(index);
-      values.getMutator().setSafe(index, value, start, length);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-      </#if>
-    }
-    </#if>
-
 
 
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void setNull(int index) {
        /* DELEGATE TO NEW VECTOR */
        values.setNull(index);
     }
-    <#else>
-    public void setNull(int index){
-      bits.getMutator().setSafe(index, 0);
-    }
-    </#if>
-
-
-
-    <#if minor.class != "Int" && minor.class != "VarChar">
-    /* these methods are probably not needed */
-    public void setSkipNull(int index, ${minor.class}Holder holder){
-      values.getMutator().set(index, holder);
-    }
-
-    public void setSkipNull(int index, Nullable${minor.class}Holder holder){
-      values.getMutator().set(index, holder);
-    }
-    </#if>
 
 
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void set(int index, Nullable${minor.class}Holder holder) {
       /* DELEGATE TO NEW VECTOR */
       values.set(index, holder);
     }
-    <#else>
-    public void set(int index, Nullable${minor.class}Holder holder) {
-      final ${valuesName}.Mutator valuesMutator = values.getMutator();
-      <#if type.major == "VarLen">
-      for (int i = lastSet + 1; i < index; i++) {
-        valuesMutator.set(i, emptyByteArray);
-      }
-      </#if>
-      bits.getMutator().set(index, holder.isSet);
-      valuesMutator.set(index, holder);
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-    </#if>
 
 
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void set(int index, ${minor.class}Holder holder) {
         /* DELEGATE TO NEW VECTOR */
         values.set(index, holder);
     }
-    <#else>
-    public void set(int index, ${minor.class}Holder holder) {
-      final ${valuesName}.Mutator valuesMutator = values.getMutator();
-      <#if type.major == "VarLen">
-      for (int i = lastSet + 1; i < index; i++) {
-        valuesMutator.set(i, emptyByteArray);
-      }
-      </#if>
-      bits.getMutator().setToOne(index);
-      valuesMutator.set(index, holder);
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-    </#if>
-
 
 
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public boolean isSafe(int outIndex) {
        /* DELEGATE TO NEW VECTOR */
        return values.isSafe(outIndex);
     }
-    <#else>
-    public boolean isSafe(int outIndex) {
-      return outIndex < ${className}.this.getValueCapacity();
-    }
-    </#if>
-
 
 
-    <#if minor.class == "Int" || minor.class == "VarChar">
-    <#if minor.class == "Int">
-    public void set(int index, int isSet, int valueField) {
-      /* DELEGATE TO NEW VECTOR */
-      values.set(index, isSet, valueField);
-    }
-    public void setSafe(int index, int isSet, int valueField) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, isSet, valueField);
-    }
-    </#if>
-    <#if minor.class == "VarChar">
-    public void set(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
-      /* DELEGATE TO NEW VECTOR */
-      values.set(index, isSet, startField, endField, bufferField);
-    }
-    public void setSafe(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
-        /* DELEGATE TO NEW VECTOR */
-        values.setSafe(index, isSet, startField, endField, bufferField);
-    }
-    </#if>
-    <#else>
     <#assign fields = minor.fields!type.fields />
     public void set(int index, int isSet<#list fields as field>, ${field.type} ${field.name}Field</#list> ){
-      final ${valuesName}.Mutator valuesMutator = values.getMutator();
-      <#if type.major == "VarLen">
-      for (int i = lastSet + 1; i < index; i++) {
-        valuesMutator.set(i, emptyByteArray);
-      }
-      </#if>
-      bits.getMutator().set(index, isSet);
-      valuesMutator.set(index<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
-      <#if type.major == "VarLen">lastSet = index;</#if>
+      values.set(index, isSet<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
     }
 
     public void setSafe(int index, int isSet<#list fields as field><#if field.include!true >, ${field.type} ${field.name}Field</#if></#list> ) {
-      <#if type.major == "VarLen">
-      fillEmpties(index);
-      </#if>
-      bits.getMutator().setSafe(index, isSet);
-      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>
+      values.setSafe(index, isSet<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
     }
-    </#if>
 
 
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     public void setSafe(int index, Nullable${minor.class}Holder value) {
       /* DELEGATE TO NEW VECTOR */
       values.setSafe(index, value);
@@ -1077,40 +536,12 @@ protected final static byte[] emptyByteArray = new byte[]{};
       /* DELEGATE TO NEW VECTOR */
       values.setSafe(index, value);
     }
-    <#else>
-    public void setSafe(int index, Nullable${minor.class}Holder value) {
-      <#if type.major == "VarLen">
-      fillEmpties(index);
-      </#if>
-      bits.getMutator().setSafe(index, value.isSet);
-      values.getMutator().setSafe(index, value);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-
-    public void setSafe(int index, ${minor.class}Holder value) {
-      <#if type.major == "VarLen">
-      fillEmpties(index);
-      </#if>
-      bits.getMutator().setSafeToOne(index);
-      values.getMutator().setSafe(index, value);
-      setCount++;
-      <#if type.major == "VarLen">lastSet = index;</#if>
-    }
-    </#if>
-
 
 
     <#if !(type.major == "VarLen" || minor.class == "IntervalDay")>
     public void setSafe(int index, ${minor.javaType!type.javaType} value) {
-      <#if minor.class == "Int">
-        /* DELEGATE TO NEW VECTOR */
-        values.setSafe(index, value);
-      <#else>
-      bits.getMutator().setSafeToOne(index);
-      values.getMutator().setSafe(index, value);
-      setCount++;
-      </#if>
+      /* DELEGATE TO NEW VECTOR */
+      values.setSafe(index, value);
     }
     </#if>
 
@@ -1118,93 +549,45 @@ protected final static byte[] emptyByteArray = new byte[]{};
 
     <#if minor.class == "Decimal">
     public void set(int index, ${friendlyType} value) {
-      bits.getMutator().setToOne(index);
-      values.getMutator().set(index, value);
+      /* DELEGATE TO NEW VECTOR */
+      values.set(index, value);
     }
 
     public void setSafe(int index, ${friendlyType} value) {
-      bits.getMutator().setSafeToOne(index);
-      values.getMutator().setSafe(index, value);
-      setCount++;
+      /* DELEGATE TO NEW VECTOR */
+      values.setSafe(index, value);
     }
     </#if>
 
 
-
-    <#if minor.class == "Int" || minor.class == "VarChar">
     @Override
     public void setValueCount(int valueCount) {
       /* DELEGATE TO NEW VECTOR */
       values.setValueCount(valueCount);
     }
-    <#else>
-    @Override
-    public void setValueCount(int valueCount) {
-      assert valueCount >= 0;
-      <#if type.major == "VarLen">
-      fillEmpties(valueCount);
-      </#if>
-      values.getMutator().setValueCount(valueCount);
-      bits.getMutator().setValueCount(valueCount);
-    }
-    </#if>
-
 
 
-    <#if minor.class != "Int" && minor.class != "VarChar">
     /* THIS METHOD IS PROBABLY NOT NEEDED FOR NEW VECTORS */
     @Override
-    public void generateTestData(int valueCount){
-      bits.getMutator().generateTestDataAlt(valueCount);
-      values.getMutator().generateTestData(valueCount);
-      <#if type.major = "VarLen">lastSet = valueCount;</#if>
-      setValueCount(valueCount);
-    }
-    </#if>
+    public void generateTestData(int valueCount) { }
 
 
-
-    <#if minor.class != "Int" && minor.class != "VarChar">
     /* MUTATOR RESET IS NOT NEEDED FOR NEW VECTORS */
     @Override
-    public void reset(){
-      setCount = 0;
-      <#if type.major = "VarLen">lastSet = -1;</#if>
-    }
-    </#if>
-
+    public void reset() { }
 
 
-    <#if minor.class == "VarChar">
+    <#if type.major == "VarLen">
     public void setLastSet(int value) {
       /* DELEGATE TO NEW VECTOR */
       values.setLastSet(value);
     }
-    <#else>
-    public void setLastSet(int value) {
-      <#if type.major = "VarLen">
-        lastSet = value;
-      <#else>
-        throw new UnsupportedOperationException();
-      </#if>
-    }
-    </#if>
 
 
-
-    <#if minor.class == "VarChar">
     public int getLastSet() {
       /* DELEGATE TO NEW VECTOR */
       return values.getLastSet();
     }
-    <#else>
-    public int getLastSet() {
-      <#if type.major != "VarLen">
-        throw new UnsupportedOperationException();
-      <#else>
-        return lastSet;
-      </#if>
-    }
     </#if>
   }
 }
diff --git a/java/vector/src/main/codegen/templates/UnionListWriter.java b/java/vector/src/main/codegen/templates/UnionListWriter.java
index 9fe41d0..8ac23fe 100644
--- a/java/vector/src/main/codegen/templates/UnionListWriter.java
+++ b/java/vector/src/main/codegen/templates/UnionListWriter.java
@@ -36,11 +36,11 @@ package org.apache.arrow.vector.complex.impl;
 public class UnionListWriter extends AbstractFieldWriter {
 
   private ListVector vector;
-  private UInt4Vector offsets;
   private PromotableWriter writer;
   private boolean inMap = false;
   private String mapName;
   private int lastIndex = 0;
+  private static final int OFFSET_WIDTH = 4;
 
   public UnionListWriter(ListVector vector) {
     this(vector, NullableMapWriterFactory.getNullableMapWriterFactoryInstance());
@@ -49,7 +49,6 @@ public class UnionListWriter extends AbstractFieldWriter {
   public UnionListWriter(ListVector vector, NullableMapWriterFactory nullableMapWriterFactory) {
     this.vector = vector;
     this.writer = new PromotableWriter(vector.getDataVector(), vector, nullableMapWriterFactory);
-    this.offsets = vector.getOffsetVector();
   }
 
   public UnionListWriter(ListVector vector, AbstractFieldWriter parent) {
@@ -72,7 +71,7 @@ public class UnionListWriter extends AbstractFieldWriter {
   }
 
   public void setValueCount(int count) {
-    vector.getMutator().setValueCount(count);
+    vector.setValueCount(count);
   }
 
   @Override
@@ -133,13 +132,13 @@ public class UnionListWriter extends AbstractFieldWriter {
 
   @Override
   public void startList() {
-    vector.getMutator().startNewValue(idx());
-    writer.setPosition(offsets.getAccessor().get(idx() + 1));
+    vector.startNewValue(idx());
+    writer.setPosition(vector.getOffsetBuffer().getInt((idx() + 1) * OFFSET_WIDTH));
   }
 
   @Override
   public void endList() {
-    offsets.getMutator().set(idx() + 1, writer.idx());
+    vector.getOffsetBuffer().setInt((idx() + 1) * OFFSET_WIDTH, writer.idx());
     setPosition(idx() + 1);
   }
 
diff --git a/java/vector/src/main/codegen/templates/UnionVector.java b/java/vector/src/main/codegen/templates/UnionVector.java
index 3c7ed01..3da383f 100644
--- a/java/vector/src/main/codegen/templates/UnionVector.java
+++ b/java/vector/src/main/codegen/templates/UnionVector.java
@@ -25,6 +25,7 @@ package org.apache.arrow.vector.complex;
 
 <#include "/@includes/vv_imports.ftl" />
 import com.google.common.collect.ImmutableList;
+import io.netty.buffer.ArrowBuf;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
@@ -32,6 +33,9 @@ import org.apache.arrow.vector.BaseDataValueVector;
 import org.apache.arrow.vector.complex.impl.ComplexCopier;
 import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.schema.ArrowFieldNode;
+import org.apache.arrow.memory.BaseAllocator;
+import org.apache.arrow.vector.BaseValueVector;
+import org.apache.arrow.vector.util.OversizedAllocationException;
 
 import static org.apache.arrow.vector.types.UnionMode.Sparse;
 
@@ -56,12 +60,10 @@ public class UnionVector implements FieldVector {
 
   private String name;
   private BufferAllocator allocator;
-  private Accessor accessor = new Accessor();
-  private Mutator mutator = new Mutator();
   int valueCount;
 
   MapVector internalMap;
-  UInt1Vector typeVector;
+  ArrowBuf typeBuffer;
 
   private NullableMapVector mapVector;
   private ListVector listVector;
@@ -71,16 +73,17 @@ public class UnionVector implements FieldVector {
   private int singleType = 0;
   private ValueVector singleVector;
 
+  private static final byte TYPE_WIDTH = 1;
   private final CallBack callBack;
-  private final List<BufferBacked> innerVectors;
+  private int typeBufferAllocationSizeInBytes;
 
   public UnionVector(String name, BufferAllocator allocator, CallBack callBack) {
     this.name = name;
     this.allocator = allocator;
     this.internalMap = new MapVector("internal", allocator, new FieldType(false, ArrowType.Struct.INSTANCE, null, null), callBack);
-    this.typeVector = new UInt1Vector("types", allocator);
+    this.typeBuffer = allocator.getEmpty();
     this.callBack = callBack;
-    this.innerVectors = Collections.unmodifiableList(Arrays.<BufferBacked>asList(typeVector));
+    this.typeBufferAllocationSizeInBytes = BaseValueVector.INITIAL_VALUE_ALLOCATION * TYPE_WIDTH;
   }
 
   public BufferAllocator getAllocator() {
@@ -104,20 +107,33 @@ public class UnionVector implements FieldVector {
 
   @Override
   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    // truncate types vector buffer to size (#0)
-    org.apache.arrow.vector.BaseDataValueVector.truncateBufferBasedOnSize(ownBuffers, 0, typeVector.getBufferSizeFor(fieldNode.getLength()));
-    BaseDataValueVector.load(fieldNode, getFieldInnerVectors(), ownBuffers);
+    if (ownBuffers.size() != 1) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 1 + ", got: " + ownBuffers.size());
+    }
+
+    ArrowBuf buffer = ownBuffers.get(0);
+    typeBuffer.release();
+    typeBuffer = buffer.retain(allocator);
+    typeBufferAllocationSizeInBytes = typeBuffer.capacity();
     this.valueCount = fieldNode.getLength();
   }
 
   @Override
   public List<ArrowBuf> getFieldBuffers() {
-    return BaseDataValueVector.unload(getFieldInnerVectors());
+    List<ArrowBuf> result = new ArrayList<>(1);
+
+    typeBuffer.readerIndex(0);
+    typeBuffer.writerIndex(valueCount * TYPE_WIDTH);
+
+    result.add(typeBuffer);
+
+    return result;
   }
 
   @Override
+  @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
-     return this.innerVectors;
+     throw new UnsupportedOperationException("There are no inner vectors. Use geFieldBuffers");
   }
 
   private String fieldName(MinorType type) {
@@ -134,7 +150,7 @@ public class UnionVector implements FieldVector {
 
   @Override
   public long getValidityBufferAddress() {
-    return typeVector.getDataBuffer().memoryAddress();
+    return typeBuffer.memoryAddress();
   }
 
   @Override
@@ -148,7 +164,7 @@ public class UnionVector implements FieldVector {
   }
 
   @Override
-  public ArrowBuf getValidityBuffer() { return typeVector.getDataBuffer(); }
+  public ArrowBuf getValidityBuffer() { return typeBuffer; }
 
   @Override
   public ArrowBuf getDataBuffer() { throw new UnsupportedOperationException(); }
@@ -211,47 +227,76 @@ public class UnionVector implements FieldVector {
   }
 
   public int getTypeValue(int index) {
-    return typeVector.getAccessor().get(index);
-  }
-
-  public UInt1Vector getTypeVector() {
-    return typeVector;
+    return typeBuffer.getByte(index * TYPE_WIDTH);
   }
 
   @Override
   public void allocateNew() throws OutOfMemoryException {
     internalMap.allocateNew();
-    typeVector.allocateNew();
-    if (typeVector != null) {
-      typeVector.zeroVector();
+    try {
+      allocateTypeBuffer();
+    } catch (Exception e) {
+      clear();
+      throw e;
     }
   }
 
   @Override
   public boolean allocateNewSafe() {
     boolean safe = internalMap.allocateNewSafe();
-    safe = safe && typeVector.allocateNewSafe();
-    if (safe) {
-      if (typeVector != null) {
-        typeVector.zeroVector();
-      }
+    if (!safe) { return false; }
+    try {
+      allocateTypeBuffer();
+    } catch (Exception e) {
+      clear();
+      return  false;
     }
-    return safe;
+
+    return true;
+  }
+
+  private void allocateTypeBuffer() {
+    typeBuffer = allocator.buffer(typeBufferAllocationSizeInBytes);
+    typeBuffer.readerIndex(0);
+    typeBuffer.setZero(0, typeBuffer.capacity());
   }
 
   @Override
   public void reAlloc() {
     internalMap.reAlloc();
-    typeVector.reAlloc();
+    reallocTypeBuffer();
   }
 
-  @Override
-  public void setInitialCapacity(int numRecords) {
+  private void reallocTypeBuffer() {
+    final int currentBufferCapacity = typeBuffer.capacity();
+    long baseSize  = typeBufferAllocationSizeInBytes;
+
+    if (baseSize < (long)currentBufferCapacity) {
+      baseSize = (long)currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > BaseValueVector.MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    newBuf.setBytes(0, typeBuffer, 0, currentBufferCapacity);
+    final int halfNewCapacity = newBuf.capacity() / 2;
+    newBuf.setZero(halfNewCapacity, halfNewCapacity);
+    typeBuffer.release(1);
+    typeBuffer = newBuf;
+    typeBufferAllocationSizeInBytes = (int)newAllocationSize;
   }
 
   @Override
+  public void setInitialCapacity(int numRecords) { }
+
+  @Override
   public int getValueCapacity() {
-    return Math.min(typeVector.getValueCapacity(), internalMap.getValueCapacity());
+    return Math.min(getTypeBufferValueCapacity(), internalMap.getValueCapacity());
   }
 
   @Override
@@ -261,7 +306,9 @@ public class UnionVector implements FieldVector {
 
   @Override
   public void clear() {
-    typeVector.clear();
+    valueCount = 0;
+    typeBuffer.release();
+    typeBuffer = allocator.getEmpty();
     internalMap.clear();
   }
 
@@ -300,7 +347,7 @@ public class UnionVector implements FieldVector {
   public void copyFrom(int inIndex, int outIndex, UnionVector from) {
     from.getReader().setPosition(inIndex);
     getWriter().setPosition(outIndex);
-    ComplexCopier.copy(from.reader, mutator.writer);
+    ComplexCopier.copy(from.reader, writer);
   }
 
   public void copyFromSafe(int inIndex, int outIndex, UnionVector from) {
@@ -321,33 +368,34 @@ public class UnionVector implements FieldVector {
 
   private class TransferImpl implements TransferPair {
     private final TransferPair internalMapVectorTransferPair;
-    private final TransferPair typeVectorTransferPair;
     private final UnionVector to;
 
     public TransferImpl(String name, BufferAllocator allocator, CallBack callBack) {
       to = new UnionVector(name, allocator, callBack);
       internalMapVectorTransferPair = internalMap.makeTransferPair(to.internalMap);
-      typeVectorTransferPair = typeVector.makeTransferPair(to.typeVector);
     }
 
     public TransferImpl(UnionVector to) {
       this.to = to;
       internalMapVectorTransferPair = internalMap.makeTransferPair(to.internalMap);
-      typeVectorTransferPair = typeVector.makeTransferPair(to.typeVector);
     }
 
     @Override
     public void transfer() {
+      to.clear();
+      to.typeBuffer = typeBuffer.transferOwnership(to.allocator).buffer;
       internalMapVectorTransferPair.transfer();
-      typeVectorTransferPair.transfer();
       to.valueCount = valueCount;
     }
 
     @Override
     public void splitAndTransfer(int startIndex, int length) {
+      to.clear();
       internalMapVectorTransferPair.splitAndTransfer(startIndex, length);
-      typeVectorTransferPair.splitAndTransfer(startIndex, length);
-      to.getMutator().setValueCount(length);
+      final int startPoint = startIndex * TYPE_WIDTH;
+      final int sliceLength = length * TYPE_WIDTH;
+      to.typeBuffer = typeBuffer.slice(startPoint, sliceLength).transferOwnership(to.allocator).buffer;
+      to.setValueCount(length);
     }
 
     @Override
@@ -362,13 +410,15 @@ public class UnionVector implements FieldVector {
   }
 
   @Override
+  @Deprecated
   public Accessor getAccessor() {
-    return accessor;
+    throw new UnsupportedOperationException("Accessor is not supported for reading from UNION");
   }
 
   @Override
+  @Deprecated
   public Mutator getMutator() {
-    return mutator;
+    throw new UnsupportedOperationException("Mutator is not supported for writing to UNION");
   }
 
   @Override
@@ -380,15 +430,17 @@ public class UnionVector implements FieldVector {
   }
 
   public FieldWriter getWriter() {
-    if (mutator.writer == null) {
-      mutator.writer = new UnionWriter(this);
+    if (writer == null) {
+      writer = new UnionWriter(this);
     }
-    return mutator.writer;
+    return writer;
   }
 
   @Override
   public int getBufferSize() {
-    return typeVector.getBufferSize() + internalMap.getBufferSize();
+    if (valueCount == 0) { return 0; }
+
+    return (valueCount * TYPE_WIDTH) + internalMap.getBufferSize();
   }
 
   @Override
@@ -402,13 +454,13 @@ public class UnionVector implements FieldVector {
       bufferSize += v.getBufferSizeFor(valueCount);
     }
 
-    return (int) bufferSize;
+    return (int) bufferSize + (valueCount * TYPE_WIDTH);
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
     ImmutableList.Builder<ArrowBuf> builder = ImmutableList.builder();
-    builder.add(typeVector.getBuffers(clear));
+    builder.add(typeBuffer);
     builder.add(internalMap.getBuffers(clear));
     List<ArrowBuf> list = builder.build();
     return list.toArray(new ArrowBuf[list.size()]);
@@ -417,15 +469,12 @@ public class UnionVector implements FieldVector {
   @Override
   public Iterator<ValueVector> iterator() {
     List<ValueVector> vectors = Lists.newArrayList(internalMap.iterator());
-    vectors.add(typeVector);
     return vectors.iterator();
   }
 
-  public class Accessor extends BaseValueVector.BaseAccessor {
 
-    @Override
     public Object getObject(int index) {
-      int type = typeVector.getAccessor().get(index);
+      int type = typeBuffer.getByte(index * TYPE_WIDTH);
       switch (MinorType.values()[type]) {
       case NULL:
         return null;
@@ -436,18 +485,14 @@ public class UnionVector implements FieldVector {
           <#assign uncappedName = name?uncap_first/>
           <#if !minor.typeParams?? >
       case ${name?upper_case}:
-        <#if minor.class != "Int" && minor.class != "VarChar">
-          return get${name}Vector().getAccessor().getObject(index);
-        <#else>
           return get${name}Vector().getObject(index);
-        </#if>
           </#if>
         </#list>
       </#list>
       case MAP:
-        return getMap().getAccessor().getObject(index);
+        return getMap().getObject(index);
       case LIST:
-        return getList().getAccessor().getObject(index);
+        return getList().getObject(index);
       default:
         throw new UnsupportedOperationException("Cannot support type: " + MinorType.values()[type]);
       }
@@ -466,30 +511,37 @@ public class UnionVector implements FieldVector {
       holder.reader = reader;
     }
 
-    @Override
     public int getValueCount() {
       return valueCount;
     }
 
-    @Override
     public boolean isNull(int index) {
-      return typeVector.getAccessor().get(index) == 0;
+      return (typeBuffer.getByte(index * TYPE_WIDTH) == 0);
+    }
+
+    @Override
+    public int getNullCount() {
+      int nullCount = 0;
+      for (int i = 0; i < getValueCount(); i++) {
+        if (isNull(i)) {
+          nullCount++;
+        }
+      }
+      return nullCount;
     }
 
     public int isSet(int index) {
       return isNull(index) ? 0 : 1;
     }
-  }
-
-  public class Mutator extends BaseValueVector.BaseMutator {
 
     UnionWriter writer;
 
-    @Override
     public void setValueCount(int valueCount) {
-      UnionVector.this.valueCount = valueCount;
-      typeVector.getMutator().setValueCount(valueCount);
-      internalMap.getMutator().setValueCount(valueCount);
+      this.valueCount = valueCount;
+      while (valueCount > getTypeBufferValueCapacity()) {
+        reallocTypeBuffer();
+      }
+      internalMap.setValueCount(valueCount);
     }
 
     public void setSafe(int index, UnionHolder holder) {
@@ -534,11 +586,7 @@ public class UnionVector implements FieldVector {
         <#if !minor.typeParams?? >
     public void setSafe(int index, Nullable${name}Holder holder) {
       setType(index, MinorType.${name?upper_case});
-      <#if minor.class != "Int" && minor.class != "VarChar">
-        get${name}Vector().getMutator().setSafe(index, holder);
-      <#else>
-        get${name}Vector().setSafe(index, holder);
-      </#if>
+      get${name}Vector().setSafe(index, holder);
     }
 
         </#if>
@@ -546,19 +594,13 @@ public class UnionVector implements FieldVector {
     </#list>
 
     public void setType(int index, MinorType type) {
-      typeVector.getMutator().setSafe(index, (byte) type.ordinal());
+      while (index >= getTypeBufferValueCapacity()) {
+        reallocTypeBuffer();
+      }
+      typeBuffer.setByte(index * TYPE_WIDTH , (byte) type.ordinal());
     }
 
-    @Override
-    public void reset() { }
-
-    @Override
-    public void generateTestData(int values) { }
-  }
-
-  public int getValueCount() { return 0; }
-
-  public void setValueCount(int valueCount) { }
-
-  public Object getObject(int index) { return null; }
+    private int getTypeBufferValueCapacity() {
+      return (int) ((typeBuffer.capacity() * 1.0) / TYPE_WIDTH);
+    }
 }
diff --git a/java/vector/src/main/codegen/templates/UnionWriter.java b/java/vector/src/main/codegen/templates/UnionWriter.java
index f892bac..526708a 100644
--- a/java/vector/src/main/codegen/templates/UnionWriter.java
+++ b/java/vector/src/main/codegen/templates/UnionWriter.java
@@ -62,7 +62,7 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
 
   @Override
   public void start() {
-    data.getMutator().setType(idx(), MinorType.MAP);
+    data.setType(idx(), MinorType.MAP);
     getMapWriter().start();
   }
 
@@ -74,7 +74,7 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
   @Override
   public void startList() {
     getListWriter().startList();
-    data.getMutator().setType(idx(), MinorType.LIST);
+    data.setType(idx(), MinorType.LIST);
   }
 
   @Override
@@ -92,7 +92,7 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
   }
 
   public MapWriter asMap() {
-    data.getMutator().setType(idx(), MinorType.MAP);
+    data.setType(idx(), MinorType.MAP);
     return getMapWriter();
   }
 
@@ -106,7 +106,7 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
   }
 
   public ListWriter asList() {
-    data.getMutator().setType(idx(), MinorType.LIST);
+    data.setType(idx(), MinorType.LIST);
     return getListWriter();
   }
 
@@ -150,19 +150,19 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
   }
 
   public ${name}Writer as${name}() {
-    data.getMutator().setType(idx(), MinorType.${name?upper_case});
+    data.setType(idx(), MinorType.${name?upper_case});
     return get${name}Writer();
   }
 
   @Override
   public void write(${name}Holder holder) {
-    data.getMutator().setType(idx(), MinorType.${name?upper_case});
+    data.setType(idx(), MinorType.${name?upper_case});
     get${name}Writer().setPosition(idx());
     get${name}Writer().write${name}(<#list fields as field>holder.${field.name}<#if field_has_next>, </#if></#list>);
   }
 
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
-    data.getMutator().setType(idx(), MinorType.${name?upper_case});
+    data.setType(idx(), MinorType.${name?upper_case});
     get${name}Writer().setPosition(idx());
     get${name}Writer().write${name}(<#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
   }
@@ -175,28 +175,28 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
 
   @Override
   public MapWriter map() {
-    data.getMutator().setType(idx(), MinorType.LIST);
+    data.setType(idx(), MinorType.LIST);
     getListWriter().setPosition(idx());
     return getListWriter().map();
   }
 
   @Override
   public ListWriter list() {
-    data.getMutator().setType(idx(), MinorType.LIST);
+    data.setType(idx(), MinorType.LIST);
     getListWriter().setPosition(idx());
     return getListWriter().list();
   }
 
   @Override
   public ListWriter list(String name) {
-    data.getMutator().setType(idx(), MinorType.MAP);
+    data.setType(idx(), MinorType.MAP);
     getMapWriter().setPosition(idx());
     return getMapWriter().list(name);
   }
 
   @Override
   public MapWriter map(String name) {
-    data.getMutator().setType(idx(), MinorType.MAP);
+    data.setType(idx(), MinorType.MAP);
     getMapWriter().setPosition(idx());
     return getMapWriter().map(name);
   }
@@ -209,14 +209,14 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
   <#if !minor.typeParams?? >
   @Override
   public ${capName}Writer ${lowerName}(String name) {
-    data.getMutator().setType(idx(), MinorType.MAP);
+    data.setType(idx(), MinorType.MAP);
     getMapWriter().setPosition(idx());
     return getMapWriter().${lowerName}(name);
   }
 
   @Override
   public ${capName}Writer ${lowerName}() {
-    data.getMutator().setType(idx(), MinorType.LIST);
+    data.setType(idx(), MinorType.LIST);
     getListWriter().setPosition(idx());
     return getListWriter().${lowerName}();
   }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java
index c5f7810..83dc409 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java
@@ -35,12 +35,17 @@ import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
+/**
+ * BaseNullableFixedWidthVector provides an abstract interface for
+ * implementing vectors of fixed width values. The vectors are nullable
+ * implying that zero or more elements in the vector could be NULL.
+ */
 public abstract class BaseNullableFixedWidthVector extends BaseValueVector
         implements FixedWidthVector, FieldVector {
    private final byte typeWidth;
 
-   private int valueAllocationSizeInBytes;
-   private int validityAllocationSizeInBytes;
+   protected int valueAllocationSizeInBytes;
+   protected int validityAllocationSizeInBytes;
 
    protected final Field field;
    private int allocationMonitor;
@@ -52,23 +57,21 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
                                        FieldType fieldType, final byte typeWidth) {
       super(name, allocator);
       this.typeWidth = typeWidth;
-      valueAllocationSizeInBytes = INITIAL_VALUE_ALLOCATION * typeWidth;
-      validityAllocationSizeInBytes = getSizeFromCount(INITIAL_VALUE_ALLOCATION);
       field = new Field(name, fieldType, null);
       valueCount = 0;
       allocationMonitor = 0;
       validityBuffer = allocator.getEmpty();
       valueBuffer = allocator.getEmpty();
+      if (typeWidth > 0) {
+         valueAllocationSizeInBytes = INITIAL_VALUE_ALLOCATION * typeWidth;
+         validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
+      } else {
+         /* specialized handling for NullableBitVector */
+         valueAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
+         validityAllocationSizeInBytes = valueAllocationSizeInBytes;
+      }
    }
 
-   /* TODO:
-    * Determine how writerIndex and readerIndex need to be used. Right now we
-    * are setting the writerIndex and readerIndex in the call to getFieldBuffers
-    * using the valueCount -- this assumes that the caller of getFieldBuffers
-    * on the vector has already invoked setValueCount.
-    *
-    * Do we need to set them during vector transfer and splitAndTransfer?
-    */
 
    /* TODO:
     *
@@ -84,57 +87,91 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
     * abstracted out to top level base class.
     *
     * Right now BaseValueVector is the top level base class for other
-    * vector types in ValueVector hierarchy and those vectors have not
-    * yet been refactored so moving things to the top class as of now
-    * is not a good idea.
+    * vector types in ValueVector hierarchy (non-nullable) and those
+    * vectors have not yet been refactored/removed so moving things to
+    * the top class as of now is not a good idea.
     */
 
-   /* TODO:
-    * See if we need logger -- unnecessary object probably
-    */
-
-   protected abstract org.slf4j.Logger getLogger();
 
    @Override
+   @Deprecated
    public Mutator getMutator() {
-      throw new  UnsupportedOperationException("Mutator is not needed to write into vector");
+      throw new UnsupportedOperationException("Mutator is not supported for writing to vector");
    }
 
    @Override
+   @Deprecated
    public Accessor getAccessor() {
-      throw new UnsupportedOperationException("Accessor is not needed to read from vector");
+      throw new UnsupportedOperationException("Accessor is not supported for reading from vector");
    }
 
+   /**
+    * Get the memory address of buffer that manages the validity
+    * (NULL or NON-NULL nature) of elements in the vector.
+    * @return starting address of the buffer
+    */
    @Override
    public long getValidityBufferAddress() {
       return (validityBuffer.memoryAddress());
    }
 
+   /**
+    * Get the memory address of buffer that stores the data for elements
+    * in the vector.
+    * @return starting address of the buffer
+    */
    @Override
    public long getDataBufferAddress() {
       return (valueBuffer.memoryAddress());
    }
 
+   /**
+    * Get the memory address of buffer that stores the offsets for elements
+    * in the vector. This operation is not supported for fixed-width vectors.
+    * @return starting address of the buffer
+    * @throws UnsupportedOperationException for fixed width vectors
+    */
    @Override
    public long getOffsetBufferAddress() {
       throw new UnsupportedOperationException("not supported for fixed-width vectors");
    }
 
+   /**
+    * Get buffer that manages the validity (NULL or NON-NULL nature) of
+    * elements in the vector. Consider it as a buffer for internal bit vector
+    * data structure.
+    * @return buffer
+    */
    @Override
    public ArrowBuf getValidityBuffer() {
       return validityBuffer;
    }
 
+   /**
+    * Get the buffer that stores the data for elements in the vector.
+    * @return buffer
+    */
    @Override
    public ArrowBuf getDataBuffer() {
       return valueBuffer;
    }
 
+   /**
+    * buffer that stores the offsets for elements
+    * in the vector. This operation is not supported for fixed-width vectors.
+    * @return buffer
+    * @throws UnsupportedOperationException for fixed width vectors
+    */
    @Override
    public ArrowBuf getOffsetBuffer() {
       throw new UnsupportedOperationException("not supported for fixed-width vectors");
    }
 
+   /**
+    * Sets the desired value capacity for the vector. This function doesn't
+    * allocate any memory for the vector.
+    * @param valueCount desired number of elements in the vector
+    */
    @Override
    public void setInitialCapacity(int valueCount) {
       final long size = (long)valueCount * typeWidth;
@@ -142,52 +179,65 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
          throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
       }
       valueAllocationSizeInBytes = (int)size;
-      validityAllocationSizeInBytes = getSizeFromCount(valueCount);
+      validityAllocationSizeInBytes = getValidityBufferSizeFromCount(valueCount);
    }
 
+   /**
+    * Get the current value capacity for the vector
+    * @return number of elements that vector can hold.
+    */
    @Override
    public int getValueCapacity(){
       return Math.min(getValueBufferValueCapacity(), getValidityBufferValueCapacity());
    }
 
-   /* for test purposes */
    private int getValueBufferValueCapacity() {
       return (int)((valueBuffer.capacity() * 1.0)/typeWidth);
    }
 
-   /* for test purposes */
    private int getValidityBufferValueCapacity() {
       return (int)(validityBuffer.capacity() * 8L);
    }
 
-   /* number of bytes for the validity buffer for the given valueCount */
-   protected int getSizeFromCount(int valueCount) {
-      return (int) Math.ceil(valueCount / 8.0);
-   }
-
+   /**
+    * zero out the vector and the data in associated buffers.
+    */
    @Override
    public void zeroVector() {
       initValidityBuffer();
       initValueBuffer();
    }
 
+   /* zero out the validity buffer */
    private void initValidityBuffer() {
       validityBuffer.setZero(0, validityBuffer.capacity());
    }
 
+   /* zero out the data buffer */
    private void initValueBuffer() {
       valueBuffer.setZero(0, valueBuffer.capacity());
    }
 
+   /**
+    * Reset the vector to initial state. Same as {@link #zeroVector()}.
+    * Note that this method doesn't release any memory.
+    */
    public void reset() {
       zeroVector();
    }
 
+   /**
+    * Close the vector and release the associated buffers.
+    */
    @Override
    public void close() { clear(); }
 
+   /**
+    * Same as {@link #close()}
+    */
    @Override
    public void clear() {
+      valueCount = 0;
       validityBuffer = releaseBuffer(validityBuffer);
       valueBuffer = releaseBuffer(valueBuffer);
    }
@@ -208,6 +258,9 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       allocationMonitor--;
    }
 
+   /**
+    * Same as {@link #allocateNewSafe()}.
+    */
    @Override
    public void allocateNew() {
       if(!allocateNewSafe()){
@@ -215,6 +268,15 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       }
    }
 
+   /**
+    * Allocate memory for the vector. We internally use a default value count
+    * of 4096 to allocate memory for at least these many elements in the
+    * vector. See {@link #allocateNew(int)} for allocating memory for specific
+    * number of elements in the vector.
+    *
+    * @return false if memory allocation fails, true otherwise.
+    */
+   @Override
    public boolean allocateNewSafe() {
       long curAllocationSizeValue = valueAllocationSizeInBytes;
       long curAllocationSizeValidity = validityAllocationSizeInBytes;
@@ -229,8 +291,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       try{
          allocateBytes(curAllocationSizeValue, curAllocationSizeValidity);
       } catch (Exception e) {
-         getLogger().error("ERROR: Failure in allocateNewSafe");
-         getLogger().error(e.getMessage());
+         e.printStackTrace();
          clear();
          return false;
       }
@@ -238,9 +299,20 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return true;
    }
 
+   /**
+    * Allocate memory for the vector to support storing at least the provided number of
+    * elements in the vector. This method must be called prior to using the ValueVector.
+    *
+    * @param valueCount the desired number of elements in the vector
+    * @throws org.apache.arrow.memory.OutOfMemoryException
+    */
    public void allocateNew(int valueCount) {
       long valueBufferSize = valueCount * typeWidth;
-      long validityBufferSize = getSizeFromCount(valueCount);
+      long validityBufferSize = getValidityBufferSizeFromCount(valueCount);
+      if (typeWidth == 0) {
+         /* specialized handling for NullableBitVector */
+         valueBufferSize = validityBufferSize;
+      }
 
       if (allocationMonitor > 10) {
          /* step down the default memory allocation since we have observed
@@ -266,8 +338,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       try {
          allocateBytes(valueBufferSize, validityBufferSize);
       } catch(Exception e) {
-         getLogger().error("ERROR: Failure in allocateNew");
-         getLogger().error(e.getMessage());
+         e.printStackTrace();
          clear();
          throw e;
       }
@@ -294,8 +365,8 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       initValidityBuffer();
    }
 
-   /*
-    * during splitAndTransfer, if we splitting from a random position within a byte,
+   /**
+    * During splitAndTransfer, if we splitting from a random position within a byte,
     * we can't just slice the source buffer so we have to explicitly allocate the
     * validityBuffer of the target vector. This is unlike the databuffer which we can
     * always slice for the target vector.
@@ -307,23 +378,49 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       initValidityBuffer();
    }
 
+   /**
+    * Get the potential buffer size for a particular number of records.
+    * @param count desired number of elements in the vector
+    * @return estimated size of underlying buffers if the vector holds
+    *         a given number of elements
+    */
    @Override
    public int getBufferSizeFor(final int count) {
       if (count == 0) { return 0; }
-      return (count * typeWidth) + getSizeFromCount(count);
+      return (count * typeWidth) + getValidityBufferSizeFromCount(count);
    }
 
+   /**
+    * Get the size (number of bytes) of underlying buffers used by this
+    * vector
+    * @return size of underlying buffers.
+    */
    @Override
    public int getBufferSize() {
       if (valueCount == 0) { return 0; }
-      return (valueCount * typeWidth) + getSizeFromCount(valueCount);
+      return (valueCount * typeWidth) + getValidityBufferSizeFromCount(valueCount);
    }
 
+   /**
+    * Get information about how this field is materialized.
+    * @return the field corresponding to this vector
+    */
    @Override
    public Field getField() {
       return field;
    }
 
+   /**
+    * Return the underlying buffers associated with this vector. Note that this doesn't
+    * impact the reference counts for this buffer so it only should be used for in-context
+    * access. Also note that this buffer changes regularly thus
+    * external classes shouldn't hold a reference to it (unless they change it).
+    *
+    * @param clear Whether to clear vector before returning; the buffers will still be refcounted
+    *              but the returned array will be the only reference to them
+    * @return The underlying {@link io.netty.buffer.ArrowBuf buffers} that is used by this
+    *         vector instance.
+    */
    @Override
    public ArrowBuf[] getBuffers(boolean clear) {
       final ArrowBuf[] buffers = new ArrowBuf[2];
@@ -338,12 +435,20 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return buffers;
    }
 
+   /**
+    * Resize the vector to increase the capacity. The internal behavior is to
+    * double the current value capacity.
+    */
    @Override
    public void reAlloc() {
       valueBuffer = reallocBufferHelper(valueBuffer, true);
       validityBuffer = reallocBufferHelper(validityBuffer, false);
    }
 
+   /**
+    * Helper method for reallocating a particular internal buffer
+    * Returns the new buffer.
+    */
    private ArrowBuf reallocBufferHelper(ArrowBuf buffer, final boolean dataBuffer) {
       final int currentBufferCapacity = buffer.capacity();
       long baseSize  = (dataBuffer ? valueAllocationSizeInBytes
@@ -360,10 +465,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
          throw new OversizedAllocationException("Unable to expand the buffer");
       }
 
-      getLogger().debug("Reallocating vector [{}]. # of bytes: [{}] -> [{}]",
-              name, (dataBuffer ? valueAllocationSizeInBytes : validityAllocationSizeInBytes),
-              newAllocationSize);
-
       final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
       newBuf.setBytes(0, buffer, 0, currentBufferCapacity);
       final int halfNewCapacity = newBuf.capacity() / 2;
@@ -381,8 +482,17 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
    }
 
    @Override
-   public List<BufferBacked> getFieldInnerVectors() { throw new UnsupportedOperationException(); }
+   @Deprecated
+   public List<BufferBacked> getFieldInnerVectors() {
+      throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
+   }
 
+   /**
+    * Initialize the children in schema for this Field. This operation is a
+    * NO-OP for scalar types since they don't have any children.
+    * @param children the schema
+    * @throws IllegalArgumentException if children is a non-empty list for scalar types.
+    */
    @Override
    public void initializeChildrenFromFields(List<Field> children) {
       if (!children.isEmpty()) {
@@ -390,11 +500,23 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       }
    }
 
+   /**
+    * Get the inner child vectors.
+    * @return list of child vectors for complex types, empty list for scalar vector
+    * types
+    */
    @Override
    public List<FieldVector> getChildrenFromFields() {
       return Collections.emptyList();
    }
 
+   /**
+    * Load the buffers of this vector with provided source buffers.
+    * The caller manages the source buffers and populates them before invoking
+    * this method.
+    * @param fieldNode  the fieldNode indicating the value count
+    * @param ownBuffers the buffers for this Field (own buffers only, children not included)
+    */
    @Override
    public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
       if (ownBuffers.size() != 2) {
@@ -415,13 +537,22 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       validityAllocationSizeInBytes = validityBuffer.capacity();
    }
 
+   /**
+    * Get the buffers belonging to this vector
+    * @return the inner buffers.
+    */
    public List<ArrowBuf> getFieldBuffers() {
       List<ArrowBuf> result = new ArrayList<>(2);
 
       validityBuffer.readerIndex(0);
-      validityBuffer.writerIndex(getSizeFromCount(valueCount));
+      validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
       valueBuffer.readerIndex(0);
-      valueBuffer.writerIndex(valueCount * typeWidth);
+      if (typeWidth == 0) {
+         /* specialized handling for NullableBitVector */
+         valueBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+      } else {
+         valueBuffer.writerIndex(valueCount * typeWidth);
+      }
 
       result.add(validityBuffer);
       result.add(valueBuffer);
@@ -429,18 +560,42 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return result;
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @param callBack
+    * @return TransferPair
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
       return getTransferPair(ref, allocator);
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param allocator allocator for the target vector
+    * @return TransferPair
+    */
    @Override
    public TransferPair getTransferPair(BufferAllocator allocator){
       return getTransferPair(name, allocator);
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return TransferPair
+    */
    public abstract TransferPair getTransferPair(String ref, BufferAllocator allocator);
 
+   /**
+    * Transfer this vector'data to another vector. The memory associated
+    * with this vector is transferred to the allocator of target vector
+    * for accounting and management purposes.
+    * @param target destination vector for transfer
+    */
    public void transferTo(BaseNullableFixedWidthVector target){
       compareTypes(target, "transferTo");
       target.clear();
@@ -450,6 +605,13 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       clear();
    }
 
+   /**
+    * Slice this vector at desired index and length and transfer the
+    * corresponding data to the target vector.
+    * @param startIndex start position of the split in source vector.
+    * @param length length of the split.
+    * @param target destination vector
+    */
    public void splitAndTransferTo(int startIndex, int length,
                                   BaseNullableFixedWidthVector target) {
       compareTypes(target, "splitAndTransferTo");
@@ -459,6 +621,9 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       target.setValueCount(length);
    }
 
+   /**
+    * Data buffer can always be split and transferred using slicing.
+    */
    private void splitAndTransferValueBuffer(int startIndex, int length,
                                             BaseNullableFixedWidthVector target) {
       final int startPoint = startIndex * typeWidth;
@@ -466,17 +631,21 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       target.valueBuffer = valueBuffer.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer;
    }
 
+   /**
+    * Validity buffer has multiple cases of split and transfer depending on
+    * the starting position of the source index.
+    */
    private void splitAndTransferValidityBuffer(int startIndex, int length,
                                                BaseNullableFixedWidthVector target) {
       assert startIndex + length <= valueCount;
       int firstByteSource = BitVectorHelper.byteIndex(startIndex);
       int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
-      int byteSizeTarget = getSizeFromCount(length);
+      int byteSizeTarget = getValidityBufferSizeFromCount(length);
       int offset = startIndex % 8;
 
       if (length > 0) {
          if (offset == 0) {
-            // slice
+            /* slice */
             if (target.validityBuffer != null) {
                target.validityBuffer.release();
             }
@@ -493,8 +662,10 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
             target.allocateValidityBuffer(byteSizeTarget);
 
             for (int i = 0; i < byteSizeTarget - 1; i++) {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer, firstByteSource + i, offset);
-               byte b2 = getBitsFromNextByte(this.validityBuffer, firstByteSource + i + 1, offset);
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
+                       firstByteSource + i, offset);
+               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
+                       firstByteSource + i + 1, offset);
 
                target.validityBuffer.setByte(i, (b1 + b2));
             }
@@ -509,15 +680,15 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
              * by shifting data from the current byte.
              */
             if((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer,
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget - 1, offset);
-               byte b2 = getBitsFromNextByte(this.validityBuffer,
+               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget, offset);
 
                target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
             }
             else {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer,
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget - 1, offset);
                target.validityBuffer.setByte(byteSizeTarget - 1, b1);
             }
@@ -525,14 +696,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       }
    }
 
-   private static byte getBitsFromCurrentByte(ArrowBuf data, int index, int offset) {
-      return (byte)((data.getByte(index) & 0xFF) >>> offset);
-   }
-
-   private static byte getBitsFromNextByte(ArrowBuf data, int index, int offset) {
-      return (byte)((data.getByte(index) << (8 - offset)));
-   }
-
 
    /******************************************************************
     *                                                                *
@@ -547,27 +710,9 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
     * @return the number of null elements.
     */
    public int getNullCount() {
-      int count = 0;
-      final int sizeInBytes = getSizeFromCount(valueCount);
-
-      for (int i = 0; i < sizeInBytes; ++i) {
-         final byte byteValue = validityBuffer.getByte(i);
-         /* Java uses two's complement binary representation, hence 11111111_b which is -1
-          * when converted to Int will have 32bits set to 1. Masking the MSB and then
-          * adding it back solves the issue.
-          */
-         count += Integer.bitCount(byteValue & 0x7F) - (byteValue >> 7);
-      }
-      int nullCount = (sizeInBytes * 8) - count;
-      /* if the valueCount is not a multiple of 8,
-       * the bits on the right were counted as null bits.
-       */
-      int remainder = valueCount % 8;
-      nullCount -= remainder == 0 ? 0 : 8 - remainder;
-      return nullCount;
+      return BitVectorHelper.getNullCount(validityBuffer, valueCount);
    }
 
-
    /**
     * Get the value count of vector. This will always be zero unless
     * {@link #setValueCount(int)} has been called prior to calling this.
@@ -578,7 +723,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return valueCount;
    }
 
-
    /**
     * Set value count for the vector.
     *
@@ -625,7 +769,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       }
    }
 
-
    /**
     * Check if the given index is within the current value capacity
     * of the vector
@@ -637,7 +780,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return index < getValueCapacity();
    }
 
-
    /**
     * Check if element at given index is null.
     *
@@ -648,7 +790,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return (isSet(index) == 0);
    }
 
-
    /**
     * Same as {@link #isNull(int)}.
     *
@@ -662,6 +803,11 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       return Long.bitCount(b & (1L << bitIndex));
    }
 
+   /**
+    * Mark the particular position in the vector as non-null.
+    *
+    * @param index position of the element.
+    */
    public void setIndexDefined(int index) {
       handleSafe(index);
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
@@ -691,7 +837,6 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
     ******************************************************************/
 
 
-
    protected void handleSafe(int index) {
       while (index >= getValueCapacity()) {
          decrementAllocationMonitor();
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java
index a79709d..6587cde 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java
@@ -45,7 +45,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    private int offsetAllocationSizeInBytes;
 
    /* protected members */
-   protected static final int OFFSET_WIDTH = 4; /* 4 byte unsigned int to track offsets */
+   public static final int OFFSET_WIDTH = 4; /* 4 byte unsigned int to track offsets */
    protected static final byte[] emptyByteArray = new byte[]{};
    protected ArrowBuf validityBuffer;
    protected ArrowBuf valueBuffer;
@@ -59,7 +59,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
                                           FieldType fieldType) {
       super(name, allocator);
       valueAllocationSizeInBytes = INITIAL_BYTE_COUNT;
-      validityAllocationSizeInBytes = getSizeFromCount(INITIAL_VALUE_ALLOCATION);
+      validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
       offsetAllocationSizeInBytes = (INITIAL_VALUE_ALLOCATION) * OFFSET_WIDTH;
       field = new Field(name, fieldType, null);
       valueCount = 0;
@@ -70,21 +70,12 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       cleared = false;
    }
 
-  /* TODO:
-    * Determine how writerIndex and readerIndex need to be used. Right now we
-    * are setting the writerIndex and readerIndex in the call to getFieldBuffers
-    * using the valueCount -- this assumes that the caller of getFieldBuffers
-    * on the vector has already invoked setValueCount.
-    *
-    * Do we need to set them during vector transfer and splitAndTransfer?
-    */
-
    /* TODO:
     *
     * see if getNullCount() can be made faster -- O(1)
     */
 
-   /* TODO:
+  /* TODO:
     * Once the entire hierarchy has been refactored, move common functions
     * like getNullCount(), splitAndTransferValidityBuffer to top level
     * base class BaseValueVector.
@@ -93,59 +84,89 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
     * abstracted out to top level base class.
     *
     * Right now BaseValueVector is the top level base class for other
-    * vector types in ValueVector hierarchy and those vectors have not
-    * yet been refactored so moving things to the top class as of now
-    * is not a good idea.
-    */
-
-   /* TODO:
-    * See if we need logger -- unnecessary object probably
+    * vector types in ValueVector hierarchy (non-nullable) and those
+    * vectors have not yet been refactored/removed so moving things to
+    * the top class as of now is not a good idea.
     */
 
-   /* TODO:
-    * Implement getBufferSize(), getCurrentSizeInBytes().
-    */
-
-   protected abstract org.slf4j.Logger getLogger();
 
+   @Override
+   @Deprecated
    public VariableWidthMutator getMutator() {
-      throw new  UnsupportedOperationException("Mutator is not needed to write into vector");
+      throw new  UnsupportedOperationException("Mutator is not supported for writing into vector");
    }
 
+   @Override
+   @Deprecated
    public VariableWidthAccessor getAccessor() {
-      throw new UnsupportedOperationException("Accessor is not needed to read from vector");
+      throw new UnsupportedOperationException("Accessor is not supported for reading from vector");
    }
 
+   /**
+    * Get buffer that manages the validity (NULL or NON-NULL nature) of
+    * elements in the vector. Consider it as a buffer for internal bit vector
+    * data structure.
+    * @return buffer
+    */
    @Override
    public ArrowBuf getValidityBuffer() {
       return validityBuffer;
    }
 
+   /**
+    * Get the buffer that stores the data for elements in the vector.
+    * @return buffer
+    */
    @Override
    public ArrowBuf getDataBuffer() {
       return valueBuffer;
    }
 
+   /**
+    * buffer that stores the offsets for elements
+    * in the vector. This operation is not supported for fixed-width vectors.
+    * @return buffer
+    */
    @Override
    public ArrowBuf getOffsetBuffer() {
       return offsetBuffer;
    }
 
+   /**
+    * Get the memory address of buffer that stores the offsets for elements
+    * in the vector.
+    * @return starting address of the buffer
+    */
    @Override
    public long getOffsetBufferAddress() {
       return offsetBuffer.memoryAddress();
    }
 
+   /**
+    * Get the memory address of buffer that manages the validity
+    * (NULL or NON-NULL nature) of elements in the vector.
+    * @return starting address of the buffer
+    */
    @Override
    public long getValidityBufferAddress() {
       return validityBuffer.memoryAddress();
    }
 
+   /**
+    * Get the memory address of buffer that stores the data for elements
+    * in the vector.
+    * @return starting address of the buffer
+    */
    @Override
    public long getDataBufferAddress() {
       return valueBuffer.memoryAddress();
    }
 
+   /**
+    * Sets the desired value capacity for the vector. This function doesn't
+    * allocate any memory for the vector.
+    * @param valueCount desired number of elements in the vector
+    */
    @Override
    public void setInitialCapacity(int valueCount) {
       final long size = (long)valueCount * DEFAULT_RECORD_BYTE_COUNT;
@@ -153,57 +174,69 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
          throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
       }
       valueAllocationSizeInBytes = (int)size;
-      validityAllocationSizeInBytes = getSizeFromCount(valueCount);
+      validityAllocationSizeInBytes = getValidityBufferSizeFromCount(valueCount);
       /* to track the end offset of last data element in vector, we need
        * an additional slot in offset buffer.
        */
       offsetAllocationSizeInBytes = (valueCount + 1) * OFFSET_WIDTH;
    }
 
+   /**
+    * Get the current value capacity for the vector
+    * @return number of elements that vector can hold.
+    */
    @Override
    public int getValueCapacity(){
       final int offsetValueCapacity = Math.max(getOffsetBufferValueCapacity() - 1, 0);
       return Math.min(offsetValueCapacity, getValidityBufferValueCapacity());
    }
 
-   /* for test purposes */
    private int getValidityBufferValueCapacity() {
       return (int)(validityBuffer.capacity() * 8L);
    }
 
-   /* for test purposes */
    private int getOffsetBufferValueCapacity() {
       return (int)((offsetBuffer.capacity() * 1.0)/OFFSET_WIDTH);
    }
 
-   /* number of bytes for the validity buffer for a given valueCount */
-   protected int getSizeFromCount(int valueCount) {
-      return (int) Math.ceil(valueCount / 8.0);
-   }
-
+   /**
+    * zero out the vector and the data in associated buffers.
+    */
    public void zeroVector() {
       initValidityBuffer();
       initOffsetBuffer();
    }
 
+   /* zero out the validity buffer */
    private void initValidityBuffer() {
       validityBuffer.setZero(0, validityBuffer.capacity());
    }
 
+   /* zero out the offset buffer */
    private void initOffsetBuffer() {
       offsetBuffer.setZero(0, offsetBuffer.capacity());
    }
 
+   /**
+    * Reset the vector to initial state. Same as {@link #zeroVector()}.
+    * Note that this method doesn't release any memory.
+    */
    public void reset() {
       zeroVector();
       lastSet = -1;
    }
 
+   /**
+    * Close the vector and release the associated buffers.
+    */
    @Override
    public void close() {
       clear();
    }
 
+   /**
+    * Same as {@link #close()}
+    */
    @Override
    public void clear() {
       validityBuffer = releaseBuffer(validityBuffer);
@@ -215,8 +248,17 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    }
 
    @Override
-   public List<BufferBacked> getFieldInnerVectors() { throw new UnsupportedOperationException(); }
+   @Deprecated
+   public List<BufferBacked> getFieldInnerVectors() {
+      throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
+   }
 
+   /**
+    * Initialize the children in schema for this Field. This operation is a
+    * NO-OP for scalar types since they don't have any children.
+    * @param children the schema
+    * @throws IllegalArgumentException if children is a non-empty list for scalar types.
+    */
    @Override
    public void initializeChildrenFromFields(List<Field> children) {
       if (!children.isEmpty()) {
@@ -224,11 +266,24 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       }
    }
 
+   /**
+    * Get the inner child vectors.
+    * @return list of child vectors for complex types, empty list for scalar vector
+    * types
+    */
    @Override
    public List<FieldVector> getChildrenFromFields() {
       return Collections.emptyList();
    }
 
+
+   /**
+    * Load the buffers of this vector with provided source buffers.
+    * The caller manages the source buffers and populates them before invoking
+    * this method.
+    * @param fieldNode  the fieldNode indicating the value count
+    * @param ownBuffers the buffers for this Field (own buffers only, children not included)
+    */
    @Override
    public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
       ArrowBuf bitBuffer = ownBuffers.get(0);
@@ -246,11 +301,15 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       valueCount = fieldNode.getLength();
    }
 
+   /**
+    * Get the buffers belonging to this vector
+    * @return the inner buffers.
+    */
    public List<ArrowBuf> getFieldBuffers() {
       List<ArrowBuf> result = new ArrayList<>(3);
       final int lastDataOffset = getstartOffset(valueCount);
       validityBuffer.readerIndex(0);
-      validityBuffer.writerIndex(getSizeFromCount(valueCount));
+      validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
       offsetBuffer.readerIndex(0);
       offsetBuffer.writerIndex((valueCount + 1) * OFFSET_WIDTH);
       valueBuffer.readerIndex(0);
@@ -263,6 +322,9 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       return result;
    }
 
+   /**
+    * Same as {@link #allocateNewSafe()}.
+    */
    @Override
    public void allocateNew() {
       if(!allocateNewSafe()){
@@ -270,6 +332,14 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       }
    }
 
+   /**
+    * Allocate memory for the vector. We internally use a default value count
+    * of 4096 to allocate memory for at least these many elements in the
+    * vector. See {@link #allocateNew(int, int)} for allocating memory for specific
+    * number of elements in the vector.
+    *
+    * @return false if memory allocation fails, true otherwise.
+    */
    @Override
    public boolean allocateNewSafe() {
       long curAllocationSizeValue = valueAllocationSizeInBytes;
@@ -287,8 +357,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       try {
          allocateBytes(curAllocationSizeValue, curAllocationSizeValidity, curAllocationSizeOffset);
       } catch (Exception e) {
-         getLogger().error("ERROR: Failure in allocateNewSafe");
-         getLogger().error(e.getMessage());
+         e.printStackTrace();
          clear();
          return false;
       }
@@ -296,11 +365,19 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       return true;
    }
 
+   /**
+    * Allocate memory for the vector to support storing at least the provided number of
+    * elements in the vector. This method must be called prior to using the ValueVector.
+    *
+    * @param totalBytes desired total memory capacity
+    * @param valueCount the desired number of elements in the vector
+    * @throws org.apache.arrow.memory.OutOfMemoryException
+    */
    @Override
    public void allocateNew(int totalBytes, int valueCount) {
       assert totalBytes >= 0;
       final int offsetBufferSize = (valueCount + 1) * OFFSET_WIDTH;
-      final int validityBufferSize = getSizeFromCount(valueCount);
+      final int validityBufferSize = getValidityBufferSizeFromCount(valueCount);
 
       if (totalBytes > MAX_ALLOCATION_SIZE ||
               offsetBufferSize > MAX_ALLOCATION_SIZE) {
@@ -313,12 +390,12 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       try {
          allocateBytes(totalBytes, validityBufferSize, offsetBufferSize);
       } catch (Exception e) {
-         getLogger().error("ERROR: Failure in allocateNewSafe");
-         getLogger().error(e.getMessage());
+         e.printStackTrace();
          clear();
       }
    }
 
+   /* allocate the inner buffers */
    private void allocateBytes(final long valueBufferSize, final long validityBufferSize,
                               final long offsetBufferSize) {
       /* allocate data buffer */
@@ -330,6 +407,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       allocateOffsetBuffer(offsetBufferSize);
    }
 
+   /* allocate offset buffer */
    private void allocateOffsetBuffer(final long size) {
       final int curSize = (int)size;
       offsetBuffer = allocator.buffer(curSize);
@@ -338,6 +416,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       initOffsetBuffer();
    }
 
+   /* allocate validity buffer */
    private void allocateValidityBuffer(final long size) {
       final int curSize = (int)size;
       validityBuffer = allocator.buffer(curSize);
@@ -346,6 +425,10 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       initValidityBuffer();
    }
 
+   /**
+    * Resize the vector to increase the capacity. The internal behavior is to
+    * double the current value capacity.
+    */
    public void reAlloc() {
       reallocValueBuffer();
       reallocValidityAndOffsetBuffers();
@@ -378,10 +461,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       validityBuffer = reallocBufferHelper(validityBuffer, false);
    }
 
-   /* need to refactor this to keep the logic in an single place and make callers
-    * more intelligent. see handleSafe() for more comments on realloc
-    */
-
+   /* helper method to realloc a particular buffer. returns the allocated buffer */
    private ArrowBuf reallocBufferHelper(ArrowBuf buffer, final boolean offsetBuffer) {
       final int currentBufferCapacity = buffer.capacity();
       long baseSize  = (offsetBuffer ? offsetAllocationSizeInBytes
@@ -398,10 +478,6 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
          throw new OversizedAllocationException("Unable to expand the buffer");
       }
 
-      getLogger().debug("Reallocating vector [{}]. # of bytes: [{}] -> [{}]",
-              name, (offsetBuffer ? offsetAllocationSizeInBytes : validityAllocationSizeInBytes),
-              newAllocationSize);
-
       final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
       newBuf.setBytes(0, buffer, 0, currentBufferCapacity);
       final int halfNewCapacity = newBuf.capacity() / 2;
@@ -418,6 +494,10 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       return buffer;
    }
 
+   /**
+    * Get the size (number of bytes) of underlying data buffer.
+    * @return
+    */
    @Override
    public int getByteCapacity(){
       return valueBuffer.capacity();
@@ -429,30 +509,55 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       return 0;
    }
 
+   /**
+    * Get the size (number of bytes) of underlying buffers used by this
+    * vector
+    * @return size of underlying buffers.
+    */
    @Override
    public int getBufferSize() {
-      /* TODO */
-      return 0;
+      return getBufferSizeFor(this.valueCount);
    }
 
+   /**
+    * Get the potential buffer size for a particular number of records.
+    * @param valueCount desired number of elements in the vector
+    * @return estimated size of underlying buffers if the vector holds
+    *         a given number of elements
+    */
    @Override
    public int getBufferSizeFor(final int valueCount) {
       if (valueCount == 0) {
          return 0;
       }
 
-      final int validityBufferSize = getSizeFromCount(valueCount);
+      final int validityBufferSize = getValidityBufferSizeFromCount(valueCount);
       final int offsetBufferSize = (valueCount + 1) * OFFSET_WIDTH;
       /* get the end offset for this valueCount */
       final int dataBufferSize = offsetBuffer.getInt(valueCount * OFFSET_WIDTH);
       return validityBufferSize + offsetBufferSize + dataBufferSize;
    }
 
+   /**
+    * Get information about how this field is materialized.
+    * @return the field corresponding to this vector
+    */
    @Override
    public Field getField() {
       return field;
    }
 
+   /**
+    * Return the underlying buffers associated with this vector. Note that this doesn't
+    * impact the reference counts for this buffer so it only should be used for in-context
+    * access. Also note that this buffer changes regularly thus
+    * external classes shouldn't hold a reference to it (unless they change it).
+    *
+    * @param clear Whether to clear vector before returning; the buffers will still be refcounted
+    *              but the returned array will be the only reference to them
+    * @return The underlying {@link io.netty.buffer.ArrowBuf buffers} that is used by this
+    *         vector instance.
+    */
    @Override
    public ArrowBuf[] getBuffers(boolean clear) {
       final ArrowBuf[] buffers = new ArrowBuf[3];
@@ -468,18 +573,42 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       return buffers;
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @param callBack
+    * @return TransferPair
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
       return getTransferPair(ref, allocator);
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param allocator allocator for the target vector
+    * @return TransferPair
+    */
    @Override
    public TransferPair getTransferPair(BufferAllocator allocator){
       return getTransferPair(name, allocator);
    }
 
+   /**
+    * Construct a transfer pair of this vector and another vector of same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return TransferPair
+    */
    public abstract TransferPair getTransferPair(String ref, BufferAllocator allocator);
 
+   /**
+    * Transfer this vector'data to another vector. The memory associated
+    * with this vector is transferred to the allocator of target vector
+    * for accounting and management purposes.
+    * @param target destination vector for transfer
+    */
    public void transferTo(BaseNullableVariableWidthVector target){
       compareTypes(target, "transferTo");
       target.clear();
@@ -491,6 +620,13 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       clear();
    }
 
+   /**
+    * Slice this vector at desired index and length and transfer the
+    * corresponding data to the target vector.
+    * @param startIndex start position of the split in source vector.
+    * @param length length of the split.
+    * @param target destination vector
+    */
    public void splitAndTransferTo(int startIndex, int length,
                                   BaseNullableVariableWidthVector target) {
       compareTypes(target, "splitAndTransferTo");
@@ -502,7 +638,12 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    }
 
    /*
-    * transfer the offsets along with data
+    * Transfer the offsets along with data. Unlike the data buffer, we cannot simply
+    * slice the offset buffer for split and transfer. The reason is that offsets
+    * in the target vector have to be adjusted and made relative to the staring
+    * offset in source vector from the start index of split. This is why, we
+    * need to explicitly allocate the offset buffer and set the adjusted offsets
+    * in the target vector.
     */
    private void splitAndTransferOffsetBuffer(int startIndex, int length, BaseNullableVariableWidthVector target) {
       final int start = offsetBuffer.getInt(startIndex * OFFSET_WIDTH);
@@ -517,14 +658,14 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    }
 
    /*
-    * transfer the validity.
+    * Transfer the validity.
     */
    private void splitAndTransferValidityBuffer(int startIndex, int length,
                                                BaseNullableVariableWidthVector target) {
       assert startIndex + length <= valueCount;
       int firstByteSource = BitVectorHelper.byteIndex(startIndex);
       int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
-      int byteSizeTarget = getSizeFromCount(length);
+      int byteSizeTarget = getValidityBufferSizeFromCount(length);
       int offset = startIndex % 8;
 
       if (length > 0) {
@@ -546,8 +687,8 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
             target.allocateValidityBuffer(byteSizeTarget);
 
             for (int i = 0; i < byteSizeTarget - 1; i++) {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer, firstByteSource + i, offset);
-               byte b2 = getBitsFromNextByte(this.validityBuffer, firstByteSource + i + 1, offset);
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer, firstByteSource + i, offset);
+               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer, firstByteSource + i + 1, offset);
 
                target.validityBuffer.setByte(i, (b1 + b2));
             }
@@ -562,15 +703,15 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
              * by shifting data from the current byte.
              */
             if((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer,
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget - 1, offset);
-               byte b2 = getBitsFromNextByte(this.validityBuffer,
+               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget, offset);
 
                target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
             }
             else {
-               byte b1 = getBitsFromCurrentByte(this.validityBuffer,
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
                        firstByteSource + byteSizeTarget - 1, offset);
                target.validityBuffer.setByte(byteSizeTarget - 1, b1);
             }
@@ -578,14 +719,6 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       }
    }
 
-   private static byte getBitsFromCurrentByte(ArrowBuf data, int index, int offset) {
-      return (byte)((data.getByte(index) & 0xFF) >>> offset);
-   }
-
-   private static byte getBitsFromNextByte(ArrowBuf data, int index, int offset) {
-      return (byte)((data.getByte(index) << (8 - offset)));
-   }
-
 
    /******************************************************************
     *                                                                *
@@ -600,24 +733,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
     * @return the number of null elements.
     */
    public int getNullCount() {
-      int count = 0;
-      final int sizeInBytes = getSizeFromCount(valueCount);
-
-      for (int i = 0; i < sizeInBytes; ++i) {
-         final byte byteValue = validityBuffer.getByte(i);
-         /* Java uses two's complement binary representation, hence 11111111_b which is -1
-          * when converted to Int will have 32bits set to 1. Masking the MSB and then
-          * adding it back solves the issue.
-          */
-         count += Integer.bitCount(byteValue & 0x7F) - (byteValue >> 7);
-      }
-      int nullCount = (sizeInBytes * 8) - count;
-      /* if the valueCount is not a multiple of 8,
-       * the bits on the right were counted as null bits.
-       */
-      int remainder = valueCount % 8;
-      nullCount -= remainder == 0 ? 0 : 8 - remainder;
-      return nullCount;
+      return BitVectorHelper.getNullCount(validityBuffer, valueCount);
    }
 
    /**
@@ -679,24 +795,55 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
       lastSet = valueCount - 1;
    }
 
+   /**
+    * Create holes in the vector upto the given index (exclusive).
+    * Holes will be created from the current last set position in
+    * the vector.
+    *
+    * @param index target index
+    */
    public void fillEmpties(int index) {
       handleSafe(index, emptyByteArray.length);
       fillHoles(index);
       lastSet = index - 1;
    }
 
+   /**
+    * Set the index of last non-null element in the vector.
+    * It is important to call this method with appropriate value
+    * before calling {@link #setValueCount(int)}.
+    *
+    * @param value desired index of last non-null element.
+    */
    public void setLastSet(int value) {
       lastSet = value;
    }
 
+   /**
+    * Get the index of last non-null element in the vector.
+    *
+    * @return index of the last non-null element
+    */
    public int getLastSet() {
       return lastSet;
    }
 
+   /**
+    * Get the starting position (offset) in the data stream for a given
+    * element in the vector.
+    *
+    * @param index position of the element in the vector
+    * @return starting offset for the element
+    */
    public long getStartEnd(int index) {
       return (long)offsetBuffer.getInt(index * OFFSET_WIDTH);
    }
 
+   /**
+    * Mark the particular position in the vector as non-null.
+    *
+    * @param index position of the element.
+    */
    public void setIndexDefined(int index) {
       handleSafe(index, 0);
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
@@ -761,4 +908,97 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
          reallocValueBuffer();
       }
    }
+
+
+   /******************************************************************
+    *                                                                *
+    *                helper methods currently                        *
+    *                used by JsonFileReader and                      *
+    *                JsonFileWriter                                  *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Method used by Json Reader to explicitly set the data of the variable
+    * width vector elements. The method takes care of allocating the memory
+    * for the vector if caller hasn't done so.
+    *
+    * This method should not be used externally.
+    *
+    * @param data ArrowBuf for storing variable width elements in the vector
+    * @param offset offset of the element
+    * @param allocator memory allocator
+    * @param index position of the element in the vector
+    * @param value array of bytes for the element
+    * @param valueCount number of elements in the vector
+    * @return buffer holding the variable width data.
+    */
+   public static ArrowBuf set(ArrowBuf data, ArrowBuf offset,
+                              BufferAllocator allocator, int index, byte[] value,
+                              int valueCount) {
+      if (data == null) {
+         data = allocator.buffer(INITIAL_BYTE_COUNT);
+      }
+      final int currentBufferCapacity = data.capacity();
+      final int currentStartOffset = offset.getInt(index * OFFSET_WIDTH);
+      while (currentBufferCapacity < currentStartOffset + value.length) {
+         final ArrowBuf newBuf = allocator.buffer(currentBufferCapacity * 2);
+         newBuf.setBytes(0, data, 0, currentBufferCapacity);
+         data.release();
+         data = newBuf;
+      }
+      data.setBytes(currentStartOffset, value, 0, value.length);
+      if (index == (valueCount - 1)) {
+         data.writerIndex(offset.getInt(valueCount * OFFSET_WIDTH));
+      }
+      return data;
+   }
+
+   /**
+    * Method used by Json Writer to read a variable width element from
+    * the variable width vector and write to Json.
+    *
+    * This method should not be used externally.
+    *
+    * @param data buffer storing the variable width vector elements
+    * @param offset buffer storing the offsets of variable width vector elements
+    * @param index position of the element in the vector
+    * @return array of bytes
+    */
+   public static byte[] get(final ArrowBuf data, final ArrowBuf offset, int index) {
+      final int currentStartOffset = offset.getInt(index * OFFSET_WIDTH);
+      final int dataLength =
+              offset.getInt((index + 1) * OFFSET_WIDTH) - currentStartOffset;
+      final byte[] result = new byte[dataLength];
+      data.getBytes(currentStartOffset, result, 0, dataLength);
+      return result;
+   }
+
+   /**
+    * Method used by Json Reader to explicitly set the offsets of the variable
+    * width vector data. The method takes care of allocating the memory for
+    * offsets if the caller hasn't done so.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer ArrowBuf to store offsets for variable width elements
+    * @param allocator memory allocator
+    * @param valueCount number of elements
+    * @param index position of the element
+    * @param value offset of the element
+    * @return buffer holding the offsets
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, int value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * OFFSET_WIDTH);
+      }
+      buffer.setInt(index * OFFSET_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * OFFSET_WIDTH);
+      }
+
+      return buffer;
+   }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java
index 1db9624..1b96782 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseValueVector.java
@@ -56,7 +56,6 @@ public abstract class BaseValueVector implements ValueVector {
 
   @Override
   public void clear() {
-    getMutator().reset();
   }
 
   @Override
@@ -137,10 +136,36 @@ public abstract class BaseValueVector implements ValueVector {
     return buffer;
   }
 
-  public int getValueCount() { return 0; }
+  @Override
+  public int getValueCount() {
+    return getAccessor().getValueCount();
+  }
+
+  @Override
+  public void setValueCount(int valueCount) {
+    getMutator().setValueCount(valueCount);
+  }
+
+  @Override
+
+  public Object getObject(int index) {
+    return getAccessor().getObject(index);
+  }
+
+  @Override
+
+  public int getNullCount() {
+    return getAccessor().getNullCount();
+  }
 
-  public void setValueCount(int valueCount) { }
+  @Override
+  public boolean isNull(int index) {
+    return getAccessor().isNull(index);
+  }
 
-  public Object getObject(int index) { return null; }
+  /* number of bytes for the validity buffer for the given valueCount */
+  protected static int getValidityBufferSizeFromCount(final int valueCount) {
+    return (int) Math.ceil(valueCount / 8.0);
+  }
 }
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java
index 2439bd2..69325b5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVectorHelper.java
@@ -19,24 +19,35 @@
 package org.apache.arrow.vector;
 
 import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
 
-class BitVectorHelper {
+/**
+ * Helper class for performing generic operations on a bit vector buffer.
+ * External use of this class is not recommended.
+ */
+public class BitVectorHelper {
 
    /**
     * Get the index of byte corresponding to bit index in validity buffer
     */
-   protected static int byteIndex(int absoluteBitIndex) {
+   public static int byteIndex(int absoluteBitIndex) {
       return absoluteBitIndex >> 3;
    }
 
    /**
     * Get the relative index of bit within the byte in validity buffer
     */
-   private static int bitIndex(int absoluteBitIndex) {
+   public static int bitIndex(int absoluteBitIndex) {
       return absoluteBitIndex & 7;
    }
 
-   protected static void setValidityBitToOne(ArrowBuf validityBuffer, int index) {
+   /**
+    * Set the bit at provided index to 1.
+    *
+    * @param validityBuffer
+    * @param index
+    */
+   public static void setValidityBitToOne(ArrowBuf validityBuffer, int index) {
       final int byteIndex = byteIndex(index);
       final int bitIndex = bitIndex(index);
       byte currentByte = validityBuffer.getByte(byteIndex);
@@ -45,7 +56,14 @@ class BitVectorHelper {
       validityBuffer.setByte(byteIndex, currentByte);
    }
 
-   protected static void setValidityBit(ArrowBuf validityBuffer, int index, int value) {
+   /**
+    * Set the bit at a given index to provided value (1 or 0)
+    *
+    * @param validityBuffer
+    * @param index
+    * @param value
+    */
+   public static void setValidityBit(ArrowBuf validityBuffer, int index, int value) {
       final int byteIndex = byteIndex(index);
       final int bitIndex = bitIndex(index);
       byte currentByte = validityBuffer.getByte(byteIndex);
@@ -57,4 +75,91 @@ class BitVectorHelper {
       }
       validityBuffer.setByte(byteIndex, currentByte);
    }
+
+   /**
+    * Set the bit at a given index to provided value (1 or 0). Internally
+    * takes care of allocating the buffer if the caller didn't do so.
+    *
+    * @param validityBuffer
+    * @param allocator
+    * @param valueCount
+    * @param index
+    * @param value
+    * @return ArrowBuf
+    */
+   public static ArrowBuf setValidityBit(ArrowBuf validityBuffer, BufferAllocator allocator,
+                                         int valueCount, int index, int value) {
+      if (validityBuffer == null) {
+         validityBuffer = allocator.buffer(getValidityBufferSize(valueCount));
+      }
+      setValidityBit(validityBuffer, index, value);
+      if (index == (valueCount - 1)) {
+         validityBuffer.writerIndex(getValidityBufferSize(valueCount));
+      }
+
+      return validityBuffer;
+   }
+
+   /**
+    * Check if a bit at a given index is set or not.
+    *
+    * @param buffer
+    * @param index
+    * @return 1 if bit is set, 0 otherwise.
+    */
+   public static int get(final ArrowBuf buffer, int index) {
+      final int byteIndex = index >> 3;
+      final byte b = buffer.getByte(byteIndex);
+      final int bitIndex = index & 7;
+      return Long.bitCount(b & (1L << bitIndex));
+   }
+
+   /**
+    * Compute the size of validity buffer required to manage a given number
+    * of elements in a vector.
+    *
+    * @param valueCount
+    * @return buffer size
+    */
+   public static int getValidityBufferSize(int valueCount) {
+      return ((int) Math.ceil(valueCount / 8.0));
+   }
+
+   /**
+    * Given a validity buffer, find the number of bits that are not set.
+    * This is used to compute the number of null elements in a nullable vector.
+    *
+    * @param validityBuffer
+    * @param valueCount
+    * @return number of bits not set.
+    */
+   public static int getNullCount(final ArrowBuf validityBuffer, final int valueCount) {
+      if (valueCount == 0) { return 0; }
+      int count = 0;
+      final int sizeInBytes = getValidityBufferSize(valueCount);
+
+      for (int i = 0; i < sizeInBytes; ++i) {
+         final byte byteValue = validityBuffer.getByte(i);
+         /* Java uses two's complement binary representation, hence 11111111_b which is -1
+          * when converted to Int will have 32bits set to 1. Masking the MSB and then
+          * adding it back solves the issue.
+          */
+         count += Integer.bitCount(byteValue & 0x7F) - (byteValue >> 7);
+      }
+      int nullCount = (sizeInBytes * 8) - count;
+      /* if the valueCount is not a multiple of 8,
+       * the bits on the right were counted as null bits.
+       */
+      int remainder = valueCount % 8;
+      nullCount -= remainder == 0 ? 0 : 8 - remainder;
+      return nullCount;
+   }
+
+   public static byte getBitsFromCurrentByte(final ArrowBuf data, final int index, final int offset) {
+      return (byte)((data.getByte(index) & 0xFF) >>> offset);
+   }
+
+   public static byte getBitsFromNextByte(ArrowBuf data, int index, int offset) {
+      return (byte)((data.getByte(index) << (8 - offset)));
+   }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
similarity index 50%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
index 26b19fa..ee40d70 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.BigIntReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.BigIntHolder;
+import org.apache.arrow.vector.holders.NullableBigIntHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableBigIntVector implements a fixed width vector (8 bytes) of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableBigIntVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableBigIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableBigIntVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.BIGINT.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableBigIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableBigIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new BigIntReaderImpl(NullableBigIntVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.BIGINT;
    }
 
 
@@ -72,18 +86,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *                                                                *
     ******************************************************************/
 
-
    /**
     * Get the element at the given index from the vector.
     *
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public long get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +106,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableBigIntHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +121,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Long getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +129,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableBigIntVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableBigIntVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableBigIntVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +163,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +173,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, long value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +186,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableBigIntHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +205,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, BigIntHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, long)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, long value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableBigIntHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableBigIntHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, BigIntHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, BigIntHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +262,81 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, long value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, long)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, long value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, long value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setLong(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static long get(final ArrowBuf buffer, final int index) {
+      return buffer.getLong(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +347,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableBigIntVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableBigIntVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableBigIntVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableBigIntVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableBigIntVector getTo(){
          return to;
       }
 
@@ -293,7 +397,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableBigIntVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.java
new file mode 100644
index 0000000..eceaa61
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.java
@@ -0,0 +1,502 @@
+/*******************************************************************************
+
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.BitReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.BitHolder;
+import org.apache.arrow.vector.holders.NullableBitHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.OversizedAllocationException;
+import org.apache.arrow.vector.util.TransferPair;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * NullableBitVector implements a fixed width (1 bit) vector of
+ * boolean values which could be null. Each value in the vector corresponds
+ * to a single bit in the underlying data stream backing the vector.
+ */
+public class NullableBitVector extends BaseNullableFixedWidthVector {
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableBitVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableBitVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.BIT.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableBitVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableBitVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, (byte)0);
+      reader = new BitReaderImpl(NullableBitVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.BIT;
+   }
+
+   /**
+    * Sets the desired value capacity for the vector. This function doesn't
+    * allocate any memory for the vector.
+    * @param valueCount desired number of elements in the vector
+    */
+   @Override
+   public void setInitialCapacity(int valueCount) {
+      final int size = getValidityBufferSizeFromCount(valueCount);
+      if (size > MAX_ALLOCATION_SIZE) {
+         throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
+      }
+      valueAllocationSizeInBytes = size;
+      validityAllocationSizeInBytes = size;
+   }
+
+   /**
+    * Get the current value capacity for the vector
+    * @return number of elements that vector can hold.
+    */
+   @Override
+   public int getValueCapacity(){
+      return (int)(validityBuffer.capacity() * 8L);
+   }
+
+   /**
+    * Get the potential buffer size for a particular number of records.
+    * @param count desired number of elements in the vector
+    * @return estimated size of underlying buffers if the vector holds
+    *         a given number of elements
+    */
+   @Override
+   public int getBufferSizeFor(final int count) {
+      if (count == 0) { return 0; }
+      return 2 * getValidityBufferSizeFromCount(count);
+   }
+
+   /**
+    * Get the size (number of bytes) of underlying buffers used by this
+    * vector
+    * @return size of underlying buffers.
+    */
+   @Override
+   public int getBufferSize() {
+     return getBufferSizeFor(valueCount);
+   }
+
+   /**
+    * Slice this vector at desired index and length and transfer the
+    * corresponding data to the target vector.
+    * @param startIndex start position of the split in source vector.
+    * @param length length of the split.
+    * @param target destination vector
+    */
+   public void splitAndTransferTo(int startIndex, int length,
+                                  BaseNullableFixedWidthVector target) {
+      compareTypes(target, "splitAndTransferTo");
+      target.clear();
+      target.validityBuffer = splitAndTransferBuffer(startIndex, length, target,
+                                 validityBuffer, target.validityBuffer);
+      target.valueBuffer = splitAndTransferBuffer(startIndex, length, target,
+                                 valueBuffer, target.valueBuffer);
+
+      target.setValueCount(length);
+   }
+
+   private ArrowBuf splitAndTransferBuffer(int startIndex, int length,
+                                           BaseNullableFixedWidthVector target,
+                                           ArrowBuf sourceBuffer, ArrowBuf destBuffer) {
+      assert startIndex + length <= valueCount;
+      int firstByteSource = BitVectorHelper.byteIndex(startIndex);
+      int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
+      int byteSizeTarget = getValidityBufferSizeFromCount(length);
+      int offset = startIndex % 8;
+
+      if (length > 0) {
+         if (offset == 0) {
+            /* slice */
+            if (destBuffer != null) {
+               destBuffer.release();
+            }
+            destBuffer = destBuffer.slice(firstByteSource, byteSizeTarget);
+            destBuffer.retain(1);
+         }
+         else {
+            /* Copy data
+             * When the first bit starts from the middle of a byte (offset != 0),
+             * copy data from src BitVector.
+             * Each byte in the target is composed by a part in i-th byte,
+             * another part in (i+1)-th byte.
+             */
+            destBuffer = allocator.buffer(byteSizeTarget);
+            destBuffer.readerIndex(0);
+            destBuffer.setZero(0, destBuffer.capacity());
+
+            for (int i = 0; i < byteSizeTarget - 1; i++) {
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer, firstByteSource + i, offset);
+               byte b2 = BitVectorHelper.getBitsFromNextByte(sourceBuffer, firstByteSource + i + 1, offset);
+
+               destBuffer.setByte(i, (b1 + b2));
+            }
+
+            /* Copying the last piece is done in the following manner:
+             * if the source vector has 1 or more bytes remaining, we copy
+             * the last piece as a byte formed by shifting data
+             * from the current byte and the next byte.
+             *
+             * if the source vector has no more bytes remaining
+             * (we are at the last byte), we copy the last piece as a byte
+             * by shifting data from the current byte.
+             */
+            if((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
+                       firstByteSource + byteSizeTarget - 1, offset);
+               byte b2 = BitVectorHelper.getBitsFromNextByte(sourceBuffer,
+                       firstByteSource + byteSizeTarget, offset);
+
+               destBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+            }
+            else {
+               byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
+                       firstByteSource + byteSizeTarget - 1, offset);
+               destBuffer.setByte(byteSizeTarget - 1, b1);
+            }
+         }
+      }
+
+      return destBuffer;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+   private int getBit(int index) {
+      final int byteIndex = index >> 3;
+      final byte b = valueBuffer.getByte(byteIndex);
+      final int bitIndex = index & 7;
+      return Long.bitCount(b & (1L << bitIndex));
+   }
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public int get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return getBit(index);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableBitHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = getBit(index);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Boolean getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return new Boolean (getBit(index) != 0);
+      }
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableBitVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         set(thisIndex, from.get(fromIndex));
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableBitVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableBitVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, int value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      if (value != 0) {
+         BitVectorHelper.setValidityBitToOne(valueBuffer, index);
+      } else {
+         BitVectorHelper.setValidityBit(valueBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableBitHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         if (holder.value != 0) {
+            BitVectorHelper.setValidityBitToOne(valueBuffer, index);
+         } else {
+            BitVectorHelper.setValidityBit(valueBuffer, index, 0);
+         }
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, BitHolder holder) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      if (holder.value != 0) {
+         BitVectorHelper.setValidityBitToOne(valueBuffer, index);
+      } else {
+         BitVectorHelper.setValidityBit(valueBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, int value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableBitHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableBitHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, BitHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, BitHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
+      if (isSet > 0) {
+         set(index, value);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
+      handleSafe(index);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableBitVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableBitVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableBitVector(ref, field.getFieldType(), allocator);
+      }
+
+      public TransferImpl(NullableBitVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableBitVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableBitVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
similarity index 55%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
index 26b19fa..949287e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,70 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.DateDayReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.DateDayHolder;
+import org.apache.arrow.vector.holders.NullableDateDayHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableDateDayVector implements a fixed width (4 bytes) vector of
+ * date values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableDateDayVector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableDateDayVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDateDayVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.DATEDAY.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableDateDayVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDateDayVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new DateDayReaderImpl(NullableDateDayVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.DATEDAY;
    }
 
 
@@ -93,7 +108,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableDateDayHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
@@ -116,13 +131,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableDateDayVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableDateDayVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableDateDayVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -158,7 +188,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableDateDayHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +207,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, DateDayHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +226,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableDateDayHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableDateDayHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, DateDayHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, DateDayHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +264,80 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader                                *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, int value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setInt(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static int get(final ArrowBuf buffer, final int index) {
+      return buffer.getInt(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +348,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableDateDayVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableDateDayVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableDateDayVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableDateDayVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableDateDayVector getTo(){
          return to;
       }
 
@@ -293,7 +398,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableDateDayVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
new file mode 100644
index 0000000..a0bdcce
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
@@ -0,0 +1,408 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.DateMilliReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.DateMilliHolder;
+import org.apache.arrow.vector.holders.NullableDateMilliHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
+import org.joda.time.LocalDateTimes;
+import org.slf4j.Logger;
+
+/**
+ * NullableDateMilliVector implements a fixed width vector (8 bytes) of
+ * date values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public class NullableDateMilliVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableDateMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDateMilliVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.DATEMILLI.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableDateMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDateMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+      reader = new DateMilliReaderImpl(NullableDateMilliVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.DATEMILLI;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public long get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableDateMilliHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public LocalDateTime getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         final long millis = get(index);
+         final LocalDateTime localDateTime = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
+         return localDateTime;
+      }
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableDateMilliVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         set(thisIndex, from.get(fromIndex));
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableDateMilliVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableDateMilliVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   private void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, long value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableDateMilliHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, DateMilliHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, long)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, long value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableDateMilliHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableDateMilliHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, DateMilliHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, DateMilliHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, long value) {
+      if (isSet > 0) {
+         set(index, value);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, long)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, long value) {
+      handleSafe(index);
+      set(index, isSet, value);
+   }
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader                                *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, long value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setLong(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static long get(final ArrowBuf buffer, final int index) {
+      return buffer.getLong(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableDateMilliVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableDateMilliVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableDateMilliVector(ref, field.getFieldType(), allocator);
+      }
+
+      public TransferImpl(NullableDateMilliVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableDateMilliVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableDateMilliVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
new file mode 100644
index 0000000..8320f90
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
@@ -0,0 +1,454 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.DecimalReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.DecimalHolder;
+import org.apache.arrow.vector.holders.NullableDecimalHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.DecimalUtility;
+import org.apache.arrow.vector.util.TransferPair;
+
+import java.math.BigDecimal;
+
+/**
+ * NullableDecimalVector implements a fixed width vector (16 bytes) of
+ * decimal values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public class NullableDecimalVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 16;
+   private final FieldReader reader;
+
+   private final int precision;
+   private final int scale;
+
+   /**
+    * Instantiate a NullableDecimalVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDecimalVector(String name, BufferAllocator allocator,
+                                int precision, int scale) {
+      this(name, FieldType.nullable(new org.apache.arrow.vector.types.pojo.ArrowType.Decimal(precision, scale)),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableDecimalVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableDecimalVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+      org.apache.arrow.vector.types.pojo.ArrowType.Decimal arrowType = (org.apache.arrow.vector.types.pojo.ArrowType.Decimal)fieldType.getType();
+      reader = new DecimalReaderImpl(NullableDecimalVector.this);
+      this.precision = arrowType.getPrecision();
+      this.scale = arrowType.getScale();
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.DECIMAL;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public ArrowBuf get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return valueBuffer.slice(index * TYPE_WIDTH, TYPE_WIDTH);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableDecimalHolder holder) {
+      if (isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.buffer = valueBuffer;
+      holder.precision = precision;
+      holder.scale = scale;
+      holder.start = index * TYPE_WIDTH;
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public BigDecimal getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return DecimalUtility.getBigDecimalFromArrowBuf(valueBuffer, index, scale);
+      }
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableDecimalVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, valueBuffer,
+                 thisIndex * TYPE_WIDTH, TYPE_WIDTH);
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableDecimalVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableDecimalVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index    position of element
+    * @param buffer   ArrowBuf containing decimal value.
+    */
+   public void set(int index, ArrowBuf buffer) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      valueBuffer.setBytes(index * TYPE_WIDTH, buffer, 0, TYPE_WIDTH);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index    position of element
+    * @param start    start index of data in the buffer
+    * @param buffer   ArrowBuf containing decimal value.
+    */
+   public void set(int index, int start, ArrowBuf buffer) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      valueBuffer.setBytes(index * TYPE_WIDTH, buffer, start, TYPE_WIDTH);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   BigDecimal containing decimal value.
+    */
+   public void set(int index, BigDecimal value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      DecimalUtility.checkPrecisionAndScale(value, precision, scale);
+      DecimalUtility.writeBigDecimalToArrowBuf(value, valueBuffer, index);
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableDecimalHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         valueBuffer.setBytes(index * TYPE_WIDTH, holder.buffer, holder.start, TYPE_WIDTH);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, DecimalHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      valueBuffer.setBytes(index * TYPE_WIDTH, holder.buffer, holder.start, TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #set(int, ArrowBuf)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param buffer  ArrowBuf containing decimal value.
+    */
+   public void setSafe(int index, ArrowBuf buffer) {
+      handleSafe(index);
+      set(index, buffer);
+   }
+
+   /**
+    * Same as {@link #set(int, int, ArrowBuf)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index    position of element
+    * @param start    start index of data in the buffer
+    * @param buffer   ArrowBuf containing decimal value.
+    */
+   public void setSafe(int index, int start, ArrowBuf buffer) {
+      handleSafe(index);
+      set(index, start, buffer);
+   }
+
+   /**
+    * Same as {@link #set(int, BigDecimal)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   BigDecimal containing decimal value.
+    */
+   public void setSafe(int index, BigDecimal value){
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableDecimalHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableDecimalHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, DecimalHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, DecimalHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of the value in the buffer
+    * @param buffer buffer containing the value to be stored in the vector
+    */
+   public void set(int index, int isSet, int start, ArrowBuf buffer) {
+      if (isSet > 0) {
+         set(index, start, buffer);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #setSafe(int, int, int, ArrowBuf)} except that it handles
+    * the case when the position of new value is beyond the current value
+    * capacity of the vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of the value in the buffer
+    * @param buffer buffer containing the value to be stored in the vector
+    */
+   public void setSafe(int index, int isSet, int start, ArrowBuf buffer) {
+      handleSafe(index);
+      set(index, isSet, start, buffer);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader                                *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value as array of bytes
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, byte[] value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      DecimalUtility.writeByteArrayToArrowBuf(value, buffer, index);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableDecimalVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableDecimalVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableDecimalVector(ref, allocator, NullableDecimalVector.this.precision,
+                 NullableDecimalVector.this.scale);
+      }
+
+      public TransferImpl(NullableDecimalVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableDecimalVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableDecimalVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
similarity index 51%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
index 26b19fa..5b28065 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.Float4ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.Float4Holder;
+import org.apache.arrow.vector.holders.NullableFloat4Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableFloat4Vector implements a fixed width vector (4 bytes) of
+ * float values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableFloat4Vector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableFloat4Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.FLOAT4.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableFloat4Vector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableFloat4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new Float4ReaderImpl(NullableFloat4Vector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.FLOAT4;
    }
 
 
@@ -79,11 +93,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public float get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getFloat(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +107,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableFloat4Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getFloat(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +122,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Float getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +130,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableFloat4Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableFloat4Vector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableFloat4Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +164,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, float value) {
+      valueBuffer.setFloat(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +174,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, float value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +187,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableFloat4Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +206,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, Float4Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, float)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, float value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableFloat4Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableFloat4Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, Float4Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, Float4Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +263,81 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, float value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, float)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, float value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, float value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setFloat(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static float get(final ArrowBuf buffer, final int index) {
+      return buffer.getFloat(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +348,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableFloat4Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableFloat4Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableFloat4Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableFloat4Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableFloat4Vector getTo(){
          return to;
       }
 
@@ -293,7 +398,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableFloat4Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
similarity index 50%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
index 26b19fa..624abf2 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.Float8ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.Float8Holder;
+import org.apache.arrow.vector.holders.NullableFloat8Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableFloat8Vector implements a fixed width vector (8 bytes) of
+ * double values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableFloat8Vector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableFloat8Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.FLOAT8.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableFloat8Vector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableFloat8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new Float8ReaderImpl(NullableFloat8Vector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.FLOAT8;
    }
 
 
@@ -79,11 +93,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public double get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getDouble(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +107,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableFloat8Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getDouble(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +122,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Double getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +130,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableFloat8Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableFloat8Vector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableFloat8Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +164,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, double value) {
+      valueBuffer.setDouble(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +174,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, double value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +187,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableFloat8Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +206,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, Float8Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, double)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, double value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableFloat8Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableFloat8Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, Float8Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, Float8Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +263,81 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, double value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, double)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, double value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, double value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setDouble(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static double get(final ArrowBuf buffer, final int index) {
+      return buffer.getDouble(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +348,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableFloat8Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableFloat8Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableFloat8Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableFloat8Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableFloat8Vector getTo(){
          return to;
       }
 
@@ -293,7 +398,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableFloat8Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
index 26b19fa..6311daf 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,10 +14,11 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.complex.impl.IntReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
@@ -27,39 +27,53 @@ import org.apache.arrow.vector.holders.NullableIntHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableIntVector implements a fixed width (4 bytes) vector of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
 public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
+   /**
+    * Instantiate a NullableIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
    public NullableIntVector(String name, BufferAllocator allocator) {
       this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
               allocator);
    }
 
+   /**
+    * Instantiate a NullableIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
    public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
       reader = new IntReaderImpl(NullableIntVector.this);
    }
 
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
-   }
-
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
       return Types.MinorType.INT;
@@ -116,12 +130,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
    public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
+   /**
+    * Same as {@link #copyFrom(int, int, NullableIntVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
    public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
@@ -234,17 +263,80 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, int value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setInt(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static int get(final ArrowBuf buffer, final int index) {
+      return buffer.getInt(index * TYPE_WIDTH);
    }
 
 
@@ -255,11 +347,23 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
       return new TransferImpl((NullableIntVector)to);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
new file mode 100644
index 0000000..174b722
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
@@ -0,0 +1,429 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.IntervalDayReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.IntervalDayHolder;
+import org.apache.arrow.vector.holders.NullableIntervalDayHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.Period;
+
+/**
+ * NullableIntervalDayVector implements a fixed width vector (8 bytes) of
+ * interval (days and milliseconds) values which could be null.
+ * A validity buffer (bit vector) is maintained to track which elements in the
+ * vector are null.
+ */
+public class NullableIntervalDayVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
+   private static final byte MILLISECOND_OFFSET = 4;
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableIntervalDayVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableIntervalDayVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.INTERVALDAY.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableIntervalDayVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableIntervalDayVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+      reader = new IntervalDayReaderImpl(NullableIntervalDayVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.INTERVALDAY;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public ArrowBuf get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         return null;
+      }
+      return valueBuffer.slice(index * TYPE_WIDTH, TYPE_WIDTH);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableIntervalDayHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      final int startIndex = index * TYPE_WIDTH;
+      holder.isSet = 1;
+      holder.days = valueBuffer.getInt(startIndex);
+      holder.milliseconds = valueBuffer.getInt(startIndex + MILLISECOND_OFFSET);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Period getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         final int startIndex = index * TYPE_WIDTH;
+         final int days = valueBuffer.getInt(startIndex);
+         final int milliseconds = valueBuffer.getInt(startIndex + MILLISECOND_OFFSET);
+         final Period p = new Period();
+         return p.plusDays(days).plusMillis(milliseconds);
+      }
+   }
+
+   /**
+    * Get the Interval value at a given index as a {@link StringBuilder} object
+    * @param index position of the element
+    * @return String Builder object with Interval value as
+    *         [days, hours, minutes, seconds, millis]
+    */
+   public StringBuilder getAsStringBuilder(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      }else{
+         return getAsStringBuilderHelper(index);
+      }
+   }
+
+   private StringBuilder getAsStringBuilderHelper(int index) {
+      final int startIndex = index * TYPE_WIDTH;
+
+      final int  days = valueBuffer.getInt(startIndex);
+      int millis = valueBuffer.getInt(startIndex + MILLISECOND_OFFSET);
+
+      final int hours = millis / (org.apache.arrow.vector.util.DateUtility.hoursToMillis);
+      millis = millis % (org.apache.arrow.vector.util.DateUtility.hoursToMillis);
+
+      final int minutes = millis / (org.apache.arrow.vector.util.DateUtility.minutesToMillis);
+      millis = millis % (org.apache.arrow.vector.util.DateUtility.minutesToMillis);
+
+      final int seconds = millis / (org.apache.arrow.vector.util.DateUtility.secondsToMillis);
+      millis = millis % (org.apache.arrow.vector.util.DateUtility.secondsToMillis);
+
+      final String dayString = (Math.abs(days) == 1) ? " day " : " days ";
+
+      return(new StringBuilder().
+              append(days).append(dayString).
+              append(hours).append(":").
+              append(minutes).append(":").
+              append(seconds).append(".").
+              append(millis));
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableIntervalDayVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, thisIndex);
+         from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, this.valueBuffer,
+                 thisIndex * TYPE_WIDTH, TYPE_WIDTH);
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableIntervalDayVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntervalDayVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, ArrowBuf value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      valueBuffer.setBytes(index * TYPE_WIDTH, value, 0, TYPE_WIDTH);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index          position of element
+    * @param days           days for the interval
+    * @param milliseconds   milliseconds for the interval
+    */
+   public void set(int index, int days, int milliseconds){
+      final int offsetIndex = index * TYPE_WIDTH;
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      valueBuffer.setInt(offsetIndex, days);
+      valueBuffer.setInt((offsetIndex + MILLISECOND_OFFSET), milliseconds);
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableIntervalDayHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         set(index, holder.days, holder.milliseconds);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, IntervalDayHolder holder){
+      set(index, holder.days, holder.milliseconds);
+   }
+
+   /**
+    * Same as {@link #set(int, ArrowBuf)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, ArrowBuf value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index          position of element
+    * @param days           days for the interval
+    * @param milliseconds   milliseconds for the interval
+    */
+   public void setSafe(int index, int days, int milliseconds) {
+      handleSafe(index);
+      set(index, days, milliseconds);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableIntervalDayHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableIntervalDayHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, IntervalDayHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, IntervalDayHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param days days component of interval
+    * @param milliseconds millisecond component of interval
+    */
+   public void set(int index, int isSet, int days, int milliseconds) {
+      if (isSet > 0) {
+         set(index, days, milliseconds);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param days days component of interval
+    * @param milliseconds millisecond component of interval
+    */
+   public void setSafe(int index, int isSet, int days, int milliseconds) {
+      handleSafe(index);
+      set(index, isSet, days, milliseconds);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableIntervalDayVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableIntervalDayVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableIntervalDayVector(ref, field.getFieldType(), allocator);
+      }
+
+      public TransferImpl(NullableIntervalDayVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableIntervalDayVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableIntervalDayVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
similarity index 55%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
index 26b19fa..303286d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.IntervalYearReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.IntervalYearHolder;
+import org.apache.arrow.vector.holders.NullableIntervalYearHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
+import org.joda.time.Period;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableIntervalYearVector implements a fixed width (4 bytes) vector of
+ * interval (years and months) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableIntervalYearVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableIntervalYearVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.INTERVALYEAR.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableIntervalYearVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableIntervalYearVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new IntervalYearReaderImpl(NullableIntervalYearVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.INTERVALYEAR;
    }
 
 
@@ -93,7 +107,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableIntervalYearHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
@@ -108,21 +122,68 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Period getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
-         return get(index);
+         final int interval = get(index);
+         final int years  = (interval / org.apache.arrow.vector.util.DateUtility.yearsToMonths);
+         final int months = (interval % org.apache.arrow.vector.util.DateUtility.yearsToMonths);
+         final Period p = new Period();
+         return p.plusYears(years).plusMonths(months);
+      }
+   }
+
+   /**
+    * Get the Interval value at a given index as a {@link StringBuilder} object
+    * @param index position of the element
+    * @return String Builder object with Interval value as
+    *         [years, months]
+    */
+   public StringBuilder getAsStringBuilder(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      }else{
+         return getAsStringBuilderHelper(index);
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   private StringBuilder getAsStringBuilderHelper(int index) {
+      int value  = valueBuffer.getInt(index * TYPE_WIDTH);
+
+      final int years  = (value / org.apache.arrow.vector.util.DateUtility.yearsToMonths);
+      final int months = (value % org.apache.arrow.vector.util.DateUtility.yearsToMonths);
+
+      final String yearString = (Math.abs(years) == 1) ? " year " : " years ";
+      final String monthString = (Math.abs(months) == 1) ? " month " : " months ";
+
+      return(new StringBuilder().
+              append(years).append(yearString).
+              append(months).append(monthString));
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableIntervalYearVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableIntervalYearVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntervalYearVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -158,7 +219,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableIntervalYearHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +238,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, IntervalYearHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +257,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableIntervalYearHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableIntervalYearHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, IntervalYearHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, IntervalYearHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +295,32 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
@@ -255,29 +331,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableIntervalYearVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableIntervalYearVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableIntervalYearVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableIntervalYearVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableIntervalYearVector getTo(){
          return to;
       }
 
@@ -293,7 +381,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableIntervalYearVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
new file mode 100644
index 0000000..c45a8d5
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
@@ -0,0 +1,431 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.SmallIntReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.SmallIntHolder;
+import org.apache.arrow.vector.holders.NullableSmallIntHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableSmallIntVector implements a fixed width (2 bytes) vector of
+ * short values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 2;
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableSmallIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableSmallIntVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.SMALLINT.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableSmallIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableSmallIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+      reader = new SmallIntReaderImpl(NullableSmallIntVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.SMALLINT;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public short get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return valueBuffer.getShort(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableSmallIntHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getShort(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Short getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableSmallIntVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         set(thisIndex, from.get(fromIndex));
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableSmallIntVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableSmallIntVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   private void setValue(int index, int value) {
+      valueBuffer.setShort(index * TYPE_WIDTH, value);
+   }
+
+   private void setValue(int index, short value) {
+      valueBuffer.setShort(index * TYPE_WIDTH, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, int value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, short value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableSmallIntHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, SmallIntHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, int)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, int value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, short)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, short value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableSmallIntHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableSmallIntHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, SmallIntHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, SmallIntHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, short value) {
+      if (isSet > 0) {
+         set(index, value);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, short)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, short value) {
+      handleSafe(index);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, short value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setShort(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static short get(final ArrowBuf buffer, final int index) {
+      return buffer.getShort(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableSmallIntVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableSmallIntVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableSmallIntVector(ref, field.getFieldType(), allocator);
+      }
+
+      public TransferImpl(NullableSmallIntVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableSmallIntVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableSmallIntVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
similarity index 50%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
index 26b19fa..454a4ac 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,71 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.TimeMicroReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.TimeMicroHolder;
+import org.apache.arrow.vector.holders.NullableTimeMicroHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableTimeMicroVector implements a fixed width vector (8 bytes) of
+ * time (microsecond resolution) values which could be null.
+ * A validity buffer (bit vector) is maintained to track which elements in the
+ * vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableTimeMicroVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeMicroVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMEMICRO.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableTimeMicroVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeMicroVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new TimeMicroReaderImpl(NullableTimeMicroVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.TIMEMICRO;
    }
 
 
@@ -72,18 +88,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *                                                                *
     ******************************************************************/
 
-
    /**
     * Get the element at the given index from the vector.
     *
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public long get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +108,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableTimeMicroHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +123,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Long getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +131,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTimeMicroVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableTimeMicroVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTimeMicroVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +165,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +175,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, long value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +188,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableTimeMicroHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +207,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, TimeMicroHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, long)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, long value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableTimeMicroHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableTimeMicroHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, TimeMicroHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, TimeMicroHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +264,80 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, long value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, long)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, long value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, long value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setLong(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static long get(final ArrowBuf buffer, int index) {
+      return buffer.getLong(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +348,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableTimeMicroVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableTimeMicroVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableTimeMicroVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableTimeMicroVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableTimeMicroVector getTo(){
          return to;
       }
 
@@ -293,7 +398,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableTimeMicroVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
similarity index 53%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
index 26b19fa..8540d16 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,71 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.TimeMilliReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.TimeMilliHolder;
+import org.apache.arrow.vector.holders.NullableTimeMilliHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableTimeMilliVector implements a fixed width (4 bytes) vector of
+ * time (millisecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableTimeMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeMilliVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMEMILLI.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableTimeMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new TimeMilliReaderImpl(NullableTimeMilliVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.TIMEMILLI;
    }
 
 
@@ -72,7 +88,6 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *                                                                *
     ******************************************************************/
 
-
    /**
     * Get the element at the given index from the vector.
     *
@@ -93,7 +108,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableTimeMilliHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
@@ -108,21 +123,37 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public LocalDateTime getObject(int index) {
       if (isSet(index) == 0) {
          return null;
-      } else {
-         return get(index);
       }
+      org.joda.time.LocalDateTime ldt = new org.joda.time.LocalDateTime(get(index),
+              org.joda.time.DateTimeZone.UTC);
+      return ldt;
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTimeMilliVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableTimeMilliVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTimeMilliVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -158,7 +189,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableTimeMilliHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +208,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, TimeMilliHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +227,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableTimeMilliHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableTimeMilliHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, TimeMilliHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, TimeMilliHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,50 +265,125 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
    /******************************************************************
     *                                                                *
-    *                      vector transfer                           *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
     *                                                                *
     ******************************************************************/
 
 
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, int value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setInt(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static int get(final ArrowBuf buffer, final int index) {
+      return buffer.getInt(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableTimeMilliVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableTimeMilliVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableTimeMilliVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableTimeMilliVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableTimeMilliVector getTo(){
          return to;
       }
 
@@ -293,7 +399,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableTimeMilliVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
similarity index 50%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
index 26b19fa..015226d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.TimeNanoReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.TimeNanoHolder;
+import org.apache.arrow.vector.holders.NullableTimeNanoHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableTimeNanoVector implements a fixed width vector (8 bytes) of
+ * time (nanosecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableTimeNanoVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeNanoVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMENANO.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableTimeNanoVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeNanoVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new TimeNanoReaderImpl(NullableTimeNanoVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.TIMENANO;
    }
 
 
@@ -79,11 +93,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public long get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +107,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableTimeNanoHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +122,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Long getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +130,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTimeNanoVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableTimeNanoVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTimeNanoVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +164,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +174,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, long value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +187,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableTimeNanoHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +206,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, TimeNanoHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, long)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, long value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableTimeNanoHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableTimeNanoHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, TimeNanoHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, TimeNanoHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,50 +263,124 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, long value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, long)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, long value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
    /******************************************************************
     *                                                                *
-    *                      vector transfer                           *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
     *                                                                *
     ******************************************************************/
 
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, long value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setLong(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static long get(final ArrowBuf buffer,final int index) {
+      return buffer.getLong(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableTimeNanoVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableTimeNanoVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableTimeNanoVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableTimeNanoVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableTimeNanoVector getTo(){
          return to;
       }
 
@@ -293,7 +396,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableTimeNanoVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
similarity index 54%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
index 26b19fa..2b2375e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,54 +14,69 @@
  * 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.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.TimeSecReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.TimeSecHolder;
+import org.apache.arrow.vector.holders.NullableTimeSecHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableTimeSecVector implements a fixed width (4 bytes) vector of
+ * time (seconds resolution) values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   /**
+    * Instantiate a NullableTimeSecVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeSecVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMESEC.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableTimeSecVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeSecVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new TimeSecReaderImpl(NullableTimeSecVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.TIMESEC;
    }
 
 
@@ -93,7 +107,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableTimeSecHolder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
@@ -116,13 +130,28 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTimeSecVector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableTimeSecVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTimeSecVector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -158,7 +187,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableTimeSecHolder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +206,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, TimeSecHolder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +225,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableTimeSecHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableTimeSecHolder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, TimeSecHolder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, TimeSecHolder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +263,80 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   /**
+    * Same as {@link #set(int, int, int)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
+   }
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, int value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setInt(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static int get(final ArrowBuf buffer, final int index) {
+      return buffer.getInt(index * TYPE_WIDTH);
    }
 
 
@@ -255,29 +347,41 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableTimeSecVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableTimeSecVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableTimeSecVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableTimeSecVector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableTimeSecVector getTo(){
          return to;
       }
 
@@ -293,7 +397,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableTimeSecVector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
new file mode 100644
index 0000000..0d25eff
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
@@ -0,0 +1,219 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampMicroTZReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampMicroTZHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampMicroTZHolder;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTimeStampMicroTZVector implements a fixed width vector (8 bytes) of
+ * timestamp (microsecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampMicroTZVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+   private final String timeZone;
+
+   /**
+    * Instantiate a NullableTimeStampMicroTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMicroTZVector(String name, BufferAllocator allocator, String timeZone) {
+      this(name, FieldType.nullable(new org.apache.arrow.vector.types.pojo.ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZone)),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampMicroTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMicroTZVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      org.apache.arrow.vector.types.pojo.ArrowType.Timestamp arrowType = (org.apache.arrow.vector.types.pojo.ArrowType.Timestamp)fieldType.getType();
+      timeZone = arrowType.getTimezone();
+      reader = new TimeStampMicroTZReaderImpl(NullableTimeStampMicroTZVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPMICROTZ;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampMicroTZHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Long getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampMicroTZHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampMicroTZHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampMicroTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampMicroTZHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampMicroTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampMicroTZHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampMicroTZVector to = new NullableTimeStampMicroTZVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampMicroTZVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
new file mode 100644
index 0000000..41f7a11
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
@@ -0,0 +1,220 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampMicroReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampMicroHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampMicroHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
+
+/**
+ * NullableTimeStampMicroVector implements a fixed width vector (8 bytes) of
+ * timestamp (microsecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampMicroVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableTimeStampMicroVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMicroVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMESTAMPMICRO.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampMicroVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMicroVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      reader = new TimeStampMicroReaderImpl(NullableTimeStampMicroVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPMICRO;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampMicroHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public LocalDateTime getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         /* value is truncated when converting microseconds to milliseconds in order to use DateTime type */
+         final long micros = get(index);
+         final long millis = java.util.concurrent.TimeUnit.MICROSECONDS.toMillis(micros);
+         final org.joda.time.LocalDateTime localDateTime = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
+         return localDateTime;
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampMicroHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampMicroHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampMicroHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampMicroHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampMicroHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampMicroHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampMicroVector to = new NullableTimeStampMicroVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampMicroVector )to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
new file mode 100644
index 0000000..6376b08
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
@@ -0,0 +1,218 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampMilliTZReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampMilliTZHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampMilliTZHolder;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTimeStampMilliTZVector implements a fixed width vector (8 bytes) of
+ * timestamp (millisecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampMilliTZVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+   private final String timeZone;
+
+   /**
+    * Instantiate a NullableTimeStampMilliTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMilliTZVector(String name, BufferAllocator allocator, String timeZone) {
+      this(name, FieldType.nullable(new org.apache.arrow.vector.types.pojo.ArrowType.Timestamp(TimeUnit.MILLISECOND, timeZone)),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampMilliTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMilliTZVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      org.apache.arrow.vector.types.pojo.ArrowType.Timestamp arrowType = (org.apache.arrow.vector.types.pojo.ArrowType.Timestamp)fieldType.getType();
+      timeZone = arrowType.getTimezone();
+      reader = new TimeStampMilliTZReaderImpl(NullableTimeStampMilliTZVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPMILLITZ;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampMilliTZHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Long getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampMilliTZHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampMilliTZHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampMilliTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampMilliTZHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampMilliTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampMilliTZHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampMilliTZVector to = new NullableTimeStampMilliTZVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampMilliTZVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
new file mode 100644
index 0000000..ff581d2
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
@@ -0,0 +1,218 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampMilliReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampMilliHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampMilliHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
+
+/**
+ * NullableTimeStampMilliVector implements a fixed width vector (8 bytes) of
+ * timestamp (millisecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampMilliVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableTimeStampMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMilliVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMESTAMPMILLI.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampMilliVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      reader = new TimeStampMilliReaderImpl(NullableTimeStampMilliVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPMILLI;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampMilliHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public LocalDateTime getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         final long millis = get(index);
+         final org.joda.time.LocalDateTime localDateTime = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
+         return localDateTime;
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampMilliHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampMilliHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampMilliHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampMilliHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampMilliHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampMilliHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampMilliVector to = new NullableTimeStampMilliVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampMilliVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
new file mode 100644
index 0000000..6c4445d
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
@@ -0,0 +1,219 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampNanoTZReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampNanoTZHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampNanoTZHolder;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTimeStampNanoTZVector implements a fixed width vector (8 bytes) of
+ * timestamp (nanosecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampNanoTZVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+   private final String timeZone;
+
+   /**
+    * Instantiate a NullableTimeStampNanoTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampNanoTZVector(String name, BufferAllocator allocator, String timeZone) {
+      this(name, FieldType.nullable(new org.apache.arrow.vector.types.pojo.ArrowType.Timestamp(TimeUnit.NANOSECOND, timeZone)),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampNanoTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampNanoTZVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      org.apache.arrow.vector.types.pojo.ArrowType.Timestamp arrowType = (org.apache.arrow.vector.types.pojo.ArrowType.Timestamp)fieldType.getType();
+      timeZone = arrowType.getTimezone();
+      reader = new TimeStampNanoTZReaderImpl(NullableTimeStampNanoTZVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPNANOTZ;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampNanoTZHolder holder) {
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Long getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampNanoTZHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampNanoTZHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampNanoTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampNanoTZHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampNanoTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampNanoTZHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampNanoTZVector to = new NullableTimeStampNanoTZVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampNanoTZVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
new file mode 100644
index 0000000..096913e
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
@@ -0,0 +1,219 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampNanoReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampNanoHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampNanoHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
+
+/**
+ * NullableTimeStampNanoVector implements a fixed width vector (8 bytes) of
+ * timestamp (nanosecond resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampNanoVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableTimeStampNanoVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampNanoVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMESTAMPNANO.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampNanoVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampNanoVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      reader = new TimeStampNanoReaderImpl(NullableTimeStampNanoVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPNANO;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampNanoHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public LocalDateTime getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         final long nanos = get(index);
+         final long millis = java.util.concurrent.TimeUnit.NANOSECONDS.toMillis(nanos);
+         final org.joda.time.LocalDateTime localDateTime = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
+         return localDateTime;
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampNanoHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampNanoHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampNanoHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampNanoHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampNanoHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampNanoHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampNanoVector to = new NullableTimeStampNanoVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampNanoVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
new file mode 100644
index 0000000..8e5ba72
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
@@ -0,0 +1,217 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampSecTZReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampSecTZHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampSecTZHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTimeStampSecTZVector implements a fixed width vector (8 bytes) of
+ * timestamp (seconds resolution) values which could be null. A validity buffer
+ * (bit vector) is maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampSecTZVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+   private final String timeZone;
+
+   /**
+    * Instantiate a NullableTimeStampSecTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampSecTZVector(String name, BufferAllocator allocator, String timeZone) {
+      this(name, FieldType.nullable(new org.apache.arrow.vector.types.pojo.ArrowType.Timestamp(org.apache.arrow.vector.types.TimeUnit.SECOND, timeZone)),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampSecTZVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampSecTZVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      org.apache.arrow.vector.types.pojo.ArrowType.Timestamp arrowType = (org.apache.arrow.vector.types.pojo.ArrowType.Timestamp)fieldType.getType();
+      timeZone = arrowType.getTimezone();
+      reader = new TimeStampSecTZReaderImpl(NullableTimeStampSecTZVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPSECTZ;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampSecTZHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Long getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampSecTZHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampSecTZHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampSecTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampSecTZHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampSecTZHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampSecTZHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampSecTZVector to = new NullableTimeStampSecTZVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampSecTZVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
new file mode 100644
index 0000000..bc800e0
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
@@ -0,0 +1,219 @@
+/**
+ * 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.arrow.vector;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TimeStampSecReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TimeStampSecHolder;
+import org.apache.arrow.vector.holders.NullableTimeStampSecHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+import org.joda.time.LocalDateTime;
+
+/**
+ * NullableTimeStampSecVector implements a fixed width vector (8 bytes) of
+ * timestamp (seconds resolution) values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public class NullableTimeStampSecVector extends NullableTimeStampVector {
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableTimeStampSecVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampSecVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TIMESTAMPSEC.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTimeStampSecVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampSecVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, fieldType, allocator);
+      reader = new TimeStampSecReaderImpl(NullableTimeStampSecVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader() {
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TIMESTAMPSEC;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTimeStampSecHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public LocalDateTime getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         final long secs = get(index);
+         final long millis = java.util.concurrent.TimeUnit.SECONDS.toMillis(secs);
+         final org.joda.time.LocalDateTime localDateTime = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
+         return localDateTime;
+      }
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTimeStampSecHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TimeStampSecHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTimeStampSecHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTimeStampSecHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TimeStampSecHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TimeStampSecHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      NullableTimeStampSecVector to = new NullableTimeStampSecVector(ref,
+              field.getFieldType(), allocator);
+      return new TransferImpl(to);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTimeStampSecVector)to);
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
new file mode 100644
index 0000000..b2a58bd
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
@@ -0,0 +1,253 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTimeStampVector is an abstract interface for fixed width vector (8 bytes)
+ * of timestamp values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public abstract class NullableTimeStampVector extends BaseNullableFixedWidthVector {
+   protected static final byte TYPE_WIDTH = 8;
+
+   /**
+    * Instantiate a NullableTimeStampVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTimeStampVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public long get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return valueBuffer.getLong(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTimeStampVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         set(thisIndex, from.get(fromIndex));
+      }
+   }
+
+   /**
+    * Same as {@link #copyFromSafe(int, int, NullableTimeStampVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTimeStampVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   protected void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, long value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, long)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, long value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, long value) {
+      if (isSet > 0) {
+         set(index, value);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, long)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, long value) {
+      handleSafe(index);
+      set(index, isSet, value);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and                            *
+    *          JsonFileWriter                                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, long value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setLong(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static long get(final ArrowBuf buffer, final int index) {
+      return buffer.getLong(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   public class TransferImpl implements TransferPair {
+      NullableTimeStampVector to;
+
+      public TransferImpl(NullableTimeStampVector to) {
+         this.to = to;
+      }
+
+      @Override
+      public NullableTimeStampVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer() {
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableTimeStampVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
new file mode 100644
index 0000000..1510053
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
@@ -0,0 +1,432 @@
+/**
+ * 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.arrow.vector;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.impl.TinyIntReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.TinyIntHolder;
+import org.apache.arrow.vector.holders.NullableTinyIntHolder;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.TransferPair;
+
+/**
+ * NullableTinyIntVector implements a fixed width (1 bytes) vector of
+ * byte values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
+ */
+public class NullableTinyIntVector extends BaseNullableFixedWidthVector {
+   public static final byte TYPE_WIDTH = 1;
+   private final FieldReader reader;
+
+   /**
+    * Instantiate a NullableTinyIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTinyIntVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.TINYINT.getType()),
+              allocator);
+   }
+
+   /**
+    * Instantiate a NullableTinyIntVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableTinyIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+      super(name, allocator, fieldType, TYPE_WIDTH);
+      reader = new TinyIntReaderImpl(NullableTinyIntVector.this);
+   }
+
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
+   @Override
+   public FieldReader getReader(){
+      return reader;
+   }
+
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
+   @Override
+   public Types.MinorType getMinorType() {
+      return Types.MinorType.TINYINT;
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value retrieval methods                        *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Get the element at the given index from the vector.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public byte get(int index) throws IllegalStateException {
+      if(isSet(index) == 0) {
+         throw new IllegalStateException("Value at index is null");
+      }
+      return valueBuffer.getByte(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Get the element at the given index from the vector and
+    * sets the state in holder. If element at given index
+    * is null, holder.isSet will be zero.
+    *
+    * @param index   position of element
+    */
+   public void get(int index, NullableTinyIntHolder holder){
+      if(isSet(index) == 0) {
+         holder.isSet = 0;
+         return;
+      }
+      holder.isSet = 1;
+      holder.value = valueBuffer.getByte(index * TYPE_WIDTH);
+   }
+
+   /**
+    * Same as {@link #get(int)}.
+    *
+    * @param index   position of element
+    * @return element at given index
+    */
+   public Byte getObject(int index) {
+      if (isSet(index) == 0) {
+         return null;
+      } else {
+         return get(index);
+      }
+   }
+
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableTinyIntVector from) {
+      if (from.isSet(fromIndex) != 0) {
+         set(thisIndex, from.get(fromIndex));
+      }
+   }
+
+   /**
+    * Same as {@link #copyFrom(int, int, NullableTinyIntVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableTinyIntVector from) {
+      handleSafe(thisIndex);
+      copyFrom(fromIndex, thisIndex, from);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *          vector value setter methods                           *
+    *                                                                *
+    ******************************************************************/
+
+   private void setValue(int index, int value) {
+      valueBuffer.setByte(index * TYPE_WIDTH, value);
+   }
+
+   private void setValue(int index, byte value) {
+      valueBuffer.setByte(index * TYPE_WIDTH, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, int value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, byte value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    * If the value in holder is not indicated as set, element in the
+    * at the given index will be null.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void set(int index, NullableTinyIntHolder holder) throws IllegalArgumentException {
+      if(holder.isSet < 0) {
+         throw new IllegalArgumentException();
+      }
+      else if(holder.isSet > 0) {
+         BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+         setValue(index, holder.value);
+      }
+      else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Set the element at the given index to the value set in data holder.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void set(int index, TinyIntHolder holder){
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, holder.value);
+   }
+
+   /**
+    * Same as {@link #set(int, int)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, int value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, byte)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, byte value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableTinyIntHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  nullable data holder for value of element
+    */
+   public void setSafe(int index, NullableTinyIntHolder holder) throws IllegalArgumentException {
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Same as {@link #set(int, TinyIntHolder)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param holder  data holder for value of element
+    */
+   public void setSafe(int index, TinyIntHolder holder){
+      handleSafe(index);
+      set(index, holder);
+   }
+
+   /**
+    * Set the element at the given index to null.
+    *
+    * @param index   position of element
+    */
+   public void setNull(int index){
+      handleSafe(index);
+      /* not really needed to set the bit to 0 as long as
+       * the buffer always starts from 0.
+       */
+      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+   }
+
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void set(int index, int isSet, byte value) {
+      if (isSet > 0) {
+         set(index, value);
+      } else {
+         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+      }
+   }
+
+   /**
+    * Same as {@link #set(int, int, byte)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param value element value
+    */
+   public void setSafe(int index, int isSet, byte value) {
+      handleSafe(index);
+      set(index, isSet, value);
+   }
+
+
+
+   /******************************************************************
+    *                                                                *
+    *          helper routines currently                             *
+    *          used in JsonFileReader and JsonFileWriter             *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Given a data buffer, this method sets the element value at a particular
+    * position. Reallocates the buffer if needed.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param allocator allocator
+    * @param valueCount number of elements in the vector
+    * @param index position of the new element
+    * @param value element value
+    * @return data buffer
+    */
+   public static ArrowBuf set(ArrowBuf buffer, BufferAllocator allocator,
+                              int valueCount, int index, byte value) {
+      if (buffer == null) {
+         buffer = allocator.buffer(valueCount * TYPE_WIDTH);
+      }
+      buffer.setByte(index * TYPE_WIDTH, value);
+      if (index == (valueCount - 1)) {
+         buffer.writerIndex(valueCount * TYPE_WIDTH);
+      }
+
+      return buffer;
+   }
+
+   /**
+    * Given a data buffer, get the value stored at a particular position
+    * in the vector.
+    *
+    * This method should not be used externally.
+    *
+    * @param buffer data buffer
+    * @param index position of the element.
+    * @return value stored at the index.
+    */
+   public static byte get(final ArrowBuf buffer, final int index) {
+      return buffer.getByte(index * TYPE_WIDTH);
+   }
+
+
+   /******************************************************************
+    *                                                                *
+    *                      vector transfer                           *
+    *                                                                *
+    ******************************************************************/
+
+
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+      return new TransferImpl(ref, allocator);
+   }
+
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return new TransferImpl((NullableTinyIntVector)to);
+   }
+
+   private class TransferImpl implements TransferPair {
+      NullableTinyIntVector to;
+
+      public TransferImpl(String ref, BufferAllocator allocator){
+         to = new NullableTinyIntVector(ref, field.getFieldType(), allocator);
+      }
+
+      public TransferImpl(NullableTinyIntVector to){
+         this.to = to;
+      }
+
+      @Override
+      public NullableTinyIntVector getTo(){
+         return to;
+      }
+
+      @Override
+      public void transfer(){
+         transferTo(to);
+      }
+
+      @Override
+      public void splitAndTransfer(int startIndex, int length) {
+         splitAndTransferTo(startIndex, length, to);
+      }
+
+      @Override
+      public void copyValueSafe(int fromIndex, int toIndex) {
+         to.copyFromSafe(fromIndex, toIndex, NullableTinyIntVector.this);
+      }
+   }
+}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
similarity index 70%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
index 26b19fa..4d20f80 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,44 +14,36 @@
  * 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.arrow.vector;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.UInt1ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.UInt1Holder;
+import org.apache.arrow.vector.holders.NullableUInt1Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
-import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableUInt1Vector implements a fixed width (1 bytes) vector of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 1;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   public NullableUInt1Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT1.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   public NullableUInt1Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new UInt1ReaderImpl(NullableUInt1Vector.this);
    }
 
    @Override
@@ -62,7 +53,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.UINT1;
    }
 
 
@@ -79,11 +70,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public byte get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getByte(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +84,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableUInt1Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getByte(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +99,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Byte getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +107,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFrom(int fromIndex, int thisIndex, NullableUInt1Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt1Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -136,7 +127,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
 
    private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+      valueBuffer.setByte(index * TYPE_WIDTH, value);
+   }
+
+   private void setValue(int index, byte value) {
+      valueBuffer.setByte(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -151,6 +146,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, byte value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
     * Set the element at the given index to the value set in data holder.
     * If the value in holder is not indicated as set, element in the
     * at the given index will be null.
@@ -158,7 +164,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableUInt1Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +183,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, UInt1Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +202,40 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, byte)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, byte value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableUInt1Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableUInt1Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, UInt1Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, UInt1Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +253,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   public void set(int index, int isSet, byte value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   public void setSafe(int index, int isSet, byte value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
@@ -262,22 +281,22 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableUInt1Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableUInt1Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableUInt1Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableUInt1Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableUInt1Vector getTo(){
          return to;
       }
 
@@ -293,7 +312,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableUInt1Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
similarity index 70%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
index 26b19fa..b140b9f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,44 +14,37 @@
  * 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.arrow.vector;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.UInt2ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.UInt2Holder;
+import org.apache.arrow.vector.holders.NullableUInt2Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableUInt2Vector implements a fixed width (2 bytes) vector of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 2;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   public NullableUInt2Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT2.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   public NullableUInt2Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new UInt2ReaderImpl(NullableUInt2Vector.this);
    }
 
    @Override
@@ -62,7 +54,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.UINT2;
    }
 
 
@@ -79,11 +71,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public char get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getChar(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +85,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableUInt2Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getChar(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +100,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Character getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +108,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFrom(int fromIndex, int thisIndex, NullableUInt2Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt2Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -136,7 +128,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
 
    private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+      valueBuffer.setChar(index * TYPE_WIDTH, value);
+   }
+
+   private void setValue(int index, char value) {
+      valueBuffer.setChar(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -151,6 +147,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
+    * Set the element at the given index to the given value.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void set(int index, char value) {
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      setValue(index, value);
+   }
+
+   /**
     * Set the element at the given index to the value set in data holder.
     * If the value in holder is not indicated as set, element in the
     * at the given index will be null.
@@ -158,7 +165,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableUInt2Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +184,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, UInt2Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +203,40 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, char)} except that it handles the
+    * case when index is greater than or equal to existing
+    * value capacity {@link #getValueCapacity()}.
+    *
+    * @param index   position of element
+    * @param value   value of element
+    */
+   public void setSafe(int index, char value) {
+      handleSafe(index);
+      set(index, value);
+   }
+
+   /**
+    * Same as {@link #set(int, NullableUInt2Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableUInt2Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, UInt2Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, UInt2Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +254,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   public void set(int index, int isSet, char value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   public void setSafe(int index, int isSet, char value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
@@ -262,22 +282,22 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableUInt2Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableUInt2Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableUInt2Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableUInt2Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableUInt2Vector getTo(){
          return to;
       }
 
@@ -293,7 +313,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableUInt2Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
similarity index 77%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
index 26b19fa..fc8cb63 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,44 +14,37 @@
  * 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.arrow.vector;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.UInt4ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.UInt4Holder;
+import org.apache.arrow.vector.holders.NullableUInt4Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableUInt4Vector implements a fixed width (4 bytes) vector of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
    private static final byte TYPE_WIDTH = 4;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   public NullableUInt4Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT4.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   public NullableUInt4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new UInt4ReaderImpl(NullableUInt4Vector.this);
    }
 
    @Override
@@ -62,7 +54,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.UINT4;
    }
 
 
@@ -93,7 +85,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableUInt4Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
@@ -116,13 +108,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFrom(int fromIndex, int thisIndex, NullableUInt4Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt4Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -158,7 +150,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableUInt4Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,7 +169,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, UInt4Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
@@ -196,27 +188,27 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableUInt4Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableUInt4Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, UInt4Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, UInt4Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +226,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   public void set(int index, int isSet, int value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   public void setSafe(int index, int isSet, int value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
@@ -262,22 +254,22 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableUInt4Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableUInt4Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableUInt4Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableUInt4Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableUInt4Vector getTo(){
          return to;
       }
 
@@ -293,7 +285,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableUInt4Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
similarity index 71%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
index 26b19fa..2110cf0 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,44 +14,37 @@
  * 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.arrow.vector;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.IntReaderImpl;
+import org.apache.arrow.vector.complex.impl.UInt8ReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.IntHolder;
-import org.apache.arrow.vector.holders.NullableIntHolder;
+import org.apache.arrow.vector.holders.UInt8Holder;
+import org.apache.arrow.vector.holders.NullableUInt8Holder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableIntVector implements a fixed width vector of values which could
- * be null. A validity buffer (bit vector) is maintained to track which
- * elements in the vector are null.
+ * NullableUInt8Vector implements a fixed width vector (8 bytes) of
+ * integer values which could be null. A validity buffer (bit vector) is
+ * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
-   private static final byte TYPE_WIDTH = 4;
+public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
+   private static final byte TYPE_WIDTH = 8;
    private final FieldReader reader;
 
-   public NullableIntVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.INT.getType()),
+   public NullableUInt8Vector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.UINT8.getType()),
               allocator);
    }
 
-   public NullableIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   public NullableUInt8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new IntReaderImpl(NullableIntVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new UInt8ReaderImpl(NullableUInt8Vector.this);
    }
 
    @Override
@@ -62,7 +54,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.INT;
+      return Types.MinorType.UINT8;
    }
 
 
@@ -79,11 +71,11 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public int get(int index) throws IllegalStateException {
+   public long get(int index) throws IllegalStateException {
       if(isSet(index) == 0) {
          throw new IllegalStateException("Value at index is null");
       }
-      return valueBuffer.getInt(index * TYPE_WIDTH);
+      return valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -93,13 +85,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     *
     * @param index   position of element
     */
-   public void get(int index, NullableIntHolder holder){
+   public void get(int index, NullableUInt8Holder holder){
       if(isSet(index) == 0) {
          holder.isSet = 0;
          return;
       }
       holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+      holder.value = valueBuffer.getLong(index * TYPE_WIDTH);
    }
 
    /**
@@ -108,7 +100,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @return element at given index
     */
-   public Integer getObject(int index) {
+   public Long getObject(int index) {
       if (isSet(index) == 0) {
          return null;
       } else {
@@ -116,13 +108,13 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       }
    }
 
-   public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFrom(int fromIndex, int thisIndex, NullableUInt8Vector from) {
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt8Vector from) {
       handleSafe(thisIndex);
       copyFrom(fromIndex, thisIndex, from);
    }
@@ -135,8 +127,8 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     ******************************************************************/
 
 
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(index * TYPE_WIDTH, value);
+   private void setValue(int index, long value) {
+      valueBuffer.setLong(index * TYPE_WIDTH, value);
    }
 
    /**
@@ -145,7 +137,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param value   value of element
     */
-   public void set(int index, int value) {
+   public void set(int index, long value) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, value);
    }
@@ -158,7 +150,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void set(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void set(int index, NullableUInt8Holder holder) throws IllegalArgumentException {
       if(holder.isSet < 0) {
          throw new IllegalArgumentException();
       }
@@ -177,46 +169,46 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void set(int index, IntHolder holder){
+   public void set(int index, UInt8Holder holder){
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
    }
 
    /**
-    * Same as {@link #set(int, int)} except that it handles the
+    * Same as {@link #set(int, long)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param value   value of element
     */
-   public void setSafe(int index, int value) {
+   public void setSafe(int index, long value) {
       handleSafe(index);
       set(index, value);
    }
 
    /**
-    * Same as {@link #set(int, NullableIntHolder)} except that it handles the
+    * Same as {@link #set(int, NullableUInt8Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  nullable data holder for value of element
     */
-   public void setSafe(int index, NullableIntHolder holder) throws IllegalArgumentException {
+   public void setSafe(int index, NullableUInt8Holder holder) throws IllegalArgumentException {
       handleSafe(index);
       set(index, holder);
    }
 
    /**
-    * Same as {@link #set(int, IntHolder)} except that it handles the
+    * Same as {@link #set(int, UInt8Holder)} except that it handles the
     * case when index is greater than or equal to existing
     * value capacity {@link #getValueCapacity()}.
     *
     * @param index   position of element
     * @param holder  data holder for value of element
     */
-   public void setSafe(int index, IntHolder holder){
+   public void setSafe(int index, UInt8Holder holder){
       handleSafe(index);
       set(index, holder);
    }
@@ -234,17 +226,17 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int valueField ) {
+   public void set(int index, int isSet, long value) {
       if (isSet > 0) {
-         set(index, valueField);
+         set(index, value);
       } else {
          BitVectorHelper.setValidityBit(validityBuffer, index, 0);
       }
    }
 
-   public void setSafe(int index, int isSet, int valueField ) {
+   public void setSafe(int index, int isSet, long value) {
       handleSafe(index);
-      set(index, isSet, valueField);
+      set(index, isSet, value);
    }
 
 
@@ -262,22 +254,22 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableIntVector)to);
+      return new TransferImpl((NullableUInt8Vector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableIntVector to;
+      NullableUInt8Vector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
+         to = new NullableUInt8Vector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableIntVector to){
+      public TransferImpl(NullableUInt8Vector to){
          this.to = to;
       }
 
       @Override
-      public NullableIntVector getTo(){
+      public NullableUInt8Vector getTo(){
          return to;
       }
 
@@ -293,7 +285,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableUInt8Vector.this);
       }
    }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
similarity index 73%
copy from java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
index b902154..330e067 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,50 +14,69 @@
  * 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.arrow.vector;
 
 import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.VarCharReaderImpl;
+import org.apache.arrow.vector.complex.impl.VarBinaryReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.VarCharHolder;
-import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.arrow.vector.holders.VarBinaryHolder;
+import org.apache.arrow.vector.holders.NullableVarBinaryHolder;
 import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.util.Text;
 import org.apache.arrow.vector.util.TransferPair;
 
 import java.nio.ByteBuffer;
 
-public class NullableVarCharVector extends BaseNullableVariableWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
+/**
+ * NullableVarBinaryVector implements a variable width vector of binary
+ * values which could be NULL. A validity buffer (bit vector) is maintained
+ * to track which elements in the vector are null.
+ */
+public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
    private final FieldReader reader;
 
-   public NullableVarCharVector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.VARCHAR.getType()), allocator);
+   /**
+    * Instantiate a NullableVarBinaryVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableVarBinaryVector(String name, BufferAllocator allocator) {
+      this(name, FieldType.nullable(Types.MinorType.VARBINARY.getType()), allocator);
    }
 
-   public NullableVarCharVector(String name, FieldType fieldType, BufferAllocator allocator) {
+   /**
+    * Instantiate a NullableVarBinaryVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
+   public NullableVarBinaryVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType);
-      reader = new VarCharReaderImpl(NullableVarCharVector.this);
-   }
-
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
+      reader = new VarBinaryReaderImpl(NullableVarBinaryVector.this);
    }
 
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
-      return Types.MinorType.VARCHAR;
+      return Types.MinorType.VARBINARY;
    }
 
 
@@ -92,18 +110,16 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     * Get the variable length element at specified index as Text.
     *
     * @param index   position of element to get
-    * @return Text object for non-null element, null otherwise
+    * @return byte array for non-null element, null otherwise
     */
-   public Text getObject(int index) {
-      Text result = new Text();
+   public byte[] getObject(int index) {
       byte[] b;
       try {
          b = get(index);
       } catch (IllegalStateException e) {
          return null;
       }
-      result.set(b);
-      return result;
+      return b;
    }
 
    /**
@@ -130,7 +146,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     * @param index   position of element to get
     * @param holder  data holder to be populated by this function
     */
-   public void get(int index, NullableVarCharHolder holder){
+   public void get(int index, NullableVarBinaryHolder holder){
       assert index >= 0;
       if(isSet(index) == 0) {
          holder.isSet = 0;
@@ -154,8 +170,14 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     ******************************************************************/
 
 
-
-   public void copyFrom(int fromIndex, int thisIndex, NullableVarCharVector from) {
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFrom(int fromIndex, int thisIndex, NullableVarBinaryVector from) {
       fillHoles(thisIndex);
       if (from.isSet(fromIndex) != 0) {
          set(thisIndex, from.get(fromIndex));
@@ -163,7 +185,15 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
       }
    }
 
-   public void copyFromSafe(int fromIndex, int thisIndex, NullableVarCharVector from) {
+   /**
+    * Same as {@link #copyFrom(int, int, NullableVarBinaryVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
+   public void copyFromSafe(int fromIndex, int thisIndex, NullableVarBinaryVector from) {
       fillEmpties(thisIndex);
       if (from.isSet(fromIndex) != 0) {
          setSafe(thisIndex, from.get(fromIndex));
@@ -288,7 +318,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     * @param index   position of the element to set
     * @param holder  holder that carries data buffer.
     */
-   public void set(int index, VarCharHolder holder) {
+   public void set(int index, VarBinaryHolder holder) {
       assert index >= 0;
       fillHoles(index);
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
@@ -300,14 +330,14 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, VarCharHolder)} except that it handles the
+    * Same as {@link #set(int, VarBinaryHolder)} except that it handles the
     * case where index and length of new element are beyond the existing
     * capacity of the vector.
     *
     * @param index   position of the element to set
     * @param holder  holder that carries data buffer.
     */
-   public void setSafe(int index, VarCharHolder holder) {
+   public void setSafe(int index, VarBinaryHolder holder) {
       assert index >= 0;
       final int dataLength = holder.end - holder.start;
       fillEmpties(index);
@@ -326,7 +356,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     * @param index   position of the element to set
     * @param holder  holder that carries data buffer.
     */
-   public void set(int index, NullableVarCharHolder holder) {
+   public void set(int index, NullableVarBinaryHolder holder) {
       assert index >= 0;
       fillHoles(index);
       BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
@@ -338,14 +368,14 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
    }
 
    /**
-    * Same as {@link #set(int, NullableVarCharHolder)} except that it handles the
+    * Same as {@link #set(int, NullableVarBinaryHolder)} except that it handles the
     * case where index and length of new element are beyond the existing
     * capacity of the vector.
     *
     * @param index   position of the element to set
     * @param holder  holder that carries data buffer.
     */
-   public void setSafe(int index, NullableVarCharHolder holder) {
+   public void setSafe(int index, NullableVarBinaryHolder holder) {
       assert index >= 0;
       final int dataLength = holder.end - holder.start;
       fillEmpties(index);
@@ -383,21 +413,42 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of data in buffer
+    * @param end end position of data in buffer
+    * @param buffer data buffer containing the variable width element to be stored
+    *               in the vector
+    */
+   public void set(int index, int isSet, int start, int end, ArrowBuf buffer) {
       assert index >= 0;
       fillHoles(index);
       BitVectorHelper.setValidityBit(validityBuffer, index, isSet);
       final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + endField);
-      final ArrowBuf bb = bufferField.slice(startField, endField);
+      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + end);
+      final ArrowBuf bb = buffer.slice(start, end);
       valueBuffer.setBytes(startOffset, bb);
       lastSet = index;
    }
 
-   public void setSafe(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
+   /**
+    * Same as {@link #set(int, int, int, int, ArrowBuf)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of data in buffer
+    * @param end end position of data in buffer
+    * @param buffer data buffer containing the variable width element to be stored
+    *               in the vector
+    */
+   public void setSafe(int index, int isSet, int start, int end, ArrowBuf buffer) {
       assert index >= 0;
-      handleSafe(index, endField);
-      set(index, isSet, startField, endField, bufferField);
+      handleSafe(index, end);
+      set(index, isSet, start, end, buffer);
    }
 
 
@@ -407,29 +458,41 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     *                                                                *
     ******************************************************************/
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableVarCharVector)to);
+      return new TransferImpl((NullableVarBinaryVector)to);
    }
 
    private class TransferImpl implements TransferPair {
-      NullableVarCharVector to;
+      NullableVarBinaryVector to;
 
       public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableVarCharVector(ref, field.getFieldType(), allocator);
+         to = new NullableVarBinaryVector(ref, field.getFieldType(), allocator);
       }
 
-      public TransferImpl(NullableVarCharVector to){
+      public TransferImpl(NullableVarBinaryVector to){
          this.to = to;
       }
 
       @Override
-      public NullableVarCharVector getTo(){
+      public NullableVarBinaryVector getTo(){
          return to;
       }
 
@@ -445,7 +508,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
 
       @Override
       public void copyValueSafe(int fromIndex, int toIndex) {
-         to.copyFromSafe(fromIndex, toIndex, NullableVarCharVector.this);
+         to.copyFromSafe(fromIndex, toIndex, NullableVarBinaryVector.this);
       }
    }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
index b902154..0b03164 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
@@ -1,5 +1,4 @@
-/*******************************************************************************
-
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,7 +14,7 @@
  * 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.arrow.vector;
 
@@ -32,30 +31,50 @@ import org.apache.arrow.vector.util.TransferPair;
 
 import java.nio.ByteBuffer;
 
+/**
+ * NullableVarCharVector implements a variable width vector of VARCHAR
+ * values which could be NULL. A validity buffer (bit vector) is maintained
+ * to track which elements in the vector are null.
+ */
 public class NullableVarCharVector extends BaseNullableVariableWidthVector {
-   private static final org.slf4j.Logger logger =
-           org.slf4j.LoggerFactory.getLogger(NullableIntVector.class);
    private final FieldReader reader;
 
+   /**
+    * Instantiate a NullableVarCharVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param allocator allocator for memory management.
+    */
    public NullableVarCharVector(String name, BufferAllocator allocator) {
       this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.VARCHAR.getType()), allocator);
    }
 
+   /**
+    * Instantiate a NullableVarCharVector. This doesn't allocate any memory for
+    * the data in vector.
+    * @param name name of the vector
+    * @param fieldType type of Field materialized by this vector
+    * @param allocator allocator for memory management.
+    */
    public NullableVarCharVector(String name, FieldType fieldType, BufferAllocator allocator) {
       super(name, allocator, fieldType);
       reader = new VarCharReaderImpl(NullableVarCharVector.this);
    }
 
-   @Override
-   protected org.slf4j.Logger getLogger() {
-      return logger;
-   }
-
+   /**
+    * Get a reader that supports reading values from this vector
+    * @return Field Reader for this vector
+    */
    @Override
    public FieldReader getReader(){
       return reader;
    }
 
+   /**
+    * Get minor type for this vector. The vector holds values belonging
+    * to a particular type.
+    * @return {@link org.apache.arrow.vector.types.Types.MinorType}
+    */
    @Override
    public Types.MinorType getMinorType() {
       return Types.MinorType.VARCHAR;
@@ -154,7 +173,13 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     ******************************************************************/
 
 
-
+   /**
+    * Copy a cell value from a particular index in source vector to a particular
+    * position in this vector
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
    public void copyFrom(int fromIndex, int thisIndex, NullableVarCharVector from) {
       fillHoles(thisIndex);
       if (from.isSet(fromIndex) != 0) {
@@ -163,6 +188,14 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
       }
    }
 
+   /**
+    * Same as {@link #copyFrom(int, int, NullableVarCharVector)} except that
+    * it handles the case when the capacity of the vector needs to be expanded
+    * before copy.
+    * @param fromIndex position to copy from in source vector
+    * @param thisIndex position to copy to in this vector
+    * @param from source vector
+    */
    public void copyFromSafe(int fromIndex, int thisIndex, NullableVarCharVector from) {
       fillEmpties(thisIndex);
       if (from.isSet(fromIndex) != 0) {
@@ -383,21 +416,42 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
    }
 
-   public void set(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
+   /**
+    * Store the given value at a particular position in the vector. isSet indicates
+    * whether the value is NULL or not.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of data in buffer
+    * @param end end position of data in buffer
+    * @param buffer data buffer containing the variable width element to be stored
+    *               in the vector
+    */
+   public void set(int index, int isSet, int start, int end, ArrowBuf buffer) {
       assert index >= 0;
       fillHoles(index);
       BitVectorHelper.setValidityBit(validityBuffer, index, isSet);
       final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + endField);
-      final ArrowBuf bb = bufferField.slice(startField, endField);
+      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + end);
+      final ArrowBuf bb = buffer.slice(start, end);
       valueBuffer.setBytes(startOffset, bb);
       lastSet = index;
    }
 
-   public void setSafe(int index, int isSet, int startField, int endField, ArrowBuf bufferField ) {
+   /**
+    * Same as {@link #set(int, int, int, int, ArrowBuf)} except that it handles the case
+    * when index is greater than or equal to current value capacity of the
+    * vector.
+    * @param index position of the new value
+    * @param isSet 0 for NULL value, 1 otherwise
+    * @param start start position of data in buffer
+    * @param end end position of data in buffer
+    * @param buffer data buffer containing the variable width element to be stored
+    *               in the vector
+    */
+   public void setSafe(int index, int isSet, int start, int end, ArrowBuf buffer) {
       assert index >= 0;
-      handleSafe(index, endField);
-      set(index, isSet, startField, endField, bufferField);
+      handleSafe(index, end);
+      set(index, isSet, start, end, buffer);
    }
 
 
@@ -407,11 +461,23 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
     *                                                                *
     ******************************************************************/
 
+   /**
+    * Construct a TransferPair comprising of this and and a target vector of
+    * the same type.
+    * @param ref name of the target vector
+    * @param allocator allocator for the target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair getTransferPair(String ref, BufferAllocator allocator){
       return new TransferImpl(ref, allocator);
    }
 
+   /**
+    * Construct a TransferPair with a desired target vector of the same type.
+    * @param to target vector
+    * @return {@link TransferPair}
+    */
    @Override
    public TransferPair makeTransferPair(ValueVector to) {
       return new TransferImpl((NullableVarCharVector)to);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/ValueVector.java
index e6048b4..dc74afa 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ValueVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ValueVector.java
@@ -256,9 +256,17 @@ public interface ValueVector extends Closeable, Iterable<ValueVector> {
    */
   public ArrowBuf getOffsetBuffer();
 
+  /* temporarily add these methods here until we remove other vectors
+   * (non-nullable) which are under ValueVector hierarchy and still
+   * use the mutator/accessor interfaces.
+   */
   public int getValueCount();
 
   public void setValueCount(int valueCount);
 
   public Object getObject(int index);
+
+  public int getNullCount();
+
+  public boolean isNull(int index);
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VectorSchemaRoot.java b/java/vector/src/main/java/org/apache/arrow/vector/VectorSchemaRoot.java
index 0c8868c..196393f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/VectorSchemaRoot.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VectorSchemaRoot.java
@@ -38,7 +38,7 @@ public class VectorSchemaRoot implements AutoCloseable {
   private final Map<String, FieldVector> fieldVectorsMap = new HashMap<>();
 
   public VectorSchemaRoot(FieldVector parent) {
-    this(parent.getField().getChildren(), parent.getChildrenFromFields(), parent.getAccessor().getValueCount());
+    this(parent.getField().getChildren(), parent.getChildrenFromFields(), parent.getValueCount());
   }
 
   public VectorSchemaRoot(List<Field> fields, List<FieldVector> fieldVectors, int rowCount) {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/VectorUnloader.java b/java/vector/src/main/java/org/apache/arrow/vector/VectorUnloader.java
index 96243eb..288afc6 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/VectorUnloader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VectorUnloader.java
@@ -55,17 +55,7 @@ public class VectorUnloader {
   }
 
   private void appendNodes(FieldVector vector, List<ArrowFieldNode> nodes, List<ArrowBuf> buffers) {
-    Accessor accessor = null;
-    if (vector instanceof NullableIntVector) {
-      nodes.add(new ArrowFieldNode(((NullableIntVector)vector).getValueCount(),
-                includeNullCount ? ((NullableIntVector)vector).getNullCount() : -1));
-    } else if (vector instanceof NullableVarCharVector) {
-      nodes.add(new ArrowFieldNode(((NullableVarCharVector)vector).getValueCount(),
-                includeNullCount ? ((NullableVarCharVector)vector).getNullCount() : -1));
-    } else {
-      accessor = vector.getAccessor();
-      nodes.add(new ArrowFieldNode(accessor.getValueCount(), includeNullCount ? accessor.getNullCount() : -1));
-    }
+    nodes.add(new ArrowFieldNode(vector.getValueCount(), includeNullCount ? vector.getNullCount() : -1));
     List<ArrowBuf> fieldBuffers = vector.getFieldBuffers();
     List<ArrowVectorType> expectedBuffers = vector.getField().getTypeLayout().getVectorTypes();
     if (fieldBuffers.size() != expectedBuffers.size()) {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ZeroVector.java b/java/vector/src/main/java/org/apache/arrow/vector/ZeroVector.java
index fcef02f..3cc93a2 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ZeroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ZeroVector.java
@@ -268,4 +268,10 @@ public class ZeroVector implements FieldVector {
 
   @Override
   public Object getObject(int index) { return null; }
+
+  @Override
+  public int getNullCount() { return 0; }
+
+  @Override
+  public boolean isNull(int index) { return false; }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractMapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractMapVector.java
index 26b0f90..1b20b09 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractMapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractMapVector.java
@@ -141,8 +141,8 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
   }
 
   private boolean nullFilled(ValueVector vector) {
-    for (int r = 0; r < vector.getAccessor().getValueCount(); r++) {
-      if (!vector.getAccessor().isNull(r)) {
+    for (int r = 0; r < vector.getValueCount(); r++) {
+      if (!vector.isNull(r)) {
         return false;
       }
     }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java
index 8e2877f..3eab30a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/BaseRepeatedValueVector.java
@@ -21,6 +21,7 @@ package org.apache.arrow.vector.complex;
 import java.util.Collections;
 import java.util.Iterator;
 
+import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.AddOrGetResult;
 import org.apache.arrow.vector.BaseValueVector;
@@ -31,6 +32,7 @@ import org.apache.arrow.vector.ZeroVector;
 import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.SchemaChangeRuntimeException;
 
 import com.google.common.base.Preconditions;
@@ -41,12 +43,14 @@ import io.netty.buffer.ArrowBuf;
 public abstract class BaseRepeatedValueVector extends BaseValueVector implements RepeatedValueVector {
 
   public final static FieldVector DEFAULT_DATA_VECTOR = ZeroVector.INSTANCE;
-  public final static String OFFSETS_VECTOR_NAME = "$offsets$";
   public final static String DATA_VECTOR_NAME = "$data$";
 
-  protected final UInt4Vector offsets;
+  public final static byte OFFSET_WIDTH = 4;
+  protected ArrowBuf offsetBuffer;
   protected FieldVector vector;
   protected final CallBack callBack;
+  protected int valueCount;
+  protected int offsetAllocationSizeInBytes = INITIAL_VALUE_ALLOCATION * OFFSET_WIDTH;
 
   protected BaseRepeatedValueVector(String name, BufferAllocator allocator, CallBack callBack) {
     this(name, allocator, DEFAULT_DATA_VECTOR, callBack);
@@ -54,42 +58,73 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
 
   protected BaseRepeatedValueVector(String name, BufferAllocator allocator, FieldVector vector, CallBack callBack) {
     super(name, allocator);
-    this.offsets = new UInt4Vector(OFFSETS_VECTOR_NAME, allocator);
+    this.offsetBuffer = allocator.getEmpty();
     this.vector = Preconditions.checkNotNull(vector, "data vector cannot be null");
     this.callBack = callBack;
+    this.valueCount = 0;
   }
 
   @Override
   public boolean allocateNewSafe() {
-    /* boolean to keep track if all the memory allocation were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    boolean success = false;
+    boolean dataAlloc = false;
     try {
-      if (!offsets.allocateNewSafe()) {
-        return false;
-      }
-      success = vector.allocateNewSafe();
-    } finally {
-      if (!success) {
+      allocateOffsetBuffer(offsetAllocationSizeInBytes);
+      dataAlloc = vector.allocateNewSafe();
+    } catch (Exception e) {
+      e.printStackTrace();
+      clear();
+      return false;
+    }
+    finally {
+      if (!dataAlloc) {
         clear();
       }
     }
-    offsets.zeroVector();
-    return success;
+    return dataAlloc;
+  }
+
+  protected void allocateOffsetBuffer(final long size) {
+    final int curSize = (int)size;
+    offsetBuffer = allocator.buffer(curSize);
+    offsetBuffer.readerIndex(0);
+    offsetAllocationSizeInBytes = curSize;
+    offsetBuffer.setZero(0, offsetBuffer.capacity());
   }
 
   @Override
   public void reAlloc() {
-    offsets.reAlloc();
+    reallocOffsetBuffer();
     vector.reAlloc();
   }
 
+  protected void reallocOffsetBuffer() {
+    final int currentBufferCapacity = offsetBuffer.capacity();
+    long baseSize = offsetAllocationSizeInBytes;
+
+    if (baseSize < (long)currentBufferCapacity) {
+      baseSize = (long)currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    newBuf.setBytes(0, offsetBuffer, 0, currentBufferCapacity);
+    final int halfNewCapacity = newBuf.capacity() / 2;
+    newBuf.setZero(halfNewCapacity, halfNewCapacity);
+    offsetBuffer.release(1);
+    offsetBuffer = newBuf;
+    offsetAllocationSizeInBytes = (int)newAllocationSize;
+  }
+
   @Override
+  @Deprecated
   public UInt4Vector getOffsetVector() {
-    return offsets;
+    throw new UnsupportedOperationException("There is no inner offset vector");
   }
 
   @Override
@@ -99,25 +134,29 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
 
   @Override
   public void setInitialCapacity(int numRecords) {
-    offsets.setInitialCapacity(numRecords + 1);
+    offsetAllocationSizeInBytes = (numRecords + 1) * OFFSET_WIDTH;
     vector.setInitialCapacity(numRecords * RepeatedValueVector.DEFAULT_REPEAT_PER_RECORD);
   }
 
   @Override
   public int getValueCapacity() {
-    final int offsetValueCapacity = Math.max(offsets.getValueCapacity() - 1, 0);
+    final int offsetValueCapacity = Math.max(getOffsetBufferValueCapacity() - 1, 0);
     if (vector == DEFAULT_DATA_VECTOR) {
       return offsetValueCapacity;
     }
     return Math.min(vector.getValueCapacity(), offsetValueCapacity);
   }
 
+  private int getOffsetBufferValueCapacity() {
+    return (int)((offsetBuffer.capacity() * 1.0)/OFFSET_WIDTH);
+  }
+
   @Override
   public int getBufferSize() {
-    if (getAccessor().getValueCount() == 0) {
+    if (getValueCount() == 0) {
       return 0;
     }
-    return offsets.getBufferSize() + vector.getBufferSize();
+    return ((valueCount + 1) * OFFSET_WIDTH) + vector.getBufferSize();
   }
 
   @Override
@@ -126,7 +165,7 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
       return 0;
     }
 
-    return offsets.getBufferSizeFor(valueCount + 1) + vector.getBufferSizeFor(valueCount);
+    return ((valueCount + 1) * OFFSET_WIDTH) + vector.getBufferSizeFor(valueCount);
   }
 
   @Override
@@ -136,14 +175,16 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
 
   @Override
   public void clear() {
-    offsets.clear();
+    offsetBuffer = releaseBuffer(offsetBuffer);
     vector.clear();
+    valueCount = 0;
     super.clear();
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(false), vector.getBuffers(false), ArrowBuf.class);
+    final ArrowBuf[] buffers = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer},
+            vector.getBuffers(false), ArrowBuf.class);
     if (clear) {
       for (ArrowBuf buffer : buffers) {
         buffer.retain();
@@ -187,54 +228,61 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
     vector = v;
   }
 
-  public abstract class BaseRepeatedAccessor extends BaseValueVector.BaseAccessor implements RepeatedAccessor {
 
-    @Override
-    public int getValueCount() {
-      return Math.max(offsets.getAccessor().getValueCount() - 1, 0);
-    }
+  @Override
+  public int getValueCount() {
+    return valueCount;
+  }
 
-    @Override
-    public int getInnerValueCount() {
-      return vector.getAccessor().getValueCount();
-    }
+  /* returns the value count for inner data vector for this list vector */
+  public int getInnerValueCount() {
+    return vector.getValueCount();
+  }
 
-    @Override
-    public int getInnerValueCountAt(int index) {
-      return offsets.getAccessor().get(index + 1) - offsets.getAccessor().get(index);
-    }
 
-    @Override
-    public boolean isNull(int index) {
-      return false;
-    }
+  /* returns the value count for inner data vector at a particular index */
+  public int getInnerValueCountAt(int index) {
+    return offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) -
+            offsetBuffer.getInt(index * OFFSET_WIDTH);
+  }
 
-    @Override
-    public boolean isEmpty(int index) {
-      return false;
-    }
+  public boolean isNull(int index) {
+    return false;
   }
 
-  public abstract class BaseRepeatedMutator extends BaseValueVector.BaseMutator implements RepeatedMutator {
+  public boolean isEmpty(int index) {
+    return false;
+  }
 
-    @Override
-    public int startNewValue(int index) {
-      while (offsets.getValueCapacity() <= index) {
-        offsets.reAlloc();
-      }
-      int offset = offsets.getAccessor().get(index);
-      offsets.getMutator().setSafe(index + 1, offset);
-      setValueCount(index + 1);
-      return offset;
+  public int startNewValue(int index) {
+    while (index >= getOffsetBufferValueCapacity()) {
+      reallocOffsetBuffer();
     }
+    int offset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, offset);
+    setValueCount(index + 1);
+    return offset;
+  }
 
-    @Override
-    public void setValueCount(int valueCount) {
-      // TODO: populate offset end points
-      offsets.getMutator().setValueCount(valueCount == 0 ? 0 : valueCount + 1);
-      final int childValueCount = valueCount == 0 ? 0 : offsets.getAccessor().get(valueCount);
-      vector.getMutator().setValueCount(childValueCount);
+  public void setValueCount(int valueCount) {
+    this.valueCount = valueCount;
+    while (valueCount > getOffsetBufferValueCapacity()) {
+      reallocOffsetBuffer();
     }
+    final int childValueCount = valueCount == 0 ? 0 :
+            offsetBuffer.getInt(valueCount * OFFSET_WIDTH);
+    vector.setValueCount(childValueCount);
+  }
+
+  @Override
+  @Deprecated
+  public RepeatedAccessor getAccessor() {
+    throw new UnsupportedOperationException("Accessor is not supported for reading from LIST.");
   }
 
+  @Override
+  @Deprecated
+  public RepeatedMutator getMutator() {
+    throw new UnsupportedOperationException("Mutator is not supported for writing to LIST");
+  }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
index e95442a..bb091f6 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/FixedSizeListVector.java
@@ -22,16 +22,14 @@ package org.apache.arrow.vector.complex;
 import static java.util.Collections.singletonList;
 import static org.apache.arrow.vector.complex.BaseRepeatedValueVector.DATA_VECTOR_NAME;
 
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
+import java.util.*;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ObjectArrays;
 
 import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.OutOfMemoryException;
 import org.apache.arrow.vector.*;
@@ -42,10 +40,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.util.CallBack;
-import org.apache.arrow.vector.util.JsonStringArrayList;
-import org.apache.arrow.vector.util.SchemaChangeRuntimeException;
-import org.apache.arrow.vector.util.TransferPair;
+import org.apache.arrow.vector.util.*;
 
 public class FixedSizeListVector extends BaseValueVector implements FieldVector, PromotableVector {
 
@@ -55,15 +50,13 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
   }
 
   private FieldVector vector;
-  private final BitVector bits;
+  private ArrowBuf validityBuffer;
   private final int listSize;
   private final FieldType fieldType;
-  private final List<BufferBacked> innerVectors;
 
   private UnionFixedSizeListReader reader;
-
-  private Mutator mutator = new Mutator();
-  private Accessor accessor = new Accessor();
+  private int valueCount;
+  private int validityAllocationSizeInBytes;
 
   // deprecated, use FieldType or static constructor instead
   @Deprecated
@@ -80,13 +73,14 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
                              FieldType fieldType,
                              CallBack schemaChangeCallback) {
     super(name, allocator);
-    this.bits = new BitVector("$bits$", allocator);
+    this.validityBuffer = allocator.getEmpty();
     this.vector = ZeroVector.INSTANCE;
     this.fieldType = fieldType;
     this.listSize = ((ArrowType.FixedSizeList) fieldType.getType()).getListSize();
     Preconditions.checkArgument(listSize > 0, "list size must be positive");
-    this.innerVectors = Collections.singletonList((BufferBacked) bits);
     this.reader = new UnionFixedSizeListReader(this);
+    this.valueCount = 0;
+    this.validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
   }
 
   @Override
@@ -124,27 +118,47 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    BaseDataValueVector.load(fieldNode, innerVectors, ownBuffers);
+    if (ownBuffers.size() != 1) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 1 + ", got: " + ownBuffers.size());
+    }
+
+    ArrowBuf bitBuffer = ownBuffers.get(0);
+
+    validityBuffer.release();
+    validityBuffer = bitBuffer.retain(allocator);
+    valueCount = fieldNode.getLength();
+
+    validityAllocationSizeInBytes = validityBuffer.capacity();
   }
 
   @Override
   public List<ArrowBuf> getFieldBuffers() {
-    return BaseDataValueVector.unload(innerVectors);
+    List<ArrowBuf> result = new ArrayList<>(1);
+
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+
+    result.add(validityBuffer);
+
+    return result;
   }
 
   @Override
+  @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
-    return innerVectors;
+    throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
   }
 
   @Override
+  @Deprecated
   public Accessor getAccessor() {
-    return accessor;
+    throw new UnsupportedOperationException("Accessor is not supported for reading from vector");
   }
 
   @Override
+  @Deprecated
   public Mutator getMutator() {
-    return mutator;
+    throw new UnsupportedOperationException("Mutator is not supported for writing to vector");
   }
 
   @Override
@@ -154,7 +168,9 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public void allocateNew() throws OutOfMemoryException {
-    allocateNewSafe();
+    if (!allocateNewSafe()) {
+      throw new OutOfMemoryException("Failure while allocating memory");
+    }
   }
 
   @Override
@@ -166,31 +182,63 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
      */
     boolean success = false;
     try {
-      success = bits.allocateNewSafe() && vector.allocateNewSafe();
+      /* allocate validity buffer */
+      allocateValidityBuffer(validityAllocationSizeInBytes);
+      success = vector.allocateNewSafe();
     } finally {
       if (!success) {
         clear();
+        return false;
       }
     }
-    if (success) {
-      bits.zeroVector();
-    }
-    return success;
+
+    return true;
+  }
+
+  private void allocateValidityBuffer(final long size) {
+    final int curSize = (int)size;
+    validityBuffer = allocator.buffer(curSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = curSize;
+    validityBuffer.setZero(0, validityBuffer.capacity());
   }
 
   @Override
   public void reAlloc() {
-    bits.reAlloc();
+    reallocValidityBuffer();
     vector.reAlloc();
   }
 
+  private void reallocValidityBuffer() {
+    final int currentBufferCapacity = validityBuffer.capacity();
+    long baseSize = validityAllocationSizeInBytes;
+
+    if (baseSize < (long)currentBufferCapacity) {
+      baseSize = (long)currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    newBuf.setZero(0, newBuf.capacity());
+    newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity);
+    validityBuffer.release(1);
+    validityBuffer = newBuf;
+    validityAllocationSizeInBytes = (int)newAllocationSize;
+  }
+
   public FieldVector getDataVector() {
     return vector;
   }
 
   @Override
   public void setInitialCapacity(int numRecords) {
-    bits.setInitialCapacity(numRecords);
+    validityAllocationSizeInBytes = getValidityBufferSizeFromCount(numRecords);
     vector.setInitialCapacity(numRecords * listSize);
   }
 
@@ -199,15 +247,15 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     if (vector == ZeroVector.INSTANCE) {
       return 0;
     }
-    return vector.getValueCapacity() / listSize;
+    return Math.min(vector.getValueCapacity() / listSize, getValidityBufferValueCapacity());
   }
 
   @Override
   public int getBufferSize() {
-    if (accessor.getValueCount() == 0) {
+    if (getValueCount() == 0) {
       return 0;
     }
-    return bits.getBufferSize() + vector.getBufferSize();
+    return getValidityBufferSizeFromCount(valueCount) + vector.getBufferSize();
   }
 
   @Override
@@ -215,7 +263,8 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     if (valueCount == 0) {
       return 0;
     }
-    return bits.getBufferSizeFor(valueCount) + vector.getBufferSizeFor(valueCount * listSize);
+    return getValidityBufferSizeFromCount(valueCount) +
+            vector.getBufferSizeFor(valueCount * listSize);
   }
 
   @Override
@@ -225,14 +274,15 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public void clear() {
-    bits.clear();
+    validityBuffer = releaseBuffer(validityBuffer);
     vector.clear();
+    valueCount = 0;
     super.clear();
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(bits.getBuffers(false), vector.getBuffers(false), ArrowBuf.class);
+    final ArrowBuf[] buffers = ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, vector.getBuffers(false), ArrowBuf.class);
     if (clear) {
       for (ArrowBuf buffer : buffers) {
         buffer.retain();
@@ -288,7 +338,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public long getValidityBufferAddress() {
-    return (bits.getDataBuffer().memoryAddress());
+    return validityBuffer.memoryAddress();
   }
 
   @Override
@@ -303,7 +353,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public ArrowBuf getValidityBuffer() {
-    return (bits.getDataBuffer());
+    return validityBuffer;
   }
 
   @Override
@@ -316,62 +366,64 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     throw new UnsupportedOperationException();
   }
 
-  public class Accessor extends BaseValueVector.BaseAccessor {
-
-    @Override
-    public Object getObject(int index) {
-      if (isNull(index)) {
-        return null;
-      }
-      final List<Object> vals = new JsonStringArrayList<>(listSize);
-      if (vector instanceof NullableIntVector || vector instanceof NullableVarCharVector) {
-        for (int i = 0; i < listSize; i++) {
-          vals.add(vector.getObject(index * listSize + i));
-        }
-      } else {
-        final ValueVector.Accessor valuesAccessor = vector.getAccessor();
-        for (int i = 0; i < listSize; i++) {
-          vals.add(valuesAccessor.getObject(index * listSize + i));
-        }
-      }
-      return vals;
+  @Override
+  public Object getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
     }
-
-    @Override
-    public boolean isNull(int index) {
-      return bits.getAccessor().get(index) == 0;
+    final List<Object> vals = new JsonStringArrayList<>(listSize);
+    for (int i = 0; i < listSize; i++) {
+      vals.add(vector.getObject(index * listSize + i));
     }
+    return vals;
+  }
 
-    @Override
-    public int getNullCount() {
-      return bits.getAccessor().getNullCount();
-    }
+  public boolean isNull(int index) {
+    return (isSet(index) == 0);
+  }
 
-    @Override
-    public int getValueCount() {
-      return bits.getAccessor().getValueCount();
-    }
+  public int isSet(int index) {
+    final int byteIndex = index >> 3;
+    final byte b = validityBuffer.getByte(byteIndex);
+    final int bitIndex = index & 7;
+    return Long.bitCount(b & (1L << bitIndex));
   }
 
-  public class Mutator extends BaseValueVector.BaseMutator {
+  @Override
+  public int getNullCount() {
+    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
+  }
 
-    public void setNull(int index) {
-      bits.getMutator().setSafe(index, 0);
+  @Override
+  public int getValueCount() {
+    return valueCount;
+  }
+
+  private int getValidityBufferValueCapacity() {
+    return (int)(validityBuffer.capacity() * 8L);
+  }
+
+  public void setNull(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      reallocValidityBuffer();
     }
+    BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+  }
 
-    public void setNotNull(int index) {
-      bits.getMutator().setSafe(index, 1);
+  public void setNotNull(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      reallocValidityBuffer();
     }
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+  }
 
-    @Override
-    public void setValueCount(int valueCount) {
-      bits.getMutator().setValueCount(valueCount);
-      if (vector instanceof  NullableIntVector || vector instanceof NullableVarCharVector) {
-        vector.setValueCount(valueCount * listSize);
-      } else {
-        vector.getMutator().setValueCount(valueCount * listSize);
-      }
+  @Override
+  public void setValueCount(int valueCount) {
+    this.valueCount = valueCount;
+    while (valueCount > getValidityBufferValueCapacity()) {
+      reallocValidityBuffer();
     }
+    vector.setValueCount(valueCount * listSize);
   }
 
   @Override
@@ -392,7 +444,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
   private class TransferImpl implements TransferPair {
 
     FixedSizeListVector to;
-    TransferPair pairs[] = new TransferPair[2];
+    TransferPair dataPair;
 
     public TransferImpl(String name, BufferAllocator allocator, CallBack callBack) {
       this(new FixedSizeListVector(name, allocator, fieldType, callBack));
@@ -401,19 +453,20 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     public TransferImpl(FixedSizeListVector to) {
       this.to = to;
       to.addOrGetVector(vector.getField().getFieldType());
-      pairs[0] = bits.makeTransferPair(to.bits);
-      pairs[1] = vector.makeTransferPair(to.vector);
+      dataPair = vector.makeTransferPair(to.vector);
     }
 
     @Override
     public void transfer() {
-      for (TransferPair pair : pairs) {
-        pair.transfer();
-      }
+      to.clear();
+      dataPair.transfer();
+      to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer;
+      to.setValueCount(valueCount);
     }
 
     @Override
     public void splitAndTransfer(int startIndex, int length) {
+      to.clear();
       to.allocateNew();
       for (int i = 0; i < length; i++) {
         copyValueSafe(startIndex + i, i);
@@ -426,12 +479,15 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     }
 
     @Override
-    public void copyValueSafe(int from, int to) {
-      pairs[0].copyValueSafe(from, to);
-      int fromOffset = from * listSize;
-      int toOffset = to * listSize;
+    public void copyValueSafe(int fromIndex, int toIndex) {
+      while (toIndex >= to.getValueCapacity()) {
+        to.reAlloc();
+      }
+      BitVectorHelper.setValidityBit(to.validityBuffer, toIndex, isSet(fromIndex));
+      int fromOffset = fromIndex * listSize;
+      int toOffset = toIndex * listSize;
       for (int i = 0; i < listSize; i++) {
-        pairs[1].copyValueSafe(fromOffset + i, toOffset + i);
+        dataPair.copyValueSafe(fromOffset + i, toOffset + i);
       }
     }
   }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyBaseRepeatedValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyBaseRepeatedValueVector.java
new file mode 100644
index 0000000..435d3d8
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyBaseRepeatedValueVector.java
@@ -0,0 +1,240 @@
+/**
+ * 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.arrow.vector.complex;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.AddOrGetResult;
+import org.apache.arrow.vector.BaseValueVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.ZeroVector;
+import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.SchemaChangeRuntimeException;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ObjectArrays;
+
+import io.netty.buffer.ArrowBuf;
+
+public abstract class LegacyBaseRepeatedValueVector extends BaseValueVector implements RepeatedValueVector {
+
+   public final static FieldVector DEFAULT_DATA_VECTOR = ZeroVector.INSTANCE;
+   public final static String OFFSETS_VECTOR_NAME = "$offsets$";
+   public final static String DATA_VECTOR_NAME = "$data$";
+
+   protected final UInt4Vector offsets;
+   protected FieldVector vector;
+   protected final CallBack callBack;
+
+   protected LegacyBaseRepeatedValueVector(String name, BufferAllocator allocator, CallBack callBack) {
+      this(name, allocator, DEFAULT_DATA_VECTOR, callBack);
+   }
+
+   protected LegacyBaseRepeatedValueVector(String name, BufferAllocator allocator, FieldVector vector, CallBack callBack) {
+      super(name, allocator);
+      this.offsets = new UInt4Vector(OFFSETS_VECTOR_NAME, allocator);
+      this.vector = Preconditions.checkNotNull(vector, "data vector cannot be null");
+      this.callBack = callBack;
+   }
+
+   @Override
+   public boolean allocateNewSafe() {
+    /* boolean to keep track if all the memory allocation were successful
+     * Used in the case of composite vectors when we need to allocate multiple
+     * buffers for multiple vectors. If one of the allocations failed we need to
+     * clear all the memory that we allocated
+     */
+      boolean success = false;
+      try {
+         if (!offsets.allocateNewSafe()) {
+            return false;
+         }
+         success = vector.allocateNewSafe();
+      } finally {
+         if (!success) {
+            clear();
+         }
+      }
+      offsets.zeroVector();
+      return success;
+   }
+
+   @Override
+   public void reAlloc() {
+      offsets.reAlloc();
+      vector.reAlloc();
+   }
+
+   @Override
+   public UInt4Vector getOffsetVector() {
+      return offsets;
+   }
+
+   @Override
+   public FieldVector getDataVector() {
+      return vector;
+   }
+
+   @Override
+   public void setInitialCapacity(int numRecords) {
+      offsets.setInitialCapacity(numRecords + 1);
+      vector.setInitialCapacity(numRecords * RepeatedValueVector.DEFAULT_REPEAT_PER_RECORD);
+   }
+
+   @Override
+   public int getValueCapacity() {
+      final int offsetValueCapacity = Math.max(offsets.getValueCapacity() - 1, 0);
+      if (vector == DEFAULT_DATA_VECTOR) {
+         return offsetValueCapacity;
+      }
+      return Math.min(vector.getValueCapacity(), offsetValueCapacity);
+   }
+
+   @Override
+   public int getBufferSize() {
+      if (getAccessor().getValueCount() == 0) {
+         return 0;
+      }
+      return offsets.getBufferSize() + vector.getBufferSize();
+   }
+
+   @Override
+   public int getBufferSizeFor(int valueCount) {
+      if (valueCount == 0) {
+         return 0;
+      }
+
+      return offsets.getBufferSizeFor(valueCount + 1) + vector.getBufferSizeFor(valueCount);
+   }
+
+   @Override
+   public Iterator<ValueVector> iterator() {
+      return Collections.<ValueVector>singleton(getDataVector()).iterator();
+   }
+
+   @Override
+   public void clear() {
+      offsets.clear();
+      vector.clear();
+      super.clear();
+   }
+
+   @Override
+   public ArrowBuf[] getBuffers(boolean clear) {
+      final ArrowBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(false), vector.getBuffers(false), ArrowBuf.class);
+      if (clear) {
+         for (ArrowBuf buffer : buffers) {
+            buffer.retain();
+         }
+         clear();
+      }
+      return buffers;
+   }
+
+   /**
+    * @return 1 if inner vector is explicitly set via #addOrGetVector else 0
+    */
+   public int size() {
+      return vector == DEFAULT_DATA_VECTOR ? 0 : 1;
+   }
+
+   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(FieldType fieldType) {
+      boolean created = false;
+      if (vector instanceof ZeroVector) {
+         vector = fieldType.createNewSingleVector(DATA_VECTOR_NAME, allocator, callBack);
+         // returned vector must have the same field
+         created = true;
+         if (callBack != null &&
+                 // not a schema change if changing from ZeroVector to ZeroVector
+                 (fieldType.getType().getTypeID() != ArrowTypeID.Null)) {
+            callBack.doWork();
+         }
+      }
+
+      if (vector.getField().getType().getTypeID() != fieldType.getType().getTypeID()) {
+         final String msg = String.format("Inner vector type mismatch. Requested type: [%s], actual type: [%s]",
+                 fieldType.getType().getTypeID(), vector.getField().getType().getTypeID());
+         throw new SchemaChangeRuntimeException(msg);
+      }
+
+      return new AddOrGetResult<>((T) vector, created);
+   }
+
+   protected void replaceDataVector(FieldVector v) {
+      vector.clear();
+      vector = v;
+   }
+
+   public abstract class LegacyBaseRepeatedAccessor extends BaseValueVector.BaseAccessor implements RepeatedAccessor {
+
+      @Override
+      public int getValueCount() {
+         return Math.max(offsets.getAccessor().getValueCount() - 1, 0);
+      }
+
+      @Override
+      public int getInnerValueCount() {
+         return vector.getAccessor().getValueCount();
+      }
+
+      @Override
+      public int getInnerValueCountAt(int index) {
+         return offsets.getAccessor().get(index + 1) - offsets.getAccessor().get(index);
+      }
+
+      @Override
+      public boolean isNull(int index) {
+         return false;
+      }
+
+      @Override
+      public boolean isEmpty(int index) {
+         return false;
+      }
+   }
+
+   public abstract class LegacyBaseRepeatedMutator extends BaseValueVector.BaseMutator implements RepeatedMutator {
+
+      @Override
+      public int startNewValue(int index) {
+         while (offsets.getValueCapacity() <= index) {
+            offsets.reAlloc();
+         }
+         int offset = offsets.getAccessor().get(index);
+         offsets.getMutator().setSafe(index + 1, offset);
+         setValueCount(index + 1);
+         return offset;
+      }
+
+      @Override
+      public void setValueCount(int valueCount) {
+         // TODO: populate offset end points
+         offsets.getMutator().setValueCount(valueCount == 0 ? 0 : valueCount + 1);
+         final int childValueCount = valueCount == 0 ? 0 : offsets.getAccessor().get(valueCount);
+         vector.getMutator().setValueCount(childValueCount);
+      }
+   }
+
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyFixedSizeListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyFixedSizeListVector.java
new file mode 100644
index 0000000..0843907
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyFixedSizeListVector.java
@@ -0,0 +1,298 @@
+/*******************************************************************************
+
+ * 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.arrow.vector.complex;
+
+import static java.util.Collections.singletonList;
+import static org.apache.arrow.vector.complex.BaseRepeatedValueVector.DATA_VECTOR_NAME;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ObjectArrays;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.apache.arrow.vector.*;
+import org.apache.arrow.vector.complex.impl.UnionFixedSizeListReader;
+import org.apache.arrow.vector.schema.ArrowFieldNode;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.JsonStringArrayList;
+import org.apache.arrow.vector.util.SchemaChangeRuntimeException;
+import org.apache.arrow.vector.util.TransferPair;
+
+public class LegacyFixedSizeListVector extends BaseValueVector implements FieldVector, PromotableVector {
+
+   public static LegacyFixedSizeListVector empty(String name, int size, BufferAllocator allocator) {
+      FieldType fieldType = FieldType.nullable(new ArrowType.FixedSizeList(size));
+      return new LegacyFixedSizeListVector(name, allocator, fieldType, null);
+   }
+
+   private Mutator mutator = new Mutator();
+   private Accessor accessor = new Accessor();
+   /* delegate */
+   private final FixedSizeListVector fixedSizeListVector;
+
+   // deprecated, use FieldType or static constructor instead
+   @Deprecated
+   public LegacyFixedSizeListVector(String name,
+                              BufferAllocator allocator,
+                              int listSize,
+                              DictionaryEncoding dictionary,
+                              CallBack schemaChangeCallback) {
+      super(name, allocator);
+      fixedSizeListVector = new FixedSizeListVector(name, allocator, listSize, dictionary, schemaChangeCallback);
+   }
+
+   public LegacyFixedSizeListVector(String name,
+                              BufferAllocator allocator,
+                              FieldType fieldType,
+                              CallBack schemaChangeCallback) {
+      super(name, allocator);
+      fixedSizeListVector = new FixedSizeListVector(name, allocator, fieldType, schemaChangeCallback);
+   }
+
+   @Override
+   public Field getField() {
+      return fixedSizeListVector.getField();
+   }
+
+   @Override
+   public MinorType getMinorType() {
+      return MinorType.FIXED_SIZE_LIST;
+   }
+
+   public int getListSize() {
+      return fixedSizeListVector.getListSize();
+   }
+
+   @Override
+   public void initializeChildrenFromFields(List<Field> children) {
+      fixedSizeListVector.initializeChildrenFromFields(children);
+   }
+
+   @Override
+   public List<FieldVector> getChildrenFromFields() {
+      return fixedSizeListVector.getChildrenFromFields();
+   }
+
+   @Override
+   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
+      fixedSizeListVector.loadFieldBuffers(fieldNode, ownBuffers);
+   }
+
+   @Override
+   public List<ArrowBuf> getFieldBuffers() {
+      return fixedSizeListVector.getFieldBuffers();
+   }
+
+   @Override
+   public List<BufferBacked> getFieldInnerVectors() {
+      return fixedSizeListVector.getFieldInnerVectors();
+   }
+
+   @Override
+   public Accessor getAccessor() {
+      return accessor;
+   }
+
+   @Override
+   public Mutator getMutator() {
+      return mutator;
+   }
+
+   @Override
+   public UnionFixedSizeListReader getReader() {
+      return fixedSizeListVector.getReader();
+   }
+
+   @Override
+   public void allocateNew() throws OutOfMemoryException {
+     fixedSizeListVector.allocateNew();
+   }
+
+   @Override
+   public boolean allocateNewSafe() {
+    return fixedSizeListVector.allocateNewSafe();
+   }
+
+   @Override
+   public void reAlloc() {
+      fixedSizeListVector.reAlloc();
+   }
+
+   public FieldVector getDataVector() {
+      return fixedSizeListVector.getDataVector();
+   }
+
+   @Override
+   public void setInitialCapacity(int numRecords) {
+      fixedSizeListVector.setInitialCapacity(numRecords);
+   }
+
+   @Override
+   public int getValueCapacity() {
+     return fixedSizeListVector.getValueCapacity();
+   }
+
+   @Override
+   public int getBufferSize() {
+     return fixedSizeListVector.getBufferSize();
+   }
+
+   @Override
+   public int getBufferSizeFor(int valueCount) {
+    return fixedSizeListVector.getBufferSizeFor(valueCount);
+   }
+
+   @Override
+   public Iterator<ValueVector> iterator() {
+      return fixedSizeListVector.iterator();
+   }
+
+   @Override
+   public void clear() {
+     fixedSizeListVector.clear();
+   }
+
+   @Override
+   public ArrowBuf[] getBuffers(boolean clear) {
+      return fixedSizeListVector.getBuffers(clear);
+   }
+
+   /**
+    * @return 1 if inner vector is explicitly set via #addOrGetVector else 0
+    */
+   public int size() {
+      return fixedSizeListVector.size();
+   }
+
+   @Override
+   @SuppressWarnings("unchecked")
+   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(FieldType type) {
+      return fixedSizeListVector.addOrGetVector(type);
+   }
+
+   public void copyFromSafe(int inIndex, int outIndex, FixedSizeListVector from) {
+      fixedSizeListVector.copyFromSafe(inIndex, outIndex, from);
+   }
+
+   public void copyFrom(int fromIndex, int thisIndex, FixedSizeListVector from) {
+     fixedSizeListVector.copyFrom(fromIndex, thisIndex, from);
+   }
+
+   @Override
+   public UnionVector promoteToUnion() {
+      return fixedSizeListVector.promoteToUnion();
+   }
+
+   @Override
+   public long getValidityBufferAddress() {
+      return fixedSizeListVector.getValidityBufferAddress();
+   }
+
+   @Override
+   public long getDataBufferAddress() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public long getOffsetBufferAddress() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getValidityBuffer() {
+      return fixedSizeListVector.getValidityBuffer();
+   }
+
+   @Override
+   public ArrowBuf getDataBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getOffsetBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   public class Accessor extends BaseValueVector.BaseAccessor {
+
+      @Override
+      public Object getObject(int index) {
+         return fixedSizeListVector.getObject(index);
+      }
+
+      @Override
+      public boolean isNull(int index) {
+         return fixedSizeListVector.isNull(index);
+      }
+
+      @Override
+      public int getNullCount() {
+         return fixedSizeListVector.getNullCount();
+      }
+
+      @Override
+      public int getValueCount() {
+         return fixedSizeListVector.getValueCount();
+      }
+   }
+
+   public class Mutator extends BaseValueVector.BaseMutator {
+
+      public void setNull(int index) {
+         fixedSizeListVector.setNull(index);
+      }
+
+      public void setNotNull(int index) {
+         fixedSizeListVector.setNotNull(index);
+      }
+
+      @Override
+      public void setValueCount(int valueCount) {
+         fixedSizeListVector.setValueCount(valueCount);
+      }
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      return fixedSizeListVector.getTransferPair(ref, allocator);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
+      return fixedSizeListVector.getTransferPair(ref, allocator, callBack);
+   }
+
+   @Override
+   public TransferPair makeTransferPair(ValueVector target) {
+      return fixedSizeListVector.makeTransferPair(((LegacyFixedSizeListVector)target).fixedSizeListVector);
+   }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyListVector.java
new file mode 100644
index 0000000..732699a
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyListVector.java
@@ -0,0 +1,288 @@
+/*******************************************************************************
+
+ * 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.arrow.vector.complex;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static java.util.Collections.list;
+import static java.util.Collections.singletonList;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ObjectArrays;
+
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.apache.arrow.vector.*;
+import org.apache.arrow.vector.complex.impl.ComplexCopier;
+import org.apache.arrow.vector.complex.impl.UnionListReader;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.complex.writer.FieldWriter;
+import org.apache.arrow.vector.schema.ArrowFieldNode;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.ArrowType.Null;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.JsonStringArrayList;
+import org.apache.arrow.vector.util.TransferPair;
+
+public class LegacyListVector extends LegacyBaseRepeatedValueVector implements FieldVector, PromotableVector {
+
+   public static LegacyListVector empty(String name, BufferAllocator allocator) {
+      return new LegacyListVector(name, allocator, FieldType.nullable(ArrowType.List.INSTANCE), null);
+   }
+
+   private Mutator mutator = new Mutator();
+   private Accessor accessor = new Accessor();
+
+   private final ListVector listVector;
+
+   // deprecated, use FieldType or static constructor instead
+   @Deprecated
+   public LegacyListVector(String name, BufferAllocator allocator, CallBack callBack) {
+      super(name, allocator, callBack);
+      listVector = new ListVector(name, allocator, callBack);
+   }
+
+   // deprecated, use FieldType or static constructor instead
+   @Deprecated
+   public LegacyListVector(String name, BufferAllocator allocator, DictionaryEncoding dictionary, CallBack callBack) {
+      super(name, allocator, callBack);
+      listVector = new ListVector(name, allocator, dictionary, callBack);
+   }
+
+   public LegacyListVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
+      super(name, allocator, callBack);
+      listVector = new ListVector(name, allocator, fieldType, callBack);
+   }
+
+   @Override
+   public void initializeChildrenFromFields(List<Field> children) {
+      listVector.initializeChildrenFromFields(children);
+   }
+
+   @Override
+   public List<FieldVector> getChildrenFromFields() {
+      return listVector.getChildrenFromFields();
+   }
+
+   @Override
+   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
+      listVector.loadFieldBuffers(fieldNode, ownBuffers);
+   }
+
+   @Override
+   public List<ArrowBuf> getFieldBuffers() {
+      return listVector.getFieldBuffers();
+   }
+
+   @Override
+   public List<BufferBacked> getFieldInnerVectors() {
+      return listVector.getFieldInnerVectors();
+   }
+
+   public UnionListWriter getWriter() {
+      return listVector.getWriter();
+   }
+
+   @Override
+   public void allocateNew() throws OutOfMemoryException {
+      listVector.allocateNew();
+   }
+
+   @Override
+   public void reAlloc() {
+      listVector.reAlloc();
+   }
+
+   public void copyFromSafe(int inIndex, int outIndex, ListVector from) {
+      listVector.copyFrom(inIndex, outIndex, from);
+   }
+
+   public void copyFrom(int inIndex, int outIndex, ListVector from) {
+      listVector.copyFrom(inIndex, outIndex, from);
+   }
+
+   @Override
+   public FieldVector getDataVector() {
+      return listVector.getDataVector();
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      return listVector.getTransferPair(ref, allocator);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
+      return listVector.getTransferPair(ref, allocator, callBack);
+   }
+
+   @Override
+   public TransferPair makeTransferPair(ValueVector target) {
+      return listVector.makeTransferPair(((LegacyListVector)target).listVector);
+   }
+
+   @Override
+   public long getValidityBufferAddress() {
+      return listVector.getValidityBufferAddress();
+   }
+
+   @Override
+   public long getDataBufferAddress() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public long getOffsetBufferAddress() {
+      return listVector.getOffsetBufferAddress();
+   }
+
+   @Override
+   public ArrowBuf getValidityBuffer() {
+      return listVector.getValidityBuffer();
+   }
+
+   @Override
+   public ArrowBuf getDataBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getOffsetBuffer() {
+      return listVector.getOffsetBuffer();
+   }
+
+   @Override
+   public Accessor getAccessor() {
+      return accessor;
+   }
+
+   @Override
+   public Mutator getMutator() {
+      return mutator;
+   }
+
+   @Override
+   public UnionListReader getReader() {
+      return listVector.getReader();
+   }
+
+   @Override
+   public boolean allocateNewSafe() {
+      return listVector.allocateNewSafe();
+   }
+
+   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(FieldType fieldType) {
+      return listVector.addOrGetVector(fieldType);
+   }
+
+   @Override
+   public int getBufferSize() {
+      return listVector.getBufferSize();
+   }
+
+   @Override
+   public Field getField() {
+     return listVector.getField();
+   }
+
+   @Override
+   public MinorType getMinorType() {
+      return MinorType.LIST;
+   }
+
+   @Override
+   public void clear() {
+      listVector.clear();
+   }
+
+   @Override
+   public ArrowBuf[] getBuffers(boolean clear) {
+     return listVector.getBuffers(clear);
+   }
+
+   @Override
+   public UnionVector promoteToUnion() {
+     return listVector.promoteToUnion();
+   }
+
+   private int lastSet = 0;
+
+   public class Accessor extends LegacyBaseRepeatedAccessor {
+
+      @Override
+      public Object getObject(int index) {
+        return listVector.getObject(index);
+      }
+
+      @Override
+      public boolean isNull(int index) {
+         return listVector.isNull(index);
+      }
+
+      @Override
+      public int getNullCount() {
+         return listVector.getNullCount();
+      }
+   }
+
+   public class Mutator extends LegacyBaseRepeatedMutator {
+      public void setNotNull(int index) {
+        listVector.setNotNull(index);
+      }
+
+      @Override
+      public int startNewValue(int index) {
+         return listVector.startNewValue(index);
+      }
+
+      /**
+       * End the current value
+       *
+       * @param index index of the value to end
+       * @param size  number of elements in the list that was written
+       */
+      public void endValue(int index, int size) {
+         listVector.endValue(index, size);
+      }
+
+      @Override
+      public void setValueCount(int valueCount) {
+        listVector.setValueCount(valueCount);
+      }
+
+      public void setLastSet(int value) {
+         listVector.setLastSet(value);
+      }
+
+      public int getLastSet() {
+         return listVector.getLastSet();
+      }
+   }
+
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyMapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyMapVector.java
new file mode 100644
index 0000000..77556ab
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyMapVector.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.arrow.vector.complex;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Ordering;
+import com.google.common.primitives.Ints;
+
+import io.netty.buffer.ArrowBuf;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.*;
+import org.apache.arrow.vector.complex.impl.SingleMapReaderImpl;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.holders.ComplexHolder;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.JsonStringHashMap;
+import org.apache.arrow.vector.util.TransferPair;
+
+public class LegacyMapVector extends AbstractMapVector {
+
+   /* delegate */
+   private final MapVector mapVector;
+
+   public static LegacyMapVector empty(String name, BufferAllocator allocator) {
+      FieldType fieldType = new FieldType(false, ArrowType.Struct.INSTANCE, null, null);
+      return new LegacyMapVector(name, allocator, fieldType, null);
+   }
+
+   private final Accessor accessor = new Accessor();
+   private final Mutator mutator = new Mutator();
+
+   @Deprecated
+   public LegacyMapVector(String name, BufferAllocator allocator, CallBack callBack) {
+      super(name, allocator, callBack);
+      mapVector = new MapVector(name, allocator, new FieldType(false, ArrowType.Struct.INSTANCE, null, null), callBack);
+   }
+
+   public LegacyMapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
+      super(name, allocator, callBack);
+      mapVector = new MapVector(name, allocator, fieldType, callBack);
+   }
+
+   @Override
+   public FieldReader getReader() {
+      return mapVector.getReader();
+   }
+
+   public void copyFromSafe(int fromIndex, int thisIndex, MapVector from) {
+     mapVector.copyFromSafe(fromIndex, thisIndex, from);
+   }
+
+   @Override
+   protected boolean supportsDirectRead() {
+      return true;
+   }
+
+   public Iterator<String> fieldNameIterator() {
+      return mapVector.fieldNameIterator();
+   }
+
+   @Override
+   public void setInitialCapacity(int numRecords) {
+     mapVector.setInitialCapacity(numRecords);
+   }
+
+   @Override
+   public int getBufferSize() {
+     return mapVector.getBufferSize();
+   }
+
+   @Override
+   public int getBufferSizeFor(final int valueCount) {
+      return (int) mapVector.getBufferSizeFor(valueCount);
+   }
+
+   @Override
+   public ArrowBuf getValidityBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getDataBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public ArrowBuf getOffsetBuffer() {
+      throw new UnsupportedOperationException();
+   }
+
+   @Override
+   public TransferPair getTransferPair(BufferAllocator allocator) {
+      return getTransferPair(name, allocator, null);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
+      return mapVector.getTransferPair(ref, allocator, callBack);
+   }
+
+   @Override
+   public TransferPair makeTransferPair(ValueVector to) {
+      return mapVector.makeTransferPair(((LegacyMapVector)to).mapVector);
+   }
+
+   @Override
+   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+      return mapVector.getTransferPair(ref, allocator);
+   }
+
+   @Override
+   public int getValueCapacity() {
+      return mapVector.getValueCapacity();
+   }
+
+   @Override
+   public Accessor getAccessor() {
+      return accessor;
+   }
+
+   @Override
+   public Mutator getMutator() {
+      return mutator;
+   }
+
+   public class Accessor extends BaseValueVector.BaseAccessor {
+
+      @Override
+      public Object getObject(int index) {
+         return mapVector.getObject(index);
+      }
+
+      public void get(int index, ComplexHolder holder) {
+        mapVector.get(index, holder);
+      }
+
+      @Override
+      public int getValueCount() {
+         return mapVector.getValueCount();
+      }
+   }
+
+   public ValueVector getVectorById(int id) {
+      return mapVector.getVectorById(id);
+   }
+
+   public class Mutator extends BaseValueVector.BaseMutator {
+
+      @Override
+      public void setValueCount(int valueCount) {
+        mapVector.setValueCount(valueCount);
+      }
+
+      @Override
+      public void reset() { }
+
+      @Override
+      public void generateTestData(int values) { }
+   }
+
+   @Override
+   public void clear() {
+     mapVector.clear();
+   }
+
+   @Override
+   public Field getField() {
+     return mapVector.getField();
+   }
+
+   @Override
+   public MinorType getMinorType() {
+      return MinorType.MAP;
+   }
+
+   @Override
+   public void close() {
+      mapVector.close();
+   }
+
+   public void initializeChildrenFromFields(List<Field> children) {
+      mapVector.initializeChildrenFromFields(children);
+   }
+
+   public List<FieldVector> getChildrenFromFields() {
+      return mapVector.getChildren();
+   }
+
+   public boolean isNull(int index) { return false; }
+
+   public int getNullCount() { return  0; }
+
+   public int getValueCount() { return 0; }
+
+   public void setValueCount(int valueCount) { }
+
+   public Object getObject(int index) { return null; }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
index 4b2c913..ea57720 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
@@ -22,26 +22,22 @@ package org.apache.arrow.vector.complex;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static java.util.Collections.singletonList;
 
-import java.util.Arrays;
-import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ObjectArrays;
 
 import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.OutOfMemoryException;
 import org.apache.arrow.vector.AddOrGetResult;
-import org.apache.arrow.vector.BaseDataValueVector;
-import org.apache.arrow.vector.BitVector;
 import org.apache.arrow.vector.BufferBacked;
 import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.UInt4Vector;
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.ZeroVector;
-import org.apache.arrow.vector.NullableIntVector;
-import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.BitVectorHelper;
 import org.apache.arrow.vector.complex.impl.ComplexCopier;
 import org.apache.arrow.vector.complex.impl.UnionListReader;
 import org.apache.arrow.vector.complex.impl.UnionListWriter;
@@ -55,6 +51,7 @@ import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.JsonStringArrayList;
+import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
 public class ListVector extends BaseRepeatedValueVector implements FieldVector, PromotableVector {
@@ -63,14 +60,12 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return new ListVector(name, allocator, FieldType.nullable(ArrowType.List.INSTANCE), null);
   }
 
-  final UInt4Vector offsets;
-  final BitVector bits;
-  private final List<BufferBacked> innerVectors;
-  private Mutator mutator = new Mutator();
-  private Accessor accessor = new Accessor();
+  private ArrowBuf validityBuffer;
   private UnionListReader reader;
   private CallBack callBack;
   private final FieldType fieldType;
+  private int validityAllocationSizeInBytes;
+  private int lastSet;
 
   // deprecated, use FieldType or static constructor instead
   @Deprecated
@@ -86,12 +81,12 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   public ListVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
     super(name, allocator, callBack);
-    this.bits = new BitVector("$bits$", allocator);
-    this.offsets = getOffsetVector();
-    this.innerVectors = Collections.unmodifiableList(Arrays.<BufferBacked>asList(bits, offsets));
+    this.validityBuffer = allocator.getEmpty();
     this.reader = new UnionListReader(this);
     this.fieldType = checkNotNull(fieldType);
     this.callBack = callBack;
+    this.validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
+    this.lastSet = 0;
   }
 
   @Override
@@ -115,20 +110,43 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    // variable width values: truncate offset vector buffer to size (#1)
-    org.apache.arrow.vector.BaseDataValueVector.truncateBufferBasedOnSize(ownBuffers, 1, offsets.getBufferSizeFor(fieldNode.getLength() + 1));
-    BaseDataValueVector.load(fieldNode, getFieldInnerVectors(), ownBuffers);
+    if (ownBuffers.size() != 2) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 2 + ", got: " + ownBuffers.size());
+    }
+
+    ArrowBuf bitBuffer = ownBuffers.get(0);
+    ArrowBuf offBuffer = ownBuffers.get(1);
+
+    validityBuffer.release();
+    validityBuffer = bitBuffer.retain(allocator);
+    offsetBuffer.release();
+    offsetBuffer = offBuffer.retain(allocator);
+
+    validityAllocationSizeInBytes = validityBuffer.capacity();
+    offsetAllocationSizeInBytes = offsetBuffer.capacity();
+
     lastSet = fieldNode.getLength();
+    valueCount = fieldNode.getLength();
   }
 
   @Override
   public List<ArrowBuf> getFieldBuffers() {
-    return BaseDataValueVector.unload(getFieldInnerVectors());
+    List<ArrowBuf> result = new ArrayList<>(2);
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+    offsetBuffer.readerIndex(0);
+    offsetBuffer.writerIndex((valueCount + 1) * OFFSET_WIDTH);
+
+    result.add(validityBuffer);
+    result.add(offsetBuffer);
+
+    return result;
   }
 
   @Override
+  @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
-    return innerVectors;
+    throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
   }
 
   public UnionListWriter getWriter() {
@@ -137,14 +155,70 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public void allocateNew() throws OutOfMemoryException {
-    super.allocateNewSafe();
-    bits.allocateNewSafe();
+   if (!allocateNewSafe()) {
+     throw new OutOfMemoryException("Failure while allocating memory");
+   }
+  }
+
+  public boolean allocateNewSafe() {
+    boolean success = false;
+    try {
+      /* allocate validity buffer */
+      allocateValidityBuffer(validityAllocationSizeInBytes);
+      /* allocate offset and data buffer */
+      success = super.allocateNewSafe();
+    } finally {
+      if (!success) {
+        clear();
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private void allocateValidityBuffer(final long size) {
+    final int curSize = (int)size;
+    validityBuffer = allocator.buffer(curSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = curSize;
+    validityBuffer.setZero(0, validityBuffer.capacity());
   }
 
   @Override
   public void reAlloc() {
+    /* reallocate the validity buffer */
+    reallocValidityBuffer();
+    /* reallocate the offset and data */
     super.reAlloc();
-    bits.reAlloc();
+  }
+
+  private void reallocValidityAndOffsetBuffers() {
+    reallocOffsetBuffer();
+    reallocValidityBuffer();
+  }
+
+  private void reallocValidityBuffer() {
+    final int currentBufferCapacity = validityBuffer.capacity();
+    long baseSize = validityAllocationSizeInBytes;
+
+    if (baseSize < (long)currentBufferCapacity) {
+      baseSize = (long)currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity);
+    final int halfNewCapacity = newBuf.capacity() / 2;
+    newBuf.setZero(halfNewCapacity, halfNewCapacity);
+    validityBuffer.release(1);
+    validityBuffer = newBuf;
+    validityAllocationSizeInBytes = (int)newAllocationSize;
   }
 
   public void copyFromSafe(int inIndex, int outIndex, ListVector from) {
@@ -181,7 +255,7 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public long getValidityBufferAddress() {
-    return (bits.getDataBuffer().memoryAddress());
+    return (validityBuffer.memoryAddress());
   }
 
   @Override
@@ -191,11 +265,11 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public long getOffsetBufferAddress() {
-    return (offsets.getDataBuffer().memoryAddress());
+    return (offsetBuffer.memoryAddress());
   }
 
   @Override
-  public ArrowBuf getValidityBuffer() { return bits.getDataBuffer(); }
+  public ArrowBuf getValidityBuffer() { return validityBuffer; }
 
   @Override
   public ArrowBuf getDataBuffer() {
@@ -203,17 +277,13 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   @Override
-  public ArrowBuf getOffsetBuffer() { return offsets.getDataBuffer(); }
+  public ArrowBuf getOffsetBuffer() { return offsetBuffer; }
 
   private class TransferImpl implements TransferPair {
 
     ListVector to;
-    TransferPair bitsTransferPair;
-    TransferPair offsetsTransferPair;
     TransferPair dataTransferPair;
 
-    TransferPair[] pairs;
-
     public TransferImpl(String name, BufferAllocator allocator, CallBack callBack) {
       this(new ListVector(name, allocator, fieldType, callBack));
     }
@@ -221,39 +291,99 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     public TransferImpl(ListVector to) {
       this.to = to;
       to.addOrGetVector(vector.getField().getFieldType());
-      offsetsTransferPair = offsets.makeTransferPair(to.offsets);
-      bitsTransferPair = bits.makeTransferPair(to.bits);
       if (to.getDataVector() instanceof ZeroVector) {
         to.addOrGetVector(vector.getField().getFieldType());
       }
       dataTransferPair = getDataVector().makeTransferPair(to.getDataVector());
-      pairs = new TransferPair[] {bitsTransferPair, offsetsTransferPair, dataTransferPair};
     }
 
     @Override
     public void transfer() {
-      for (TransferPair pair : pairs) {
-        pair.transfer();
-      }
+      dataTransferPair.transfer();
+      to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer;
+      to.offsetBuffer = offsetBuffer.transferOwnership(to.allocator).buffer;
       to.lastSet = lastSet;
+      to.setValueCount(valueCount);
     }
 
     @Override
     public void splitAndTransfer(int startIndex, int length) {
-      UInt4Vector.Accessor offsetVectorAccessor = ListVector.this.offsets.getAccessor();
-      final int startPoint = offsetVectorAccessor.get(startIndex);
-      final int sliceLength = offsetVectorAccessor.get(startIndex + length) - startPoint;
+      final int startPoint = offsetBuffer.getInt(startIndex * OFFSET_WIDTH);
+      final int sliceLength = offsetBuffer.getInt((startIndex + length) * OFFSET_WIDTH) - startPoint;
       to.clear();
-      to.offsets.allocateNew(length + 1);
-      offsetVectorAccessor = ListVector.this.offsets.getAccessor();
-      final UInt4Vector.Mutator targetOffsetVectorMutator = to.offsets.getMutator();
+      to.allocateOffsetBuffer((length + 1) * OFFSET_WIDTH);
+      /* splitAndTransfer offset buffer */
       for (int i = 0; i < length + 1; i++) {
-        targetOffsetVectorMutator.set(i, offsetVectorAccessor.get(startIndex + i) - startPoint);
+        final int relativeOffset = offsetBuffer.getInt((startIndex + i) * OFFSET_WIDTH) - startPoint;
+        to.offsetBuffer.setInt(i * OFFSET_WIDTH, relativeOffset);
       }
-      bitsTransferPair.splitAndTransfer(startIndex, length);
+      /* splitAndTransfer validity buffer */
+      splitAndTransferValidityBuffer(startIndex, length, to);
+      /* splitAndTransfer data buffer */
       dataTransferPair.splitAndTransfer(startPoint, sliceLength);
       to.lastSet = length;
-      to.mutator.setValueCount(length);
+      to.setValueCount(length);
+    }
+
+    /*
+     * transfer the validity.
+     */
+    private void splitAndTransferValidityBuffer(int startIndex, int length, ListVector target) {
+      assert startIndex + length <= valueCount;
+      int firstByteSource = BitVectorHelper.byteIndex(startIndex);
+      int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
+      int byteSizeTarget = getValidityBufferSizeFromCount(length);
+      int offset = startIndex % 8;
+
+      if (length > 0) {
+        if (offset == 0) {
+          // slice
+          if (target.validityBuffer != null) {
+            target.validityBuffer.release();
+          }
+          target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
+          target.validityBuffer.retain(1);
+        }
+        else {
+          /* Copy data
+           * When the first bit starts from the middle of a byte (offset != 0),
+           * copy data from src BitVector.
+           * Each byte in the target is composed by a part in i-th byte,
+           * another part in (i+1)-th byte.
+           */
+          target.allocateValidityBuffer(byteSizeTarget);
+
+          for (int i = 0; i < byteSizeTarget - 1; i++) {
+            byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer, firstByteSource + i, offset);
+            byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer, firstByteSource + i + 1, offset);
+
+            target.validityBuffer.setByte(i, (b1 + b2));
+          }
+
+          /* Copying the last piece is done in the following manner:
+           * if the source vector has 1 or more bytes remaining, we copy
+           * the last piece as a byte formed by shifting data
+           * from the current byte and the next byte.
+           *
+           * if the source vector has no more bytes remaining
+           * (we are at the last byte), we copy the last piece as a byte
+           * by shifting data from the current byte.
+           */
+          if((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
+            byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                    firstByteSource + byteSizeTarget - 1, offset);
+            byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer,
+                    firstByteSource + byteSizeTarget, offset);
+
+            target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+          }
+          else {
+            byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                    firstByteSource + byteSizeTarget - 1, offset);
+            target.validityBuffer.setByte(byteSizeTarget - 1, b1);
+          }
+        }
+      }
     }
 
     @Override
@@ -268,46 +398,10 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   @Override
-  public Accessor getAccessor() {
-    return accessor;
-  }
-
-  @Override
-  public Mutator getMutator() {
-    return mutator;
-  }
-
-  @Override
   public UnionListReader getReader() {
     return reader;
   }
 
-  @Override
-  public boolean allocateNewSafe() {
-    /* boolean to keep track if all the memory allocation were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    boolean success = false;
-    try {
-      if (!offsets.allocateNewSafe()) {
-        return false;
-      }
-      success = vector.allocateNewSafe();
-      success = success && bits.allocateNewSafe();
-    } finally {
-      if (!success) {
-        clear();
-      }
-    }
-    if (success) {
-      offsets.zeroVector();
-      bits.zeroVector();
-    }
-    return success;
-  }
-
   public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(FieldType fieldType) {
     AddOrGetResult<T> result = super.addOrGetVector(fieldType);
     reader = new UnionListReader(this);
@@ -316,10 +410,12 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public int getBufferSize() {
-    if (getAccessor().getValueCount() == 0) {
+    if (getValueCount() == 0) {
       return 0;
     }
-    return offsets.getBufferSize() + bits.getBufferSize() + vector.getBufferSize();
+    final int offsetBufferSize = (valueCount + 1) * OFFSET_WIDTH;
+    final int validityBufferSize = getValidityBufferSizeFromCount(valueCount);
+    return offsetBufferSize + validityBufferSize + vector.getBufferSize();
   }
 
   @Override
@@ -334,16 +430,14 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
 
   @Override
   public void clear() {
-    offsets.clear();
-    vector.clear();
-    bits.clear();
-    lastSet = 0;
     super.clear();
+    validityBuffer = releaseBuffer(validityBuffer);
+    lastSet = 0;
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(offsets.getBuffers(false), ObjectArrays.concat(bits.getBuffers(false),
+    final ArrowBuf[] buffers = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer}, ObjectArrays.concat(new ArrowBuf[] {validityBuffer},
         vector.getBuffers(false), ArrowBuf.class), ArrowBuf.class);
     if (clear) {
       for (ArrowBuf buffer : buffers) {
@@ -365,99 +459,97 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return vector;
   }
 
-  private int lastSet = 0;
+  @Override
+  public Object getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    }
+    final List<Object> vals = new JsonStringArrayList<>();
+    final int start = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    final int end = offsetBuffer.getInt((index + 1) * OFFSET_WIDTH);
+    final ValueVector vv = getDataVector();
+    for (int i = start; i < end; i++) {
+      vals.add(vv.getObject(i));
+    }
 
-  public class Accessor extends BaseRepeatedAccessor {
+    return vals;
+  }
 
-    @Override
-    public Object getObject(int index) {
-      if (isNull(index)) {
-        return null;
-      }
-      final List<Object> vals = new JsonStringArrayList<>();
-      final UInt4Vector.Accessor offsetsAccessor = offsets.getAccessor();
-      final int start = offsetsAccessor.get(index);
-      final int end = offsetsAccessor.get(index + 1);
-      final ValueVector vv = getDataVector();
-      if (vv instanceof  NullableIntVector || vv instanceof NullableVarCharVector) {
-        for (int i = start; i < end; i++) {
-          vals.add(vv.getObject(i));
-        }
-      } else {
-        final ValueVector.Accessor valuesAccessor = vv.getAccessor();
-        for (int i = start; i < end; i++) {
-          vals.add(valuesAccessor.getObject(i));
-        }
-      }
+  @Override
+  public boolean isNull(int index) {
+    return (isSet(index) == 0);
+  }
 
-      return vals;
-    }
+  public int isSet(int index) {
+    final int byteIndex = index >> 3;
+    final byte b = validityBuffer.getByte(byteIndex);
+    final int bitIndex = index & 7;
+    return Long.bitCount(b & (1L << bitIndex));
+  }
 
-    @Override
-    public boolean isNull(int index) {
-      return bits.getAccessor().get(index) == 0;
-    }
+  @Override
+  public int getNullCount() {
+    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
+  }
 
-    @Override
-    public int getNullCount() {
-      return bits.getAccessor().getNullCount();
+  public void setNotNull(int index) {
+    if (index >= getValueCapacity()) {
+      reallocValidityAndOffsetBuffers();
     }
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    lastSet = index + 1;
   }
 
-  public class Mutator extends BaseRepeatedMutator {
-    public void setNotNull(int index) {
-      bits.getMutator().setSafe(index, 1);
-      lastSet = index + 1;
+  @Override
+  public int startNewValue(int index) {
+    if (index >= getValueCapacity()) {
+      reallocValidityAndOffsetBuffers();
     }
-
-    @Override
-    public int startNewValue(int index) {
-      for (int i = lastSet; i <= index; i++) {
-        offsets.getMutator().setSafe(i + 1, offsets.getAccessor().get(i));
-      }
-      setNotNull(index);
-      lastSet = index + 1;
-      return offsets.getAccessor().get(lastSet);
+    for (int i = lastSet; i <= index; i++) {
+      final int currentOffset = offsetBuffer.getInt(i * OFFSET_WIDTH);
+      offsetBuffer.setInt((i + 1) * OFFSET_WIDTH, currentOffset);
     }
+    setNotNull(index);
+    lastSet = index + 1;
+    return offsetBuffer.getInt(lastSet * OFFSET_WIDTH);
+  }
 
-    /**
-     * End the current value
-     *
-     * @param index index of the value to end
-     * @param size  number of elements in the list that was written
-     */
-    public void endValue(int index, int size) {
-      offsets.getMutator().set(index + 1, offsets.getAccessor().get(index + 1) + size);
-    }
+  /**
+   * End the current value
+   *
+   * @param index index of the value to end
+   * @param size  number of elements in the list that was written
+   */
+  public void endValue(int index, int size) {
+    final int currentOffset = offsetBuffer.getInt((index + 1) * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, currentOffset + size);
+  }
 
-    @Override
-    public void setValueCount(int valueCount) {
-      // TODO: populate offset end points
-      if (valueCount == 0) {
-        offsets.getMutator().setValueCount(0);
-      } else {
-        for (int i = lastSet; i < valueCount; i++) {
-          offsets.getMutator().setSafe(i + 1, offsets.getAccessor().get(i));
-        }
-        offsets.getMutator().setValueCount(valueCount + 1);
+  @Override
+  public void setValueCount(int valueCount) {
+    this.valueCount = valueCount;
+    if (valueCount > 0) {
+      while (valueCount > getValueCapacity()) {
+        /* realloc the inner buffers if needed */
+        reallocValidityAndOffsetBuffers();
       }
-      final int childValueCount = valueCount == 0 ? 0 : offsets.getAccessor().get(valueCount);
-      if (vector instanceof NullableIntVector || vector instanceof NullableVarCharVector) {
-        vector.setValueCount(childValueCount);
-      } else {
-        vector.getMutator().setValueCount(childValueCount);
+      for (int i = lastSet; i < valueCount; i++) {
+        /* fill the holes with offsets */
+        final int currentOffset = offsetBuffer.getInt(i * OFFSET_WIDTH);
+        offsetBuffer.setInt((i + 1) * OFFSET_WIDTH, currentOffset);
       }
-
-      bits.getMutator().setValueCount(valueCount);
-    }
-
-    public void setLastSet(int value) {
-      lastSet = value;
     }
+    /* valueCount for the data vector is the current end offset */
+    final int childValueCount = (valueCount == 0) ? 0 :
+            offsetBuffer.getInt(valueCount * OFFSET_WIDTH);
+    vector.setValueCount(childValueCount);
+  }
 
-    public int getLastSet() {
-      return lastSet;
-    }
+  public void setLastSet(int value) {
+    lastSet = value;
   }
 
+  public int getLastSet() {
+    return lastSet;
+  }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
index 95efa60..6089a67 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
@@ -56,8 +56,6 @@ public class MapVector extends AbstractMapVector {
   }
 
   private final SingleMapReaderImpl reader = new SingleMapReaderImpl(this);
-  private final Accessor accessor = new Accessor();
-  private final Mutator mutator = new Mutator();
   protected final FieldType fieldType;
   public int valueCount;
 
@@ -70,6 +68,7 @@ public class MapVector extends AbstractMapVector {
   public MapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
     super(name, allocator, callBack);
     this.fieldType = checkNotNull(fieldType);
+    this.valueCount = 0;
   }
 
   @Override
@@ -230,7 +229,7 @@ public class MapVector extends AbstractMapVector {
       for (TransferPair p : pairs) {
         p.splitAndTransfer(startIndex, length);
       }
-      to.getMutator().setValueCount(length);
+      to.setValueCount(length);
     }
   }
 
@@ -255,76 +254,54 @@ public class MapVector extends AbstractMapVector {
 
   @Override
   public Accessor getAccessor() {
-    return accessor;
+    throw new UnsupportedOperationException("accessor is not needed for MAP");
   }
 
   @Override
   public Mutator getMutator() {
-    return mutator;
+    throw new UnsupportedOperationException("mutator is not needed for MAP");
   }
 
-  public class Accessor extends BaseValueVector.BaseAccessor {
-
-    @Override
-    public Object getObject(int index) {
-      Map<String, Object> vv = new JsonStringHashMap<>();
-      for (String child : getChildFieldNames()) {
-        ValueVector v = getChild(child);
-        if (v instanceof  NullableVarCharVector || v instanceof  NullableIntVector) {
-          if (v != null && index < v.getValueCount()) {
-            Object value = v.getObject(index);
-            if (value != null) {
-              vv.put(child, value);
-            }
-          }
-        } else {
-          if (v != null && index < v.getAccessor().getValueCount()) {
-            Object value = v.getAccessor().getObject(index);
-            if (value != null) {
-              vv.put(child, value);
-            }
-          }
+  @Override
+  public Object getObject(int index) {
+    Map<String, Object> vv = new JsonStringHashMap<>();
+    for (String child : getChildFieldNames()) {
+      ValueVector v = getChild(child);
+      if (v != null && index < v.getValueCount()) {
+        Object value = v.getObject(index);
+        if (value != null) {
+          vv.put(child, value);
         }
       }
-      return vv;
     }
+    return vv;
+  }
 
-    public void get(int index, ComplexHolder holder) {
-      reader.setPosition(index);
-      holder.reader = reader;
-    }
+  @Override
+  public boolean isNull(int index) { return false; }
+  @Override
+  public int getNullCount() { return 0; }
 
-    @Override
-    public int getValueCount() {
-      return valueCount;
-    }
+  public void get(int index, ComplexHolder holder) {
+    reader.setPosition(index);
+    holder.reader = reader;
   }
 
-  public ValueVector getVectorById(int id) {
-    return getChildByOrdinal(id);
+  @Override
+  public int getValueCount() {
+    return valueCount;
   }
 
-  public class Mutator extends BaseValueVector.BaseMutator {
-
-    @Override
-    public void setValueCount(int valueCount) {
-      for (final ValueVector v : getChildren()) {
-        if (v instanceof NullableIntVector || v instanceof NullableVarCharVector) {
-          v.setValueCount(valueCount);
-        } else {
-          v.getMutator().setValueCount(valueCount);
-        }
-      }
-      MapVector.this.valueCount = valueCount;
-    }
-
-    @Override
-    public void reset() {
-    }
+  public ValueVector getVectorById(int id) {
+  return getChildByOrdinal(id);
+}
 
-    @Override
-    public void generateTestData(int values) {
+  @Override
+  public void setValueCount(int valueCount) {
+    for (final ValueVector v : getChildren()) {
+      v.setValueCount(valueCount);
     }
+    MapVector.this.valueCount = valueCount;
   }
 
   @Override
@@ -372,11 +349,4 @@ public class MapVector extends AbstractMapVector {
   public List<FieldVector> getChildrenFromFields() {
     return getChildren();
   }
-
-  public int getValueCount() { return 0; }
-
-  public void setValueCount(int valueCount) { }
-
-  public Object getObject(int index) { return null; }
-
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/NullableMapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/NullableMapVector.java
index 1c9d5aa..a1478c5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/NullableMapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/NullableMapVector.java
@@ -20,6 +20,7 @@ package org.apache.arrow.vector.complex;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -27,13 +28,9 @@ import java.util.List;
 import com.google.common.collect.ObjectArrays;
 
 import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.BaseDataValueVector;
-import org.apache.arrow.vector.BitVector;
-import org.apache.arrow.vector.BufferBacked;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.NullableVectorDefinitionSetter;
-import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.*;
 import org.apache.arrow.vector.complex.impl.NullableMapReaderImpl;
 import org.apache.arrow.vector.complex.impl.NullableMapWriter;
 import org.apache.arrow.vector.holders.ComplexHolder;
@@ -44,6 +41,7 @@ import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.util.CallBack;
+import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
 public class NullableMapVector extends MapVector implements FieldVector {
@@ -56,12 +54,8 @@ public class NullableMapVector extends MapVector implements FieldVector {
   private final NullableMapReaderImpl reader = new NullableMapReaderImpl(this);
   private final NullableMapWriter writer = new NullableMapWriter(this);
 
-  protected final BitVector bits;
-
-  private final List<BufferBacked> innerVectors;
-
-  private final Accessor accessor;
-  private final Mutator mutator;
+  private ArrowBuf validityBuffer;
+  private int validityAllocationSizeInBytes;
 
   // deprecated, use FieldType or static constructor instead
   @Deprecated
@@ -77,10 +71,8 @@ public class NullableMapVector extends MapVector implements FieldVector {
 
   public NullableMapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
     super(name, checkNotNull(allocator), fieldType, callBack);
-    this.bits = new BitVector("$bits$", allocator);
-    this.innerVectors = Collections.unmodifiableList(Arrays.<BufferBacked>asList(bits));
-    this.accessor = new Accessor();
-    this.mutator = new Mutator();
+    this.validityBuffer = allocator.getEmpty();
+    this.validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(BaseValueVector.INITIAL_VALUE_ALLOCATION);
   }
 
   @Override
@@ -92,18 +84,33 @@ public class NullableMapVector extends MapVector implements FieldVector {
 
   @Override
   public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    BaseDataValueVector.load(fieldNode, getFieldInnerVectors(), ownBuffers);
-    this.valueCount = fieldNode.getLength();
+    if (ownBuffers.size() != 1) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 1 + ", got: " + ownBuffers.size());
+    }
+
+    ArrowBuf bitBuffer = ownBuffers.get(0);
+
+    validityBuffer.release();
+    validityBuffer = bitBuffer.retain(allocator);
+    valueCount = fieldNode.getLength();
+    validityAllocationSizeInBytes = validityBuffer.capacity();
   }
 
   @Override
   public List<ArrowBuf> getFieldBuffers() {
-    return BaseDataValueVector.unload(getFieldInnerVectors());
+    List<ArrowBuf> result = new ArrayList<>(1);
+
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(BitVectorHelper.getValidityBufferSize(valueCount));
+    result.add(validityBuffer);
+
+    return result;
   }
 
   @Override
+  @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
-    return innerVectors;
+    throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
   }
 
   @Override
@@ -146,49 +153,129 @@ public class NullableMapVector extends MapVector implements FieldVector {
 
     @Override
     public void transfer() {
-      bits.transferTo(target.bits);
+      target.clear();
+      target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
       super.transfer();
     }
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      target.bits.copyFromSafe(fromIndex, toIndex, bits);
+      while (toIndex >= target.getValidityBufferValueCapacity()) {
+        target.reallocValidityBuffer();
+      }
+      BitVectorHelper.setValidityBit(target.validityBuffer, toIndex, isSet(fromIndex));
       super.copyValueSafe(fromIndex, toIndex);
     }
 
     @Override
     public void splitAndTransfer(int startIndex, int length) {
-      bits.splitAndTransferTo(startIndex, length, target.bits);
+      target.clear();
+      splitAndTransferValidityBuffer(startIndex, length, target);
       super.splitAndTransfer(startIndex, length);
     }
   }
 
+  /*
+   * transfer the validity.
+   */
+  private void splitAndTransferValidityBuffer(int startIndex, int length, NullableMapVector target) {
+    assert startIndex + length <= valueCount;
+    int firstByteSource = BitVectorHelper.byteIndex(startIndex);
+    int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
+    int byteSizeTarget = BitVectorHelper.getValidityBufferSize(length);
+    int offset = startIndex % 8;
+
+    if (length > 0) {
+      if (offset == 0) {
+        // slice
+        if (target.validityBuffer != null) {
+          target.validityBuffer.release();
+        }
+        target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
+        target.validityBuffer.retain(1);
+      }
+      else {
+        /* Copy data
+         * When the first bit starts from the middle of a byte (offset != 0),
+         * copy data from src BitVector.
+         * Each byte in the target is composed by a part in i-th byte,
+         * another part in (i+1)-th byte.
+         */
+        target.allocateValidityBuffer(byteSizeTarget);
+
+        for (int i = 0; i < byteSizeTarget - 1; i++) {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer, firstByteSource + i, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer, firstByteSource + i + 1, offset);
+
+          target.validityBuffer.setByte(i, (b1 + b2));
+        }
+
+        /* Copying the last piece is done in the following manner:
+         * if the source vector has 1 or more bytes remaining, we copy
+         * the last piece as a byte formed by shifting data
+         * from the current byte and the next byte.
+         *
+         * if the source vector has no more bytes remaining
+         * (we are at the last byte), we copy the last piece as a byte
+         * by shifting data from the current byte.
+         */
+        if((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer,
+                  firstByteSource + byteSizeTarget, offset);
+
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+        }
+        else {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1);
+        }
+      }
+    }
+  }
+
+  private int getValidityBufferValueCapacity() {
+    return (int)(validityBuffer.capacity() * 8L);
+  }
+
   @Override
   public int getValueCapacity() {
-    return Math.min(bits.getValueCapacity(), super.getValueCapacity());
+    return Math.min(getValidityBufferValueCapacity(),
+            super.getValueCapacity());
   }
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    return ObjectArrays.concat(bits.getBuffers(clear), super.getBuffers(clear), ArrowBuf.class);
+    if (clear) {
+      validityBuffer.retain(1);
+    }
+    return ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, super.getBuffers(clear), ArrowBuf.class);
   }
 
   @Override
   public void close() {
-    bits.close();
+    clearValidityBuffer();
     super.close();
   }
 
   @Override
   public void clear() {
-    bits.clear();
+    clearValidityBuffer();
     super.clear();
   }
 
+  private void clearValidityBuffer() {
+    validityBuffer.release();
+    validityBuffer = allocator.getEmpty();
+  }
 
   @Override
   public int getBufferSize() {
-    return super.getBufferSize() + bits.getBufferSize();
+    if (valueCount == 0) { return 0; }
+    return super.getBufferSize() +
+            BitVectorHelper.getValidityBufferSize(valueCount);
   }
 
   @Override
@@ -197,12 +284,12 @@ public class NullableMapVector extends MapVector implements FieldVector {
       return 0;
     }
     return super.getBufferSizeFor(valueCount)
-        + bits.getBufferSizeFor(valueCount);
+        + BitVectorHelper.getValidityBufferSize(valueCount);
   }
 
   @Override
   public void setInitialCapacity(int numRecords) {
-    bits.setInitialCapacity(numRecords);
+    validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(numRecords);
     super.setInitialCapacity(numRecords);
   }
 
@@ -215,25 +302,59 @@ public class NullableMapVector extends MapVector implements FieldVector {
      */
     boolean success = false;
     try {
-      success = super.allocateNewSafe() && bits.allocateNewSafe();
+      clearValidityBuffer();
+      allocateValidityBuffer(validityAllocationSizeInBytes);
+      success = super.allocateNewSafe();
     } finally {
       if (!success) {
         clear();
+        return false;
       }
     }
-    bits.zeroVector();
-    return success;
+    return true;
+  }
+
+  private void allocateValidityBuffer(final long size) {
+    final int curSize = (int)size;
+    validityBuffer = allocator.buffer(curSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = curSize;
+    validityBuffer.setZero(0, validityBuffer.capacity());
   }
 
   @Override
   public void reAlloc() {
-    bits.reAlloc();
+    /* reallocate the validity buffer */
+    reallocValidityBuffer();
     super.reAlloc();
   }
 
+  private void reallocValidityBuffer() {
+    final int currentBufferCapacity = validityBuffer.capacity();
+    long baseSize = validityAllocationSizeInBytes;
+
+    if (baseSize < (long)currentBufferCapacity) {
+      baseSize = (long)currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > BaseValueVector.MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    newBuf.setZero(0, newBuf.capacity());
+    newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity);
+    validityBuffer.release(1);
+    validityBuffer = newBuf;
+    validityAllocationSizeInBytes = (int)newAllocationSize;
+  }
+
   @Override
   public long getValidityBufferAddress() {
-    return bits.getBuffer().memoryAddress();
+    return validityBuffer.memoryAddress();
   }
 
   @Override
@@ -248,7 +369,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
 
   @Override
   public ArrowBuf getValidityBuffer() {
-    return bits.getDataBuffer();
+    return validityBuffer;
   }
 
   @Override
@@ -261,82 +382,76 @@ public class NullableMapVector extends MapVector implements FieldVector {
     throw new UnsupportedOperationException();
   }
 
-  public final class Accessor extends MapVector.Accessor {
-    final BitVector.Accessor bAccessor = bits.getAccessor();
-
-    @Override
-    public Object getObject(int index) {
-      if (isNull(index)) {
-        return null;
-      } else {
-        return super.getObject(index);
-      }
-    }
-
-    @Override
-    public void get(int index, ComplexHolder holder) {
-      holder.isSet = isSet(index);
-      super.get(index, holder);
-    }
-
-    @Override
-    public int getNullCount() {
-      return bits.getAccessor().getNullCount();
-    }
-
-    @Override
-    public boolean isNull(int index) {
-      return isSet(index) == 0;
-    }
-
-    public int isSet(int index) {
-      return bAccessor.get(index);
+  @Override
+  public Object getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
+      return super.getObject(index);
     }
-
   }
 
-  public final class Mutator extends MapVector.Mutator implements NullableVectorDefinitionSetter {
+  @Override
+  public void get(int index, ComplexHolder holder) {
+    holder.isSet = isSet(index);
+    super.get(index, holder);
+  }
 
-    private Mutator() {
-    }
+  public int getNullCount() {
+    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
+  }
 
-    @Override
-    public void setIndexDefined(int index) {
-      bits.getMutator().setSafe(index, 1);
-    }
+  public boolean isNull(int index) {
+    return isSet(index) == 0;
+  }
 
-    public void setNull(int index) {
-      bits.getMutator().setSafe(index, 0);
-    }
+  public int isSet(int index) {
+    final int byteIndex = index >> 3;
+    final byte b = validityBuffer.getByte(byteIndex);
+    final int bitIndex = index & 7;
+    return Long.bitCount(b & (1L << bitIndex));
+  }
 
-    @Override
-    public void setValueCount(int valueCount) {
-      assert valueCount >= 0;
-      super.setValueCount(valueCount);
-      bits.getMutator().setValueCount(valueCount);
+  public void setIndexDefined(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
     }
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+  }
 
-    @Override
-    public void generateTestData(int valueCount) {
-      super.generateTestData(valueCount);
-      bits.getMutator().generateTestDataAlt(valueCount);
+  public void setNull(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
     }
+    BitVectorHelper.setValidityBit(validityBuffer, index, 0);
+  }
 
-    @Override
-    public void reset() {
-      bits.getMutator().setValueCount(0);
+  @Override
+  public void setValueCount(int valueCount) {
+    assert valueCount >= 0;
+    while (valueCount > getValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
     }
+    super.setValueCount(valueCount);
+    this.valueCount = valueCount;
+  }
 
+  public void reset() {
+    valueCount = 0;
   }
 
   @Override
+  @Deprecated
   public Accessor getAccessor() {
-    return accessor;
+    throw new UnsupportedOperationException("Accessor is not supported for reading from Nullable MAP");
   }
 
   @Override
+  @Deprecated
   public Mutator getMutator() {
-    return mutator;
+    throw new UnsupportedOperationException("Mutator is not supported for writing to Nullable MAP");
   }
-
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
index 614c266..06b0f4d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
@@ -52,6 +52,6 @@ public class NullableMapReaderImpl extends SingleMapReaderImpl {
 
   @Override
   public boolean isSet() {
-    return !nullableMapVector.getAccessor().isNull(idx());
+    return !nullableMapVector.isNull(idx());
   }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
index 9722196..5bd439c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
@@ -162,7 +162,7 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
     writer = new UnionWriter(unionVector, nullableMapWriterFactory);
     writer.setPosition(idx());
     for (int i = 0; i <= idx(); i++) {
-      unionVector.getMutator().setType(i, vector.getMinorType());
+      unionVector.setType(i, vector.getMinorType());
     }
     vector = null;
     state = State.UNION;
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
index 3ebd0cd..c77ca4e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
@@ -78,7 +78,7 @@ public class SingleMapReaderImpl extends AbstractFieldReader {
 
   @Override
   public Object readObject() {
-    return vector.getAccessor().getObject(idx());
+    return vector.getObject(idx());
   }
 
   @Override
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionFixedSizeListReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionFixedSizeListReader.java
index f3e9b87..4ad2f6f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionFixedSizeListReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionFixedSizeListReader.java
@@ -46,7 +46,7 @@ public class UnionFixedSizeListReader extends AbstractFieldReader {
 
   @Override
   public boolean isSet() {
-    return !vector.getAccessor().isNull(idx());
+    return !vector.isNull(idx());
   }
 
   @Override
@@ -56,7 +56,7 @@ public class UnionFixedSizeListReader extends AbstractFieldReader {
 
   @Override
   public Object readObject() {
-    return vector.getAccessor().getObject(idx());
+    return vector.getObject(idx());
   }
 
   @Override
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
index b98c36d..6243a28 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
@@ -19,6 +19,7 @@
 
 package org.apache.arrow.vector.complex.impl;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.vector.UInt4Vector;
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.complex.ListVector;
@@ -33,12 +34,11 @@ public class UnionListReader extends AbstractFieldReader {
 
   private ListVector vector;
   private ValueVector data;
-  private UInt4Vector offsets;
+  private static final int OFFSET_WIDTH = 4;
 
   public UnionListReader(ListVector vector) {
     this.vector = vector;
     this.data = vector.getDataVector();
-    this.offsets = vector.getOffsetVector();
   }
 
   @Override
@@ -48,7 +48,7 @@ public class UnionListReader extends AbstractFieldReader {
 
   @Override
   public boolean isSet() {
-    return !vector.getAccessor().isNull(idx());
+    return !vector.isNull(idx());
   }
 
   private int currentOffset;
@@ -57,8 +57,8 @@ public class UnionListReader extends AbstractFieldReader {
   @Override
   public void setPosition(int index) {
     super.setPosition(index);
-    currentOffset = offsets.getAccessor().get(index) - 1;
-    maxOffset = offsets.getAccessor().get(index + 1);
+    currentOffset = vector.getOffsetBuffer().getInt(index * OFFSET_WIDTH) - 1;
+    maxOffset = vector.getOffsetBuffer().getInt((index + 1) * OFFSET_WIDTH);
   }
 
   @Override
@@ -68,7 +68,7 @@ public class UnionListReader extends AbstractFieldReader {
 
   @Override
   public Object readObject() {
-    return vector.getAccessor().getObject(idx());
+    return vector.getObject(idx());
   }
 
   @Override
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
index 3b7dc4a..762a442 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryEncoder.java
@@ -47,11 +47,10 @@ public class DictionaryEncoder {
   public static ValueVector encode(ValueVector vector, Dictionary dictionary) {
     validateType(vector.getMinorType());
     // load dictionary values into a hashmap for lookup
-    ValueVector.Accessor dictionaryAccessor = dictionary.getVector().getAccessor();
-    Map<Object, Integer> lookUps = new HashMap<>(dictionaryAccessor.getValueCount());
-    for (int i = 0; i < dictionaryAccessor.getValueCount(); i++) {
+    Map<Object, Integer> lookUps = new HashMap<>(dictionary.getVector().getValueCount());
+    for (int i = 0; i < dictionary.getVector().getValueCount(); i++) {
       // for primitive array types we need a wrapper that implements equals and hashcode appropriately
-      lookUps.put(dictionaryAccessor.getObject(i), i);
+      lookUps.put(dictionary.getVector().getObject(i), i);
     }
 
     Field valueField = vector.getField();
@@ -61,14 +60,13 @@ public class DictionaryEncoder {
 
     // vector to hold our indices (dictionary encoded values)
     FieldVector indices = indexField.createVector(vector.getAllocator());
-    ValueVector.Mutator mutator = indices.getMutator();
 
     // use reflection to pull out the set method
     // TODO implement a common interface for int vectors
     Method setter = null;
     for (Class<?> c : ImmutableList.of(int.class, long.class)) {
       try {
-        setter = mutator.getClass().getMethod("setSafe", int.class, c);
+        setter = indices.getClass().getMethod("setSafe", int.class, c);
         break;
       } catch (NoSuchMethodException e) {
         // ignore
@@ -78,21 +76,20 @@ public class DictionaryEncoder {
       throw new IllegalArgumentException("Dictionary encoding does not have a valid int type:" + indices.getClass());
     }
 
-    ValueVector.Accessor accessor = vector.getAccessor();
-    int count = accessor.getValueCount();
+    int count = vector.getValueCount();
 
     indices.allocateNew();
 
     try {
       for (int i = 0; i < count; i++) {
-        Object value = accessor.getObject(i);
+        Object value = vector.getObject(i);
         if (value != null) { // if it's null leave it null
           // note: this may fail if value was not included in the dictionary
           Object encoded = lookUps.get(value);
           if (encoded == null) {
             throw new IllegalArgumentException("Dictionary encoding not defined for value:" + value);
           }
-          setter.invoke(mutator, i, encoded);
+          setter.invoke(indices, i, encoded);
         }
       }
     } catch (IllegalAccessException e) {
@@ -101,7 +98,7 @@ public class DictionaryEncoder {
       throw new RuntimeException("InvocationTargetException invoking vector mutator set():", e.getCause());
     }
 
-    mutator.setValueCount(count);
+    indices.setValueCount(count);
 
     return indices;
   }
@@ -114,15 +111,14 @@ public class DictionaryEncoder {
    * @return vector with values restored from dictionary
    */
   public static ValueVector decode(ValueVector indices, Dictionary dictionary) {
-    ValueVector.Accessor accessor = indices.getAccessor();
-    int count = accessor.getValueCount();
+    int count = indices.getValueCount();
     ValueVector dictionaryVector = dictionary.getVector();
-    int dictionaryCount = dictionaryVector.getAccessor().getValueCount();
+    int dictionaryCount = dictionaryVector.getValueCount();
     // copy the dictionary values into the decoded vector
     TransferPair transfer = dictionaryVector.getTransferPair(indices.getAllocator());
     transfer.getTo().allocateNewSafe();
     for (int i = 0; i < count; i++) {
-      Object index = accessor.getObject(i);
+      Object index = indices.getObject(i);
       if (index != null) {
         int indexAsInt = ((Number) index).intValue();
         if (indexAsInt > dictionaryCount) {
@@ -133,7 +129,7 @@ public class DictionaryEncoder {
     }
     // TODO do we need to worry about the field?
     ValueVector decoded = transfer.getTo();
-    decoded.getMutator().setValueCount(count);
+    decoded.setValueCount(count);
     return decoded;
   }
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
index b35aba5..7dc10b5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
@@ -82,7 +82,7 @@ public abstract class ArrowWriter implements AutoCloseable {
     for (long id : dictionaryIdsUsed) {
       Dictionary dictionary = provider.lookup(id);
       FieldVector vector = dictionary.getVector();
-      int count = vector.getAccessor().getValueCount();
+      int count = vector.getValueCount();
       VectorSchemaRoot dictRoot = new VectorSchemaRoot(ImmutableList.of(vector.getField()), ImmutableList.of(vector), count);
       VectorUnloader unloader = new VectorUnloader(dictRoot);
       ArrowRecordBatch batch = unloader.getRecordBatch();
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
index 0de8044..b21fcaa 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
@@ -23,60 +23,28 @@ import static com.fasterxml.jackson.core.JsonToken.END_OBJECT;
 import static com.fasterxml.jackson.core.JsonToken.START_ARRAY;
 import static com.fasterxml.jackson.core.JsonToken.START_OBJECT;
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.arrow.vector.schema.ArrowVectorType.OFFSET;
+import static org.apache.arrow.vector.schema.ArrowVectorType.*;
 
 import java.io.File;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.*;
 
 import com.google.common.collect.ImmutableList;
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.BigIntVector;
-import org.apache.arrow.vector.BitVector;
-import org.apache.arrow.vector.BufferBacked;
-import org.apache.arrow.vector.DateDayVector;
-import org.apache.arrow.vector.DateMilliVector;
-import org.apache.arrow.vector.DecimalVector;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.Float4Vector;
-import org.apache.arrow.vector.Float8Vector;
-import org.apache.arrow.vector.IntVector;
-import org.apache.arrow.vector.NullableVarBinaryVector;
-import org.apache.arrow.vector.NullableVarCharVector;
-import org.apache.arrow.vector.SmallIntVector;
-import org.apache.arrow.vector.TimeMicroVector;
-import org.apache.arrow.vector.TimeMilliVector;
-import org.apache.arrow.vector.TimeNanoVector;
-import org.apache.arrow.vector.TimeSecVector;
-import org.apache.arrow.vector.TimeStampMicroTZVector;
-import org.apache.arrow.vector.TimeStampMicroVector;
-import org.apache.arrow.vector.TimeStampMilliTZVector;
-import org.apache.arrow.vector.TimeStampMilliVector;
-import org.apache.arrow.vector.TimeStampNanoTZVector;
-import org.apache.arrow.vector.TimeStampNanoVector;
-import org.apache.arrow.vector.TimeStampSecTZVector;
-import org.apache.arrow.vector.TimeStampSecVector;
-import org.apache.arrow.vector.TinyIntVector;
-import org.apache.arrow.vector.UInt1Vector;
-import org.apache.arrow.vector.UInt2Vector;
-import org.apache.arrow.vector.UInt4Vector;
-import org.apache.arrow.vector.UInt8Vector;
-import org.apache.arrow.vector.ValueVector;
-import org.apache.arrow.vector.VarBinaryVector;
-import org.apache.arrow.vector.VarCharVector;
-import org.apache.arrow.vector.VectorSchemaRoot;
-import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.*;
 import org.apache.arrow.vector.dictionary.Dictionary;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
+import org.apache.arrow.vector.schema.ArrowFieldNode;
 import org.apache.arrow.vector.schema.ArrowVectorType;
+import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
-import org.apache.arrow.vector.util.DecimalUtility;
 import org.apache.arrow.vector.util.DictionaryUtility;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.binary.Hex;
@@ -153,7 +121,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
       FieldVector vector = dict.getVector();
       List<Field> fields = ImmutableList.of(vector.getField());
       List<FieldVector> vectors = ImmutableList.of(vector);
-      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getAccessor().getValueCount());
+      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getValueCount());
       read(root);
 
       readToken(END_OBJECT);
@@ -177,7 +145,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
         {
           for (Field field : root.getSchema().getFields()) {
             FieldVector vector = root.getVector(field.getName());
-            readVector(field, vector);
+            readFromJsonIntoVector(field, vector);
           }
         }
         readToken(END_ARRAY);
@@ -204,7 +172,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
         {
           for (Field field : schema.getFields()) {
             FieldVector vector = recordBatch.getVector(field.getName());
-            readVector(field, vector);
+            readFromJsonIntoVector(field, vector);
           }
         }
         readToken(END_ARRAY);
@@ -218,16 +186,24 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     }
   }
 
-  /**
-   * TODO: A better way of implementing this function is to use `loadFieldBuffers` methods in
-   * FieldVector to set the inner-vector data as done in `ArrowFileReader`.
-   */
-  private void readVector(Field field, FieldVector vector) throws JsonParseException, IOException {
+  private void readFromJsonIntoVector(Field field, FieldVector vector) throws JsonParseException, IOException {
     List<ArrowVectorType> vectorTypes = field.getTypeLayout().getVectorTypes();
-    List<BufferBacked> fieldInnerVectors = vector.getFieldInnerVectors();
-    if (vectorTypes.size() != fieldInnerVectors.size()) {
-      throw new IllegalArgumentException("vector types and inner vectors are not the same size: " + vectorTypes.size() + " != " + fieldInnerVectors.size());
-    }
+    ArrowBuf[] vectorBuffers = new ArrowBuf[vectorTypes.size()];
+    /*
+     * The order of inner buffers is :
+     * Fixed width vector:
+     *    -- validity buffer
+     *    -- data buffer
+     * Variable width vector:
+     *    -- validity buffer
+     *    -- offset buffer
+     *    -- data buffer
+     *
+     * This is similar to what getFieldInnerVectors() used to give but now that we don't have
+     * inner vectors anymore, we will work directly at the buffer level -- populate buffers
+     * locally as we read from Json parser and do loadFieldBuffers on the vector followed by
+     * releasing the local buffers.
+     */
     readToken(START_OBJECT);
     {
       // If currently reading dictionaries, field name is not important so don't check
@@ -236,39 +212,41 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
         throw new IllegalArgumentException("Expected field " + field.getName() + " but got " + name);
       }
 
-      // Initialize the vector with required capacity
-      int count = readNextField("count", Integer.class);
-      vector.setInitialCapacity(count);
-      vector.allocateNew();
+      /* Initialize the vector with required capacity but don't allocate since we would
+       * be doing loadFieldBuffers.
+       */
+      int valueCount = readNextField("count", Integer.class);
+      vector.setInitialCapacity(valueCount);
 
-      // Read inner vectors
       for (int v = 0; v < vectorTypes.size(); v++) {
         ArrowVectorType vectorType = vectorTypes.get(v);
-        ValueVector valueVector = (ValueVector) fieldInnerVectors.get(v);
         nextFieldIs(vectorType.getName());
         readToken(START_ARRAY);
-        int innerVectorCount = vectorType.equals(OFFSET) ? count + 1 : count;
-        for (int i = 0; i < innerVectorCount; i++) {
+        int innerBufferValueCount = valueCount;
+        if (vectorType.equals(OFFSET)) {
+          /* offset buffer has 1 additional value capacity */
+          innerBufferValueCount = valueCount + 1;
+        }
+        for (int i = 0; i < innerBufferValueCount; i++) {
+          /* write data to the buffer */
           parser.nextToken();
-          setValueFromParser(valueVector, i);
+          /* for variable width vectors, value count doesn't help pre-determining the capacity of
+           * the underlying data buffer. So we need to pass down the offset buffer (which was already
+           * populated in the previous iteration of this loop).
+           */
+          if (vectorType.equals(DATA) && (vector.getMinorType() == Types.MinorType.VARCHAR
+                  || vector.getMinorType() == Types.MinorType.VARBINARY)) {
+            vectorBuffers[v] = setValueFromParser(vectorType, vector, vectorBuffers[v],
+                    vectorBuffers[v-1], i, innerBufferValueCount);
+          } else {
+            vectorBuffers[v] = setValueFromParser(vectorType, vector, vectorBuffers[v],
+                    null, i, innerBufferValueCount);
+          }
         }
         readToken(END_ARRAY);
       }
 
-      // Set lastSet before valueCount to prevent setValueCount from filling empty values
-      switch (vector.getMinorType()) {
-        case LIST:
-          // ListVector starts lastSet from index 0, so lastSet value is always last index written + 1
-          ((ListVector) vector).getMutator().setLastSet(count);
-          break;
-        case VARBINARY:
-          ((NullableVarBinaryVector) vector).getMutator().setLastSet(count - 1);
-          break;
-        case VARCHAR:
-          ((NullableVarCharVector) vector).setLastSet(count - 1);
-          break;
-      }
-      vector.getMutator().setValueCount(count);
+      vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), Arrays.asList(vectorBuffers));
 
       // read child vectors, if any
       List<Field> fields = field.getChildren();
@@ -282,12 +260,16 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
         for (int i = 0; i < fields.size(); i++) {
           Field childField = fields.get(i);
           FieldVector childVector = vectorChildren.get(i);
-          readVector(childField, childVector);
+          readFromJsonIntoVector(childField, childVector);
         }
         readToken(END_ARRAY);
       }
     }
     readToken(END_OBJECT);
+
+    for (ArrowBuf buffer: vectorBuffers) {
+      buffer.release();
+    }
   }
 
   private byte[] decodeHexSafe(String hexString) throws IOException {
@@ -298,99 +280,124 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     }
   }
 
-  private void setValueFromParser(ValueVector valueVector, int i) throws IOException {
-    switch (valueVector.getMinorType()) {
-      case BIT:
-        ((BitVector) valueVector).getMutator().set(i, parser.readValueAs(Boolean.class) ? 1 : 0);
-        break;
-      case TINYINT:
-        ((TinyIntVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case SMALLINT:
-        ((SmallIntVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case INT:
-        ((IntVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case BIGINT:
-        ((BigIntVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case UINT1:
-        ((UInt1Vector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case UINT2:
-        ((UInt2Vector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case UINT4:
-        ((UInt4Vector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case UINT8:
-        ((UInt8Vector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case FLOAT4:
-        ((Float4Vector) valueVector).getMutator().set(i, parser.readValueAs(Float.class));
-        break;
-      case FLOAT8:
-        ((Float8Vector) valueVector).getMutator().set(i, parser.readValueAs(Double.class));
-        break;
-      case DECIMAL: {
-          DecimalVector decimalVector = (DecimalVector) valueVector;
-          // Here we assume the decimal value is the unscaled integer value as a string
-          BigDecimal decimalValue = new BigDecimal(parser.readValueAs(String.class));
-          DecimalUtility.writeBigDecimalToArrowBuf(decimalValue, decimalVector.getBuffer(), i);
-        }
-        break;
-      case VARBINARY:
-        ((VarBinaryVector) valueVector).getMutator().setSafe(i, decodeHexSafe(parser.readValueAs(String.class)));
-        break;
-      case VARCHAR:
-        ((VarCharVector) valueVector).getMutator().setSafe(i, parser.readValueAs(String.class).getBytes(UTF_8));
-        break;
-      case DATEDAY:
-        ((DateDayVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case DATEMILLI:
-        ((DateMilliVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESEC:
-        ((TimeSecVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case TIMEMILLI:
-        ((TimeMilliVector) valueVector).getMutator().set(i, parser.readValueAs(Integer.class));
-        break;
-      case TIMEMICRO:
-        ((TimeMicroVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMENANO:
-        ((TimeNanoVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPSEC:
-        ((TimeStampSecVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPMILLI:
-        ((TimeStampMilliVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPMICRO:
-        ((TimeStampMicroVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPNANO:
-        ((TimeStampNanoVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPSECTZ:
-        ((TimeStampSecTZVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPMILLITZ:
-        ((TimeStampMilliTZVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPMICROTZ:
-        ((TimeStampMicroTZVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      case TIMESTAMPNANOTZ:
-        ((TimeStampNanoTZVector) valueVector).getMutator().set(i, parser.readValueAs(Long.class));
-        break;
-      default:
-        throw new UnsupportedOperationException("minor type: " + valueVector.getMinorType());
+  private ArrowBuf setValueFromParser(ArrowVectorType bufferType, FieldVector vector,
+                                      ArrowBuf buffer, ArrowBuf offsetBuffer, int index,
+                                      int valueCount) throws IOException {
+    if (bufferType.equals(TYPE)) {
+      buffer = NullableTinyIntVector.set(buffer, allocator,
+              valueCount, index, parser.readValueAs(Byte.class));
+    } else if (bufferType.equals(OFFSET)) {
+      buffer = BaseNullableVariableWidthVector.set(buffer, allocator,
+              valueCount, index, parser.readValueAs(Integer.class));
+    } else if (bufferType.equals(VALIDITY)) {
+      buffer = BitVectorHelper.setValidityBit(buffer, allocator,
+              valueCount, index, parser.readValueAs(Boolean.class) ? 1 : 0);
+    } else if (bufferType.equals(DATA)) {
+      switch (vector.getMinorType()) {
+        case BIT:
+          buffer = BitVectorHelper.setValidityBit(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Boolean.class) ? 1 : 0);
+          break;
+        case TINYINT:
+          buffer = NullableTinyIntVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Byte.class));
+          break;
+        case SMALLINT:
+          buffer = NullableSmallIntVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Short.class));
+          break;
+        case INT:
+          buffer = NullableIntVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Integer.class));
+          break;
+        case BIGINT:
+          buffer = NullableBigIntVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case FLOAT4:
+          buffer = NullableFloat4Vector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Float.class));
+          break;
+        case FLOAT8:
+          buffer = NullableFloat8Vector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Double.class));
+          break;
+        case DECIMAL:
+          buffer = NullableDecimalVector.set(buffer, allocator,
+                  valueCount, index, decodeHexSafe(parser.readValueAs(String.class)));
+          break;
+        case VARBINARY:
+          assert (offsetBuffer != null);
+          buffer = BaseNullableVariableWidthVector.set(buffer, offsetBuffer, allocator, index,
+                  decodeHexSafe(parser.readValueAs(String.class)), valueCount);
+          break;
+        case VARCHAR:
+          assert (offsetBuffer != null);
+          buffer = BaseNullableVariableWidthVector.set(buffer, offsetBuffer, allocator, index,
+                  parser.readValueAs(String.class).getBytes(UTF_8), valueCount);
+          break;
+        case DATEDAY:
+          buffer = NullableDateDayVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Integer.class));
+          break;
+        case DATEMILLI:
+          buffer = NullableDateMilliVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESEC:
+          buffer = NullableTimeSecVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Integer.class));
+          break;
+        case TIMEMILLI:
+          buffer = NullableTimeMilliVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Integer.class));
+          break;
+        case TIMEMICRO:
+          buffer = NullableTimeMicroVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMENANO:
+          buffer = NullableTimeNanoVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPSEC:
+          buffer = NullableTimeStampSecVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPMILLI:
+          buffer = NullableTimeStampMilliVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPMICRO:
+          buffer = NullableTimeStampMicroVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPNANO:
+          buffer = NullableTimeStampNanoVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPSECTZ:
+          buffer = NullableTimeStampSecTZVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPMILLITZ:
+          buffer = NullableTimeStampMilliTZVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPMICROTZ:
+          buffer = NullableTimeStampMicroTZVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        case TIMESTAMPNANOTZ:
+          buffer = NullableTimeStampNanoTZVector.set(buffer, allocator,
+                  valueCount, index, parser.readValueAs(Long.class));
+          break;
+        default:
+          throw new UnsupportedOperationException("minor type: " + vector.getMinorType());
+      }
     }
+
+    return buffer;
   }
 
   @Override
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
index 05341be..7921cd3 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
@@ -18,9 +18,10 @@
 
 package org.apache.arrow.vector.file.json;
 
+import static org.apache.arrow.vector.schema.ArrowVectorType.*;
+
 import java.io.File;
 import java.io.IOException;
-import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -28,28 +29,11 @@ import java.util.Set;
 
 import com.google.common.collect.ImmutableList;
 import io.netty.buffer.ArrowBuf;
-import org.apache.arrow.vector.BitVector;
-import org.apache.arrow.vector.BufferBacked;
-import org.apache.arrow.vector.DateDayVector;
-import org.apache.arrow.vector.DateMilliVector;
-import org.apache.arrow.vector.DecimalVector;
-import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.TimeMicroVector;
-import org.apache.arrow.vector.TimeMilliVector;
-import org.apache.arrow.vector.TimeNanoVector;
-import org.apache.arrow.vector.TimeSecVector;
-import org.apache.arrow.vector.TimeStampMicroVector;
-import org.apache.arrow.vector.TimeStampMilliVector;
-import org.apache.arrow.vector.TimeStampNanoVector;
-import org.apache.arrow.vector.TimeStampSecVector;
-import org.apache.arrow.vector.ValueVector;
-import org.apache.arrow.vector.ValueVector.Accessor;
-import org.apache.arrow.vector.VarBinaryVector;
-import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.*;
 import org.apache.arrow.vector.dictionary.Dictionary;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.schema.ArrowVectorType;
-import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.Types;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
 
@@ -135,7 +119,7 @@ public class JsonFileWriter implements AutoCloseable {
       FieldVector vector = dictionary.getVector();
       List<Field> fields = ImmutableList.of(vector.getField());
       List<FieldVector> vectors = ImmutableList.of(vector);
-      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getAccessor().getValueCount());
+      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getValueCount());
       writeBatch(root);
 
       generator.writeEndObject();
@@ -157,31 +141,36 @@ public class JsonFileWriter implements AutoCloseable {
       generator.writeArrayFieldStart("columns");
       for (Field field : recordBatch.getSchema().getFields()) {
         FieldVector vector = recordBatch.getVector(field.getName());
-        writeVector(field, vector);
+        writeFromVectorIntoJson(field, vector);
       }
       generator.writeEndArray();
     }
     generator.writeEndObject();
   }
 
-  private void writeVector(Field field, FieldVector vector) throws IOException {
+  private void writeFromVectorIntoJson(Field field, FieldVector vector) throws IOException {
     List<ArrowVectorType> vectorTypes = field.getTypeLayout().getVectorTypes();
-    List<BufferBacked> fieldInnerVectors = vector.getFieldInnerVectors();
-    if (vectorTypes.size() != fieldInnerVectors.size()) {
-      throw new IllegalArgumentException("vector types and inner vectors are not the same size: " + vectorTypes.size() + " != " + fieldInnerVectors.size());
+    List<ArrowBuf> vectorBuffers = vector.getFieldBuffers();
+    if (vectorTypes.size() != vectorBuffers.size()) {
+      throw new IllegalArgumentException("vector types and inner vector buffers are not the same size: " + vectorTypes.size() + " != " + vectorBuffers.size());
     }
     generator.writeStartObject();
     {
       generator.writeObjectField("name", field.getName());
-      int valueCount = vector.getAccessor().getValueCount();
+      int valueCount = vector.getValueCount();
       generator.writeObjectField("count", valueCount);
       for (int v = 0; v < vectorTypes.size(); v++) {
         ArrowVectorType vectorType = vectorTypes.get(v);
-        BufferBacked innerVector = fieldInnerVectors.get(v);
+        ArrowBuf vectorBuffer = vectorBuffers.get(v);
         generator.writeArrayFieldStart(vectorType.getName());
-        ValueVector valueVector = (ValueVector) innerVector;
-        for (int i = 0; i < valueVector.getAccessor().getValueCount(); i++) {
-          writeValueToGenerator(valueVector, i);
+        final int bufferValueCount = (vectorType.equals(OFFSET)) ? valueCount + 1 : valueCount;
+        for (int i = 0; i < bufferValueCount; i++) {
+          if (vectorType.equals(DATA) && (vector.getMinorType() == Types.MinorType.VARCHAR ||
+                  vector.getMinorType() == Types.MinorType.VARBINARY)) {
+            writeValueToGenerator(vectorType, vectorBuffer, vectorBuffers.get(v-1), vector, i);
+          } else {
+            writeValueToGenerator(vectorType, vectorBuffer, null, vector, i);
+          }
         }
         generator.writeEndArray();
       }
@@ -195,7 +184,7 @@ public class JsonFileWriter implements AutoCloseable {
         for (int i = 0; i < fields.size(); i++) {
           Field childField = fields.get(i);
           FieldVector childVector = children.get(i);
-          writeVector(childField, childVector);
+          writeFromVectorIntoJson(childField, childVector);
         }
         generator.writeEndArray();
       }
@@ -203,62 +192,101 @@ public class JsonFileWriter implements AutoCloseable {
     generator.writeEndObject();
   }
 
-  private void writeValueToGenerator(ValueVector valueVector, int i) throws IOException {
-    switch (valueVector.getMinorType()) {
-      case DATEDAY:
-        generator.writeNumber(((DateDayVector) valueVector).getAccessor().get(i));
-        break;
-      case DATEMILLI:
-        generator.writeNumber(((DateMilliVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMESEC:
-        generator.writeNumber(((TimeSecVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMEMILLI:
-        generator.writeNumber(((TimeMilliVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMEMICRO:
-        generator.writeNumber(((TimeMicroVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMENANO:
-        generator.writeNumber(((TimeNanoVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMESTAMPSEC:
-        generator.writeNumber(((TimeStampSecVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMESTAMPMILLI:
-        generator.writeNumber(((TimeStampMilliVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMESTAMPMICRO:
-        generator.writeNumber(((TimeStampMicroVector) valueVector).getAccessor().get(i));
-        break;
-      case TIMESTAMPNANO:
-        generator.writeNumber(((TimeStampNanoVector) valueVector).getAccessor().get(i));
-        break;
-      case BIT:
-        generator.writeNumber(((BitVector) valueVector).getAccessor().get(i));
-        break;
-      case VARBINARY: {
-          String hexString = Hex.encodeHexString(((VarBinaryVector) valueVector).getAccessor().get(i));
-          generator.writeString(hexString);
+  private void writeValueToGenerator(ArrowVectorType bufferType, ArrowBuf buffer,
+                                     ArrowBuf offsetBuffer, FieldVector vector, int index) throws IOException {
+    if (bufferType.equals(TYPE)) {
+      generator.writeNumber(buffer.getByte(index * NullableTinyIntVector.TYPE_WIDTH));
+    } else if (bufferType.equals(OFFSET)) {
+      generator.writeNumber(buffer.getInt(index * BaseNullableVariableWidthVector.OFFSET_WIDTH));
+    } else if(bufferType.equals(VALIDITY)) {
+      generator.writeNumber(vector.isNull(index) ? 0 : 1);
+    } else if (bufferType.equals(DATA)) {
+      switch (vector.getMinorType()) {
+        case TINYINT:
+          generator.writeNumber(NullableTinyIntVector.get(buffer, index));
+          break;
+        case SMALLINT:
+          generator.writeNumber(NullableSmallIntVector.get(buffer, index));
+          break;
+        case INT:
+          generator.writeNumber(NullableIntVector.get(buffer, index));
+          break;
+        case BIGINT:
+          generator.writeNumber(NullableBigIntVector.get(buffer, index));
+          break;
+        case FLOAT4:
+          generator.writeNumber(NullableFloat4Vector.get(buffer, index));
+          break;
+        case FLOAT8:
+          generator.writeNumber(NullableFloat8Vector.get(buffer, index));
+          break;
+        case DATEDAY:
+          generator.writeNumber(NullableDateDayVector.get(buffer, index));
+          break;
+        case DATEMILLI:
+          generator.writeNumber(NullableDateMilliVector.get(buffer, index));
+          break;
+        case TIMESEC:
+          generator.writeNumber(NullableTimeSecVector.get(buffer, index));
+          break;
+        case TIMEMILLI:
+          generator.writeNumber(NullableTimeMilliVector.get(buffer, index));
+          break;
+        case TIMEMICRO:
+          generator.writeNumber(NullableTimeMicroVector.get(buffer, index));
+          break;
+        case TIMENANO:
+          generator.writeNumber(NullableTimeNanoVector.get(buffer, index));
+          break;
+        case TIMESTAMPSEC:
+          generator.writeNumber(NullableTimeStampSecVector.get(buffer, index));
+          break;
+        case TIMESTAMPMILLI:
+          generator.writeNumber(NullableTimeStampMilliVector.get(buffer, index));
+          break;
+        case TIMESTAMPMICRO:
+          generator.writeNumber(NullableTimeStampMicroVector.get(buffer, index));
+          break;
+        case TIMESTAMPNANO:
+          generator.writeNumber(NullableTimeStampNanoVector.get(buffer, index));
+          break;
+        case TIMESTAMPSECTZ:
+          generator.writeNumber(NullableTimeStampSecTZVector.get(buffer, index));
+          break;
+        case TIMESTAMPMILLITZ:
+          generator.writeNumber(NullableTimeStampMilliTZVector.get(buffer, index));
+          break;
+        case TIMESTAMPMICROTZ:
+          generator.writeNumber(NullableTimeStampMicroTZVector.get(buffer, index));
+          break;
+        case TIMESTAMPNANOTZ:
+          generator.writeNumber(NullableTimeStampNanoTZVector.get(buffer, index));
+          break;
+        case BIT:
+          generator.writeNumber(BitVectorHelper.get(buffer, index));
+          break;
+        case VARBINARY: {
+          assert offsetBuffer != null;
+          String hexString = Hex.encodeHexString(BaseNullableVariableWidthVector.get(buffer,
+                  offsetBuffer, index));
+          generator.writeObject(hexString);
+          break;
         }
-        break;
-      case DECIMAL: {
-          BigDecimal decimalValue = ((DecimalVector) valueVector).getAccessor().getObject(i);
-          // We write the unscaled value, because the scale is stored in the type metadata.
-          generator.writeString(decimalValue.unscaledValue().toString());
+        case VARCHAR: {
+          assert offsetBuffer != null;
+          byte[] b = (BaseNullableVariableWidthVector.get(buffer, offsetBuffer, index));
+          generator.writeString(new String(b, "UTF-8"));
+          break;
         }
-        break;
-      default:
-        // TODO: each type
-        Accessor accessor = valueVector.getAccessor();
-        Object value = accessor.getObject(i);
-        if (value instanceof Number || value instanceof Boolean) {
-          generator.writeObject(value);
-        } else {
-          generator.writeObject(value.toString());
+        case DECIMAL: {
+          String hexString = Hex.encodeHexString(DecimalUtility.getByteArrayFromArrowBuf(buffer,
+                  index));
+          generator.writeString(hexString);
+          break;
         }
-        break;
+        default:
+          throw new UnsupportedOperationException("minor type: " + vector.getMinorType());
+      }
     }
   }
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
index 5851bd5..c27e5e5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
@@ -114,13 +114,13 @@ public class Validator {
     if (!field1.equals(vector2.getField())) {
       throw new IllegalArgumentException("Different Fields:\n" + field1 + "\n!=\n" + vector2.getField());
     }
-    int valueCount = vector1.getAccessor().getValueCount();
-    if (valueCount != vector2.getAccessor().getValueCount()) {
-      throw new IllegalArgumentException("Different value count for field " + field1 + " : " + valueCount + " != " + vector2.getAccessor().getValueCount());
+    int valueCount = vector1.getValueCount();
+    if (valueCount != vector2.getValueCount()) {
+      throw new IllegalArgumentException("Different value count for field " + field1 + " : " + valueCount + " != " + vector2.getValueCount());
     }
     for (int j = 0; j < valueCount; j++) {
-      Object obj1 = vector1.getAccessor().getObject(j);
-      Object obj2 = vector2.getAccessor().getObject(j);
+      Object obj1 = vector1.getObject(j);
+      Object obj2 = vector2.getObject(j);
       if (!equals(field1.getType(), obj1, obj2)) {
         throw new IllegalArgumentException(
             "Different values in column:\n" + field1 + " at index " + j + ": " + obj1 + " != " + obj2);
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
index 17fcf05..ada3414 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
@@ -340,59 +340,56 @@ public class TestBitVector {
       int valueCapacity = vector.getValueCapacity();
       assertEquals(4096, valueCapacity);
 
-      final NullableBitVector.Mutator mutator = vector.getMutator();
-      final NullableBitVector.Accessor accessor = vector.getAccessor();
-
       for (int i = 0; i < valueCapacity; i++) {
         if ((i & 1) == 1) {
-          mutator.set(i, 1);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity; i++) {
         if ((i & 1) == 1) {
-          assertFalse("unexpected cleared bit at index: " + i, accessor.isNull(i));
+          assertFalse("unexpected cleared bit at index: " + i, vector.isNull(i));
         }
         else {
-          assertTrue("unexpected set bit at index: " + i, accessor.isNull(i));
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
       /* trigger first realloc */
-      mutator.setSafe(valueCapacity, 1, 1);
+      vector.setSafe(valueCapacity, 1, 1);
       assertEquals(valueCapacity * 2, vector.getValueCapacity());
 
       for (int i = valueCapacity; i < valueCapacity*2; i++) {
         if ((i & 1) == 1) {
-          mutator.set(i, 1);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity*2; i++) {
         if (((i & 1) == 1) || (i == valueCapacity)) {
-          assertFalse("unexpected cleared bit at index: " + i, accessor.isNull(i));
+          assertFalse("unexpected cleared bit at index: " + i, vector.isNull(i));
         }
         else {
-          assertTrue("unexpected set bit at index: " + i, accessor.isNull(i));
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
       /* trigger second realloc */
-      mutator.setSafe(valueCapacity*2, 1, 1);
+      vector.setSafe(valueCapacity*2, 1, 1);
       assertEquals(valueCapacity * 4, vector.getValueCapacity());
 
       for (int i = valueCapacity*2; i < valueCapacity*4; i++) {
         if ((i & 1) == 1) {
-          mutator.set(i, 1);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity*4; i++) {
         if (((i & 1) == 1) || (i == valueCapacity) || (i == valueCapacity*2)) {
-          assertFalse("unexpected cleared bit at index: " + i, accessor.isNull(i));
+          assertFalse("unexpected cleared bit at index: " + i, vector.isNull(i));
         }
         else {
-          assertTrue("unexpected set bit at index: " + i, accessor.isNull(i));
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
@@ -400,26 +397,24 @@ public class TestBitVector {
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
       final NullableBitVector toVector = (NullableBitVector)transferPair.getTo();
-      final NullableBitVector.Accessor toAccessor = toVector.getAccessor();
-      final NullableBitVector.Mutator toMutator = toVector.getMutator();
 
       assertEquals(valueCapacity * 4, toVector.getValueCapacity());
 
       /* realloc the toVector */
-      toMutator.setSafe(valueCapacity * 4, 1, 1);
+      toVector.setSafe(valueCapacity * 4, 1, 1);
 
       for (int i = 0; i < toVector.getValueCapacity(); i++) {
         if (i <= valueCapacity * 4) {
           if (((i & 1) == 1) || (i == valueCapacity) ||
                   (i == valueCapacity*2) || (i == valueCapacity*4)) {
-            assertFalse("unexpected cleared bit at index: " + i, toAccessor.isNull(i));
+            assertFalse("unexpected cleared bit at index: " + i, toVector.isNull(i));
           }
           else {
-            assertTrue("unexpected set bit at index: " + i, toAccessor.isNull(i));
+            assertTrue("unexpected set bit at index: " + i, toVector.isNull(i));
           }
         }
         else {
-          assertTrue("unexpected set bit at index: " + i, toAccessor.isNull(i));
+          assertTrue("unexpected set bit at index: " + i, toVector.isNull(i));
         }
       }
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
index 56d2293..4d844d6 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
@@ -71,14 +71,14 @@ public class TestDecimalVector {
       for (int i = 0; i < intValues.length; i++) {
         BigDecimal decimal = new BigDecimal(BigInteger.valueOf(intValues[i]), scale);
         values[i] = decimal;
-        decimalVector.getMutator().setSafe(i, decimal);
+        decimalVector.setSafe(i, decimal);
       }
 
-      decimalVector.getMutator().setValueCount(intValues.length);
+      decimalVector.setValueCount(intValues.length);
 
       for (int i = 0; i < intValues.length; i++) {
-        BigDecimal value = decimalVector.getAccessor().getObject(i);
-        assertEquals(values[i], value);
+        BigDecimal value = decimalVector.getObject(i);
+        assertEquals("unexpected data at index: " + i, values[i], value);
       }
     }
   }
@@ -92,7 +92,7 @@ public class TestDecimalVector {
       boolean hasError = false;
       try {
         BigDecimal decimal = new BigDecimal(BigInteger.valueOf(0), 3);
-        decimalVector.getMutator().setSafe(0, decimal);
+        decimalVector.setSafe(0, decimal);
       } catch (UnsupportedOperationException ue) {
         hasError = true;
       } finally {
@@ -103,7 +103,7 @@ public class TestDecimalVector {
       hasError = false;
       try {
         BigDecimal decimal = new BigDecimal(BigInteger.valueOf(12345), 2);
-        decimalVector.getMutator().setSafe(0, decimal);
+        decimalVector.setSafe(0, decimal);
       } catch (UnsupportedOperationException ue) {
         hasError = true;
       } finally {
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
index 1185246..46a2baf 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
@@ -135,9 +135,9 @@ public class TestDictionaryVector {
         // now run through the decoder and verify we get the original back
         try (ValueVector decoded = DictionaryEncoder.decode(encoded, dictionary)) {
           assertEquals(vector.getClass(), decoded.getClass());
-          assertEquals(vector.getAccessor().getValueCount(), decoded.getAccessor().getValueCount());
+          assertEquals(vector.getValueCount(), decoded.getValueCount());
           for (int i = 0; i < count; ++i) {
-            assertEquals(vector.getAccessor().getObject(i), decoded.getAccessor().getObject(i));
+            assertEquals(vector.getObject(i), decoded.getObject(i));
           }
         }
       }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
index 168deac..2af6cd5 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
@@ -57,7 +57,7 @@ public class TestFixedSizeListVector {
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
-        vector.getMutator().setNotNull(i);
+        vector.setNotNull(i);
         nested.set(i * 2, i);
         nested.set(i * 2 + 1, i + 10);
       }
@@ -81,17 +81,16 @@ public class TestFixedSizeListVector {
   public void testFloatTypeNullable() {
     try (FixedSizeListVector vector = FixedSizeListVector.empty("list", 2, allocator)) {
       NullableFloat4Vector nested = (NullableFloat4Vector) vector.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
-      NullableFloat4Vector.Mutator mutator = nested.getMutator();
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
         if (i % 2 == 0) {
-          vector.getMutator().setNotNull(i);
-          mutator.set(i * 2, i + 0.1f);
-          mutator.set(i * 2 + 1, i + 10.1f);
+          vector.setNotNull(i);
+          nested.set(i * 2, i + 0.1f);
+          nested.set(i * 2 + 1, i + 10.1f);
         }
       }
-      vector.getMutator().setValueCount(10);
+      vector.setValueCount(10);
 
       UnionFixedSizeListReader reader = vector.getReader();
       for (int i = 0; i < 10; i++) {
@@ -115,24 +114,22 @@ public class TestFixedSizeListVector {
   @Test
   public void testNestedInList() {
     try (ListVector vector = ListVector.empty("list", allocator)) {
-      ListVector.Mutator mutator = vector.getMutator();
       FixedSizeListVector tuples = (FixedSizeListVector) vector.addOrGetVector(FieldType.nullable(new ArrowType.FixedSizeList(2))).getVector();
-      FixedSizeListVector.Mutator tupleMutator = tuples.getMutator();
       NullableIntVector innerVector = (NullableIntVector) tuples.addOrGetVector(FieldType.nullable(MinorType.INT.getType())).getVector();
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
         if (i % 2 == 0) {
-          int position = mutator.startNewValue(i);
+          int position = vector.startNewValue(i);
           for (int j = 0; j < i % 7; j++) {
-            tupleMutator.setNotNull(position + j);
+            tuples.setNotNull(position + j);
             innerVector.set((position + j) * 2, j);
             innerVector.set((position + j) * 2 + 1, j + 1);
           }
-          mutator.endValue(i, i % 7);
+          vector.endValue(i, i % 7);
         }
       }
-      mutator.setValueCount(10);
+      vector.setValueCount(10);
 
       UnionListReader reader = vector.getReader();
       for (int i = 0; i < 10; i++) {
@@ -161,24 +158,24 @@ public class TestFixedSizeListVector {
     try (FixedSizeListVector from = new FixedSizeListVector("from", allocator, 2, null, null);
          FixedSizeListVector to = new FixedSizeListVector("to", allocator, 2, null, null)) {
       NullableFloat4Vector nested = (NullableFloat4Vector) from.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
-      NullableFloat4Vector.Mutator mutator = nested.getMutator();
       from.allocateNew();
 
       for (int i = 0; i < 10; i++) {
         if (i % 2 == 0) {
-          from.getMutator().setNotNull(i);
-          mutator.set(i * 2, i + 0.1f);
-          mutator.set(i * 2 + 1, i + 10.1f);
+          from.setNotNull(i);
+          nested.set(i * 2, i + 0.1f);
+          nested.set(i * 2 + 1, i + 10.1f);
         }
       }
-      from.getMutator().setValueCount(10);
+      from.setValueCount(10);
 
       TransferPair pair = from.makeTransferPair(to);
 
       pair.copyValueSafe(0, 1);
       pair.copyValueSafe(2, 2);
       to.copyFromSafe(4, 3, from);
-      to.getMutator().setValueCount(10);
+
+      to.setValueCount(10);
 
       UnionFixedSizeListReader reader = to.getReader();
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
index 59e1646..f6aa86a 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
@@ -85,7 +85,7 @@ public class TestListVector {
       for (int i = 0; i < 3; i++) {
         outVector.copyFrom(i, i, inVector);
       }
-      outVector.getMutator().setValueCount(3);
+      outVector.setValueCount(3);
 
       // assert the output vector is correct
       FieldReader reader = outVector.getReader();
@@ -95,11 +95,9 @@ public class TestListVector {
       reader.setPosition(2);
       Assert.assertTrue("shouldn't be null", reader.isSet());
 
-      /* check the exact contents of vector */
-      final ListVector.Accessor accessor = outVector.getAccessor();
 
       /* index 0 */
-      Object result = accessor.getObject(0);
+      Object result = outVector.getObject(0);
       ArrayList<Long> resultSet = (ArrayList<Long>) result;
       assertEquals(3, resultSet.size());
       assertEquals(new Long(1), (Long) resultSet.get(0));
@@ -107,11 +105,11 @@ public class TestListVector {
       assertEquals(new Long(3), (Long) resultSet.get(2));
 
       /* index 1 */
-      result = accessor.getObject(1);
+      result = outVector.getObject(1);
       assertNull(result);
 
       /* index 2 */
-      result = accessor.getObject(2);
+      result = outVector.getObject(2);
       resultSet = (ArrayList<Long>) result;
       assertEquals(0, resultSet.size());
     }
@@ -128,46 +126,46 @@ public class TestListVector {
       /* allocate memory */
       listVector.allocateNew();
 
-      /* get inner vectors; bitVector and offsetVector */
-      List<BufferBacked> innerVectors = listVector.getFieldInnerVectors();
-      BitVector bitVector = (BitVector) innerVectors.get(0);
-      UInt4Vector offsetVector = (UInt4Vector) innerVectors.get(1);
+      /* get inner buffers; validityBuffer and offsetBuffer */
+
+      ArrowBuf validityBuffer = listVector.getValidityBuffer();
+      ArrowBuf offsetBuffer = listVector.getOffsetBuffer();
 
       /* get the underlying data vector -- NullableBigIntVector */
       NullableBigIntVector dataVector = (NullableBigIntVector) listVector.getDataVector();
 
       /* check current lastSet */
-      assertEquals(Integer.toString(0), Integer.toString(listVector.getMutator().getLastSet()));
+      assertEquals(Integer.toString(0), Integer.toString(listVector.getLastSet()));
 
       int index = 0;
       int offset = 0;
 
-      /* write [10, 11, 12] to the list vector at index */
-      bitVector.getMutator().setSafe(index, 1);
-      dataVector.getMutator().setSafe(0, 1, 10);
-      dataVector.getMutator().setSafe(1, 1, 11);
-      dataVector.getMutator().setSafe(2, 1, 12);
-      offsetVector.getMutator().setSafe(index + 1, 3);
+      /* write [10, 11, 12] to the list vector at index 0 */
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      dataVector.setSafe(0, 1, 10);
+      dataVector.setSafe(1, 1, 11);
+      dataVector.setSafe(2, 1, 12);
+      offsetBuffer.setInt((index + 1) * ListVector.OFFSET_WIDTH, 3);
 
       index += 1;
 
       /* write [13, 14] to the list vector at index 1 */
-      bitVector.getMutator().setSafe(index, 1);
-      dataVector.getMutator().setSafe(3, 1, 13);
-      dataVector.getMutator().setSafe(4, 1, 14);
-      offsetVector.getMutator().setSafe(index + 1, 5);
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      dataVector.setSafe(3, 1, 13);
+      dataVector.setSafe(4, 1, 14);
+      offsetBuffer.setInt((index + 1) * ListVector.OFFSET_WIDTH, 5);
 
       index += 1;
 
       /* write [15, 16, 17] to the list vector at index 2 */
-      bitVector.getMutator().setSafe(index, 1);
-      dataVector.getMutator().setSafe(5, 1, 15);
-      dataVector.getMutator().setSafe(6, 1, 16);
-      dataVector.getMutator().setSafe(7, 1, 17);
-      offsetVector.getMutator().setSafe(index + 1, 8);
+      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+      dataVector.setSafe(5, 1, 15);
+      dataVector.setSafe(6, 1, 16);
+      dataVector.setSafe(7, 1, 17);
+      offsetBuffer.setInt((index + 1) * ListVector.OFFSET_WIDTH, 8);
 
       /* check current lastSet */
-      assertEquals(Integer.toString(0), Integer.toString(listVector.getMutator().getLastSet()));
+      assertEquals(Integer.toString(0), Integer.toString(listVector.getLastSet()));
 
       /* set lastset and arbitrary valuecount for list vector.
        *
@@ -208,54 +206,50 @@ public class TestListVector {
        *                [15, 16, 17]
        *              }
        */
-      listVector.getMutator().setLastSet(3);
-      listVector.getMutator().setValueCount(10);
-
-      /* check the vector output */
-      final UInt4Vector.Accessor offsetAccessor = offsetVector.getAccessor();
-      final ValueVector.Accessor valueAccessor = dataVector.getAccessor();
+      listVector.setLastSet(3);
+      listVector.setValueCount(10);
 
       index = 0;
-      offset = offsetAccessor.get(index);
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(0), Integer.toString(offset));
 
-      Object actual = valueAccessor.getObject(offset);
+      Object actual = dataVector.getObject(offset);
       assertEquals(new Long(10), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(11), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(12), (Long) actual);
 
       index++;
-      offset = offsetAccessor.get(index);
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(3), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(13), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(14), (Long) actual);
 
       index++;
-      offset = offsetAccessor.get(index);
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(5), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offsetAccessor.get(index));
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(15), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(16), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(17), (Long) actual);
 
       index++;
-      offset = offsetAccessor.get(index);
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(8), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offsetAccessor.get(index));
+      actual = dataVector.getObject(offset);
       assertNull(actual);
     }
   }
@@ -308,99 +302,97 @@ public class TestListVector {
       listWriter.bigInt().writeBigInt(23);
       listWriter.endList();
 
-      listVector.getMutator().setValueCount(5);
+      listVector.setValueCount(5);
 
-      assertEquals(5, listVector.getMutator().getLastSet());
+      assertEquals(5, listVector.getLastSet());
 
-      /* get offsetVector */
-      UInt4Vector offsetVector = (UInt4Vector) listVector.getOffsetVector();
+      /* get offset buffer */
+      final ArrowBuf offsetBuffer = listVector.getOffsetBuffer();
 
       /* get dataVector */
       NullableBigIntVector dataVector = (NullableBigIntVector) listVector.getDataVector();
 
       /* check the vector output */
-      final UInt4Vector.Accessor offsetAccessor = offsetVector.getAccessor();
-      final ValueVector.Accessor valueAccessor = dataVector.getAccessor();
 
       int index = 0;
       int offset = 0;
       Object actual = null;
 
       /* index 0 */
-      assertFalse(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertFalse(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(0), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(10), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(11), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(12), (Long) actual);
 
       /* index 1 */
       index++;
-      assertFalse(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertFalse(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(3), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(13), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(14), (Long) actual);
 
       /* index 2 */
       index++;
-      assertFalse(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertFalse(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(5), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(15), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(16), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(17), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(18), (Long) actual);
 
       /* index 3 */
       index++;
-      assertFalse(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertFalse(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(9), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(19), (Long) actual);
 
       /* index 4 */
       index++;
-      assertFalse(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertFalse(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(10), Integer.toString(offset));
 
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(20), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(21), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(22), (Long) actual);
       offset++;
-      actual = valueAccessor.getObject(offset);
+      actual = dataVector.getObject(offset);
       assertEquals(new Long(23), (Long) actual);
 
       /* index 5 */
       index++;
-      assertTrue(listVector.getAccessor().isNull(index));
-      offset = offsetAccessor.get(index);
+      assertTrue(listVector.isNull(index));
+      offset = offsetBuffer.getInt(index * ListVector.OFFSET_WIDTH);
       assertEquals(Integer.toString(14), Integer.toString(offset));
 
       /* do split and transfer */
@@ -423,26 +415,26 @@ public class TestListVector {
           transferPair.splitAndTransfer(start, splitLength);
 
           /* get offsetVector of toVector */
-          UInt4Vector offsetVector1 = (UInt4Vector) toVector.getOffsetVector();
-          UInt4Vector.Accessor offsetAccessor1 = offsetVector1.getAccessor();
+          final ArrowBuf toOffsetBuffer = toVector.getOffsetBuffer();
 
           /* get dataVector of toVector */
           NullableBigIntVector dataVector1 = (NullableBigIntVector) toVector.getDataVector();
-          NullableBigIntVector.Accessor valueAccessor1 = dataVector1.getAccessor();
 
           for (int i = 0; i < splitLength; i++) {
-            dataLength1 = offsetAccessor.get(start + i + 1) - offsetAccessor.get(start + i);
-            dataLength2 = offsetAccessor1.get(i + 1) - offsetAccessor1.get(i);
+            dataLength1 = offsetBuffer.getInt((start + i + 1) * ListVector.OFFSET_WIDTH) -
+                    offsetBuffer.getInt((start + i) * ListVector.OFFSET_WIDTH);
+            dataLength2 = toOffsetBuffer.getInt((i + 1) * ListVector.OFFSET_WIDTH)
+                    - toOffsetBuffer.getInt(i * ListVector.OFFSET_WIDTH);
 
             assertEquals("Different data lengths at index: " + i + " and start: " + start,
                     dataLength1, dataLength2);
 
-            offset1 = offsetAccessor.get(start + i);
-            offset2 = offsetAccessor1.get(i);
+            offset1 = offsetBuffer.getInt((start + i) * ListVector.OFFSET_WIDTH);
+            offset2 = toOffsetBuffer.getInt(i * ListVector.OFFSET_WIDTH);
 
             for (int j = 0; j < dataLength1; j++) {
               assertEquals("Different data at indexes: " + offset1 + " and " + offset2,
-                      valueAccessor.getObject(offset1), valueAccessor1.getObject(offset2));
+                      dataVector.getObject(offset1), dataVector1.getObject(offset2));
 
               offset1++;
               offset2++;
@@ -506,15 +498,14 @@ public class TestListVector {
 
       listWriter.endList();
 
-      assertEquals(2, listVector.getMutator().getLastSet());
+      assertEquals(2, listVector.getLastSet());
 
-      listVector.getMutator().setValueCount(2);
+      listVector.setValueCount(2);
 
-      final ListVector.Accessor accessor = listVector.getAccessor();
-      assertEquals(2, accessor.getValueCount());
+      assertEquals(2, listVector.getValueCount());
 
       /* get listVector value at index 0 -- the value itself is a listvector */
-      Object result = accessor.getObject(0);
+      Object result = listVector.getObject(0);
       ArrayList<ArrayList<Long>> resultSet = (ArrayList<ArrayList<Long>>) result;
       ArrayList<Long> list;
 
@@ -534,7 +525,7 @@ public class TestListVector {
       assertEquals(new Long(175), list.get(3));
 
        /* get listVector value at index 1 -- the value itself is a listvector */
-      result = accessor.getObject(1);
+      result = listVector.getObject(1);
       resultSet = (ArrayList<ArrayList<Long>>) result;
 
       assertEquals(3, resultSet.size());              /* 3 inner lists at index 1 */
@@ -555,17 +546,16 @@ public class TestListVector {
       assertEquals(new Long(35), list.get(2));
 
       /* check underlying bitVector */
-      assertFalse(accessor.isNull(0));
-      assertFalse(accessor.isNull(1));
+      assertFalse(listVector.isNull(0));
+      assertFalse(listVector.isNull(1));
 
-      /* check underlying offsetVector */
-      UInt4Vector offsetVector = listVector.getOffsetVector();
-      final UInt4Vector.Accessor offsetAccessor = offsetVector.getAccessor();
+      /* check underlying offsets */
+      final ArrowBuf offsetBuffer = listVector.getOffsetBuffer();
 
       /* listVector has 2 lists at index 0 and 3 lists at index 1 */
-      assertEquals(0, offsetAccessor.get(0));
-      assertEquals(2, offsetAccessor.get(1));
-      assertEquals(5, offsetAccessor.get(2));
+      assertEquals(0, offsetBuffer.getInt(0 * ListVector.OFFSET_WIDTH));
+      assertEquals(2, offsetBuffer.getInt(1 * ListVector.OFFSET_WIDTH));
+      assertEquals(5, offsetBuffer.getInt(2 * ListVector.OFFSET_WIDTH));
     }
   }
 
@@ -591,17 +581,15 @@ public class TestListVector {
       listWriter.bigInt().writeBigInt(300);
       listWriter.endList();
 
-      final ListVector.Accessor accessor = listVector.getAccessor();
-
       /* check listVector contents */
-      Object result = accessor.getObject(0);
+      Object result = listVector.getObject(0);
       ArrayList<Long> resultSet = (ArrayList<Long>) result;
       assertEquals(3, resultSet.size());
       assertEquals(new Long(50), resultSet.get(0));
       assertEquals(new Long(100), resultSet.get(1));
       assertEquals(new Long(200), resultSet.get(2));
 
-      result = accessor.getObject(1);
+      result = listVector.getObject(1);
       resultSet = (ArrayList<Long>) result;
       assertEquals(2, resultSet.size());
       assertEquals(new Long(250), resultSet.get(0));
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
index 86f0bf3..ede4f99 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestUnionVector.java
@@ -64,26 +64,24 @@ public class TestUnionVector {
       unionVector.allocateNew();
 
       // write some data
-      final UnionVector.Mutator mutator = unionVector.getMutator();
-      mutator.setType(0, Types.MinorType.UINT4);
-      mutator.setSafe(0, uInt4Holder);
-      mutator.setType(2, Types.MinorType.UINT4);
-      mutator.setSafe(2, uInt4Holder);
-      mutator.setValueCount(4);
+      unionVector.setType(0, Types.MinorType.UINT4);
+      unionVector.setSafe(0, uInt4Holder);
+      unionVector.setType(2, Types.MinorType.UINT4);
+      unionVector.setSafe(2, uInt4Holder);
+      unionVector.setValueCount(4);
 
       // check that what we wrote is correct
-      final UnionVector.Accessor accessor = unionVector.getAccessor();
-      assertEquals(4, accessor.getValueCount());
+      assertEquals(4, unionVector.getValueCount());
 
-      assertEquals(false, accessor.isNull(0));
-      assertEquals(100, accessor.getObject(0));
+      assertEquals(false, unionVector.isNull(0));
+      assertEquals(100, unionVector.getObject(0));
 
-      assertEquals(true, accessor.isNull(1));
+      assertEquals(true, unionVector.isNull(1));
 
-      assertEquals(false, accessor.isNull(2));
-      assertEquals(100, accessor.getObject(2));
+      assertEquals(false, unionVector.isNull(2));
+      assertEquals(100, unionVector.getObject(2));
 
-      assertEquals(true, accessor.isNull(3));
+      assertEquals(true, unionVector.isNull(3));
     }
   }
 
@@ -93,16 +91,15 @@ public class TestUnionVector {
       srcVector.allocateNew();
 
       // write some data
-      final UnionVector.Mutator mutator = srcVector.getMutator();
-      mutator.setType(0, MinorType.INT);
-      mutator.setSafe(0, newIntHolder(5));
-      mutator.setType(1, MinorType.BIT);
-      mutator.setSafe(1, newBitHolder(false));
-      mutator.setType(3, MinorType.INT);
-      mutator.setSafe(3, newIntHolder(10));
-      mutator.setType(5, MinorType.BIT);
-      mutator.setSafe(5, newBitHolder(false));
-      mutator.setValueCount(6);
+      srcVector.setType(0, MinorType.INT);
+      srcVector.setSafe(0, newIntHolder(5));
+      srcVector.setType(1, MinorType.BIT);
+      srcVector.setSafe(1, newBitHolder(false));
+      srcVector.setType(3, MinorType.INT);
+      srcVector.setSafe(3, newIntHolder(10));
+      srcVector.setType(5, MinorType.BIT);
+      srcVector.setSafe(5, newBitHolder(false));
+      srcVector.setValueCount(6);
 
       try (UnionVector destVector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
         TransferPair pair = srcVector.makeTransferPair(destVector);
@@ -116,9 +113,9 @@ public class TestUnionVector {
         assertEquals(srcVector.getField(), destVector.getField());
 
         // now check the values are transferred
-        assertEquals(srcVector.getAccessor().getValueCount(), destVector.getAccessor().getValueCount());
-        for (int i = 0; i < srcVector.getAccessor().getValueCount(); i++) {
-          assertEquals("Different values at index " + i, srcVector.getAccessor().get(i), destVector.getAccessor().get(i));
+        assertEquals(srcVector.getValueCount(), destVector.getValueCount());
+        for (int i = 0; i < srcVector.getValueCount(); i++) {
+          assertEquals("Different values at index " + i, srcVector.get(i), destVector.get(i));
         }
       }
     }
@@ -127,61 +124,58 @@ public class TestUnionVector {
   @Test
   public void testSplitAndTransfer() throws Exception {
     try (UnionVector sourceVector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
-      final UnionVector.Mutator sourceMutator = sourceVector.getMutator();
-      final UnionVector.Accessor sourceAccessor = sourceVector.getAccessor();
 
       sourceVector.allocateNew();
 
       /* populate the UnionVector */
-      sourceMutator.setType(0, MinorType.INT);
-      sourceMutator.setSafe(0, newIntHolder(5));
-      sourceMutator.setType(1, MinorType.INT);
-      sourceMutator.setSafe(1, newIntHolder(10));
-      sourceMutator.setType(2, MinorType.INT);
-      sourceMutator.setSafe(2, newIntHolder(15));
-      sourceMutator.setType(3, MinorType.INT);
-      sourceMutator.setSafe(3, newIntHolder(20));
-      sourceMutator.setType(4, MinorType.INT);
-      sourceMutator.setSafe(4, newIntHolder(25));
-      sourceMutator.setType(5, MinorType.INT);
-      sourceMutator.setSafe(5, newIntHolder(30));
-      sourceMutator.setType(6, MinorType.INT);
-      sourceMutator.setSafe(6, newIntHolder(35));
-      sourceMutator.setType(7, MinorType.INT);
-      sourceMutator.setSafe(7, newIntHolder(40));
-      sourceMutator.setType(8, MinorType.INT);
-      sourceMutator.setSafe(8, newIntHolder(45));
-      sourceMutator.setType(9, MinorType.INT);
-      sourceMutator.setSafe(9, newIntHolder(50));
-      sourceMutator.setValueCount(10);
+      sourceVector.setType(0, MinorType.INT);
+      sourceVector.setSafe(0, newIntHolder(5));
+      sourceVector.setType(1, MinorType.INT);
+      sourceVector.setSafe(1, newIntHolder(10));
+      sourceVector.setType(2, MinorType.INT);
+      sourceVector.setSafe(2, newIntHolder(15));
+      sourceVector.setType(3, MinorType.INT);
+      sourceVector.setSafe(3, newIntHolder(20));
+      sourceVector.setType(4, MinorType.INT);
+      sourceVector.setSafe(4, newIntHolder(25));
+      sourceVector.setType(5, MinorType.INT);
+      sourceVector.setSafe(5, newIntHolder(30));
+      sourceVector.setType(6, MinorType.INT);
+      sourceVector.setSafe(6, newIntHolder(35));
+      sourceVector.setType(7, MinorType.INT);
+      sourceVector.setSafe(7, newIntHolder(40));
+      sourceVector.setType(8, MinorType.INT);
+      sourceVector.setSafe(8, newIntHolder(45));
+      sourceVector.setType(9, MinorType.INT);
+      sourceVector.setSafe(9, newIntHolder(50));
+      sourceVector.setValueCount(10);
 
       /* check the vector output */
-      assertEquals(10, sourceAccessor.getValueCount());
-      assertEquals(false, sourceAccessor.isNull(0));
-      assertEquals(5, sourceAccessor.getObject(0));
-      assertEquals(false, sourceAccessor.isNull(1));
-      assertEquals(10, sourceAccessor.getObject(1));
-      assertEquals(false, sourceAccessor.isNull(2));
-      assertEquals(15, sourceAccessor.getObject(2));
-      assertEquals(false, sourceAccessor.isNull(3));
-      assertEquals(20, sourceAccessor.getObject(3));
-      assertEquals(false, sourceAccessor.isNull(4));
-      assertEquals(25, sourceAccessor.getObject(4));
-      assertEquals(false, sourceAccessor.isNull(5));
-      assertEquals(30, sourceAccessor.getObject(5));
-      assertEquals(false, sourceAccessor.isNull(6));
-      assertEquals(35, sourceAccessor.getObject(6));
-      assertEquals(false, sourceAccessor.isNull(7));
-      assertEquals(40, sourceAccessor.getObject(7));
-      assertEquals(false, sourceAccessor.isNull(8));
-      assertEquals(45, sourceAccessor.getObject(8));
-      assertEquals(false, sourceAccessor.isNull(9));
-      assertEquals(50, sourceAccessor.getObject(9));
+      assertEquals(10, sourceVector.getValueCount());
+      assertEquals(false, sourceVector.isNull(0));
+      assertEquals(5, sourceVector.getObject(0));
+      assertEquals(false, sourceVector.isNull(1));
+      assertEquals(10, sourceVector.getObject(1));
+      assertEquals(false, sourceVector.isNull(2));
+      assertEquals(15, sourceVector.getObject(2));
+      assertEquals(false, sourceVector.isNull(3));
+      assertEquals(20, sourceVector.getObject(3));
+      assertEquals(false, sourceVector.isNull(4));
+      assertEquals(25, sourceVector.getObject(4));
+      assertEquals(false, sourceVector.isNull(5));
+      assertEquals(30, sourceVector.getObject(5));
+      assertEquals(false, sourceVector.isNull(6));
+      assertEquals(35, sourceVector.getObject(6));
+      assertEquals(false, sourceVector.isNull(7));
+      assertEquals(40, sourceVector.getObject(7));
+      assertEquals(false, sourceVector.isNull(8));
+      assertEquals(45, sourceVector.getObject(8));
+      assertEquals(false, sourceVector.isNull(9));
+      assertEquals(50, sourceVector.getObject(9));
 
       try (UnionVector toVector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
 
         final TransferPair transferPair = sourceVector.makeTransferPair(toVector);
-        final UnionVector.Accessor toAccessor = toVector.getAccessor();
 
         final int[][] transferLengths = {{0, 3},
             {3, 1},
@@ -199,8 +193,8 @@ public class TestUnionVector {
 
           /* check the toVector output after doing the splitAndTransfer */
           for (int i = 0; i < length; i++) {
-            assertEquals("Different data at indexes: " + (start + i) + "and " + i, sourceAccessor.getObject(start + i),
-                toAccessor.getObject(i));
+            assertEquals("Different data at indexes: " + (start + i) + "and " + i, sourceVector.getObject(start + i),
+                toVector.getObject(i));
           }
         }
       }
@@ -210,70 +204,67 @@ public class TestUnionVector {
   @Test
   public void testSplitAndTransferWithMixedVectors() throws Exception {
     try (UnionVector sourceVector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
-      final UnionVector.Mutator sourceMutator = sourceVector.getMutator();
-      final UnionVector.Accessor sourceAccessor = sourceVector.getAccessor();
 
       sourceVector.allocateNew();
 
       /* populate the UnionVector */
-      sourceMutator.setType(0, MinorType.INT);
-      sourceMutator.setSafe(0, newIntHolder(5));
+      sourceVector.setType(0, MinorType.INT);
+      sourceVector.setSafe(0, newIntHolder(5));
 
-      sourceMutator.setType(1, MinorType.FLOAT4);
-      sourceMutator.setSafe(1, newFloat4Holder(5.5f));
+      sourceVector.setType(1, MinorType.FLOAT4);
+      sourceVector.setSafe(1, newFloat4Holder(5.5f));
 
-      sourceMutator.setType(2, MinorType.INT);
-      sourceMutator.setSafe(2, newIntHolder(10));
+      sourceVector.setType(2, MinorType.INT);
+      sourceVector.setSafe(2, newIntHolder(10));
 
-      sourceMutator.setType(3, MinorType.FLOAT4);
-      sourceMutator.setSafe(3, newFloat4Holder(10.5f));
+      sourceVector.setType(3, MinorType.FLOAT4);
+      sourceVector.setSafe(3, newFloat4Holder(10.5f));
 
-      sourceMutator.setType(4, MinorType.INT);
-      sourceMutator.setSafe(4, newIntHolder(15));
+      sourceVector.setType(4, MinorType.INT);
+      sourceVector.setSafe(4, newIntHolder(15));
 
-      sourceMutator.setType(5, MinorType.FLOAT4);
-      sourceMutator.setSafe(5, newFloat4Holder(15.5f));
+      sourceVector.setType(5, MinorType.FLOAT4);
+      sourceVector.setSafe(5, newFloat4Holder(15.5f));
 
-      sourceMutator.setType(6, MinorType.INT);
-      sourceMutator.setSafe(6, newIntHolder(20));
+      sourceVector.setType(6, MinorType.INT);
+      sourceVector.setSafe(6, newIntHolder(20));
 
-      sourceMutator.setType(7, MinorType.FLOAT4);
-      sourceMutator.setSafe(7, newFloat4Holder(20.5f));
+      sourceVector.setType(7, MinorType.FLOAT4);
+      sourceVector.setSafe(7, newFloat4Holder(20.5f));
 
-      sourceMutator.setType(8, MinorType.INT);
-      sourceMutator.setSafe(8, newIntHolder(30));
+      sourceVector.setType(8, MinorType.INT);
+      sourceVector.setSafe(8, newIntHolder(30));
 
-      sourceMutator.setType(9, MinorType.FLOAT4);
-      sourceMutator.setSafe(9, newFloat4Holder(30.5f));
-      sourceMutator.setValueCount(10);
+      sourceVector.setType(9, MinorType.FLOAT4);
+      sourceVector.setSafe(9, newFloat4Holder(30.5f));
+      sourceVector.setValueCount(10);
 
       /* check the vector output */
-      assertEquals(10, sourceAccessor.getValueCount());
-      assertEquals(false, sourceAccessor.isNull(0));
-      assertEquals(5, sourceAccessor.getObject(0));
-      assertEquals(false, sourceAccessor.isNull(1));
-      assertEquals(5.5f, sourceAccessor.getObject(1));
-      assertEquals(false, sourceAccessor.isNull(2));
-      assertEquals(10, sourceAccessor.getObject(2));
-      assertEquals(false, sourceAccessor.isNull(3));
-      assertEquals(10.5f, sourceAccessor.getObject(3));
-      assertEquals(false, sourceAccessor.isNull(4));
-      assertEquals(15, sourceAccessor.getObject(4));
-      assertEquals(false, sourceAccessor.isNull(5));
-      assertEquals(15.5f, sourceAccessor.getObject(5));
-      assertEquals(false, sourceAccessor.isNull(6));
-      assertEquals(20, sourceAccessor.getObject(6));
-      assertEquals(false, sourceAccessor.isNull(7));
-      assertEquals(20.5f, sourceAccessor.getObject(7));
-      assertEquals(false, sourceAccessor.isNull(8));
-      assertEquals(30, sourceAccessor.getObject(8));
-      assertEquals(false, sourceAccessor.isNull(9));
-      assertEquals(30.5f, sourceAccessor.getObject(9));
+      assertEquals(10, sourceVector.getValueCount());
+      assertEquals(false, sourceVector.isNull(0));
+      assertEquals(5, sourceVector.getObject(0));
+      assertEquals(false, sourceVector.isNull(1));
+      assertEquals(5.5f, sourceVector.getObject(1));
+      assertEquals(false, sourceVector.isNull(2));
+      assertEquals(10, sourceVector.getObject(2));
+      assertEquals(false, sourceVector.isNull(3));
+      assertEquals(10.5f, sourceVector.getObject(3));
+      assertEquals(false, sourceVector.isNull(4));
+      assertEquals(15, sourceVector.getObject(4));
+      assertEquals(false, sourceVector.isNull(5));
+      assertEquals(15.5f, sourceVector.getObject(5));
+      assertEquals(false, sourceVector.isNull(6));
+      assertEquals(20, sourceVector.getObject(6));
+      assertEquals(false, sourceVector.isNull(7));
+      assertEquals(20.5f, sourceVector.getObject(7));
+      assertEquals(false, sourceVector.isNull(8));
+      assertEquals(30, sourceVector.getObject(8));
+      assertEquals(false, sourceVector.isNull(9));
+      assertEquals(30.5f, sourceVector.getObject(9));
 
       try (UnionVector toVector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
 
         final TransferPair transferPair = sourceVector.makeTransferPair(toVector);
-        final UnionVector.Accessor toAccessor = toVector.getAccessor();
 
         final int[][] transferLengths = {{0, 2},
             {2, 1},
@@ -290,7 +281,7 @@ public class TestUnionVector {
 
           /* check the toVector output after doing the splitAndTransfer */
           for (int i = 0; i < length; i++) {
-            assertEquals("Different values at index: " + i, sourceAccessor.getObject(start + i), toAccessor.getObject(i));
+            assertEquals("Different values at index: " + i, sourceVector.getObject(start + i), toVector.getObject(i));
           }
         }
       }
@@ -300,37 +291,35 @@ public class TestUnionVector {
   @Test
   public void testGetBufferAddress() throws Exception {
     try (UnionVector vector = new UnionVector(EMPTY_SCHEMA_PATH, allocator, null)) {
-      final UnionVector.Mutator mutator = vector.getMutator();
-      final UnionVector.Accessor accessor = vector.getAccessor();
       boolean error = false;
 
       vector.allocateNew();
 
       /* populate the UnionVector */
-      mutator.setType(0, MinorType.INT);
-      mutator.setSafe(0, newIntHolder(5));
+      vector.setType(0, MinorType.INT);
+      vector.setSafe(0, newIntHolder(5));
 
-      mutator.setType(1, MinorType.FLOAT4);
-      mutator.setSafe(1, newFloat4Holder(5.5f));
+      vector.setType(1, MinorType.FLOAT4);
+      vector.setSafe(1, newFloat4Holder(5.5f));
 
-      mutator.setType(2, MinorType.INT);
-      mutator.setSafe(2, newIntHolder(10));
+      vector.setType(2, MinorType.INT);
+      vector.setSafe(2, newIntHolder(10));
 
-      mutator.setType(3, MinorType.FLOAT4);
-      mutator.setSafe(3, newFloat4Holder(10.5f));
+      vector.setType(3, MinorType.FLOAT4);
+      vector.setSafe(3, newFloat4Holder(10.5f));
 
-      mutator.setValueCount(10);
+      vector.setValueCount(10);
 
       /* check the vector output */
-      assertEquals(10, accessor.getValueCount());
-      assertEquals(false, accessor.isNull(0));
-      assertEquals(5, accessor.getObject(0));
-      assertEquals(false, accessor.isNull(1));
-      assertEquals(5.5f, accessor.getObject(1));
-      assertEquals(false, accessor.isNull(2));
-      assertEquals(10, accessor.getObject(2));
-      assertEquals(false, accessor.isNull(3));
-      assertEquals(10.5f, accessor.getObject(3));
+      assertEquals(10, vector.getValueCount());
+      assertEquals(false, vector.isNull(0));
+      assertEquals(5, vector.getObject(0));
+      assertEquals(false, vector.isNull(1));
+      assertEquals(5.5f, vector.getObject(1));
+      assertEquals(false, vector.isNull(2));
+      assertEquals(10, vector.getObject(2));
+      assertEquals(false, vector.isNull(3));
+      assertEquals(10.5f, vector.getObject(3));
 
       List<ArrowBuf> buffers = vector.getFieldBuffers();
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
index b7f88c3..c7ee202 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
@@ -487,8 +487,6 @@ public class TestValueVector {
 
     // Create a new value vector for 1024 integers.
     try (final NullableUInt4Vector vector = newVector(NullableUInt4Vector.class, EMPTY_SCHEMA_PATH, new ArrowType.Int(32, false), allocator);) {
-      final NullableUInt4Vector.Mutator mutator = vector.getMutator();
-      final NullableUInt4Vector.Accessor accessor = vector.getAccessor();
       boolean error = false;
       int initialCapacity = 1024;
 
@@ -500,33 +498,33 @@ public class TestValueVector {
       assertEquals(initialCapacity, vector.getValueCapacity());
 
       // Put and set a few values
-      mutator.set(0, 100);
-      mutator.set(1, 101);
-      mutator.set(100, 102);
-      mutator.set(1022, 103);
-      mutator.set(1023, 104);
+      vector.set(0, 100);
+      vector.set(1, 101);
+      vector.set(100, 102);
+      vector.set(1022, 103);
+      vector.set(1023, 104);
 
       /* check vector contents */
-      assertEquals(100, accessor.get(0));
-      assertEquals(101, accessor.get(1));
-      assertEquals(102, accessor.get(100));
-      assertEquals(103, accessor.get(1022));
-      assertEquals(104, accessor.get(1023));
+      assertEquals(100, vector.get(0));
+      assertEquals(101, vector.get(1));
+      assertEquals(102, vector.get(100));
+      assertEquals(103, vector.get(1022));
+      assertEquals(104, vector.get(1023));
 
       int val = 0;
 
       /* check unset bits/null values */
       for (int i = 2, j = 101; i <= 99 || j <= 1021; i++, j++) {
         if (i <= 99) {
-          assertTrue(accessor.isNull(i));
+          assertTrue(vector.isNull(i));
         }
         if(j <= 1021) {
-          assertTrue(accessor.isNull(j));
+          assertTrue(vector.isNull(j));
         }
       }
 
       try {
-        mutator.set(1024, 10000);
+        vector.set(1024, 10000);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -537,7 +535,7 @@ public class TestValueVector {
       }
 
       try {
-        accessor.get(1024);
+        vector.get(1024);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -548,28 +546,28 @@ public class TestValueVector {
       }
 
       /* should trigger a realloc of the underlying bitvector and valuevector */
-      mutator.setSafe(1024, 10000);
+      vector.setSafe(1024, 10000);
 
       /* check new capacity */
       assertEquals(initialCapacity * 2, vector.getValueCapacity());
 
       /* vector contents should still be intact after realloc */
-      assertEquals(100, accessor.get(0));
-      assertEquals(101, accessor.get(1));
-      assertEquals(102, accessor.get(100));
-      assertEquals(103, accessor.get(1022));
-      assertEquals(104, accessor.get(1023));
-      assertEquals(10000, accessor.get(1024));
+      assertEquals(100, vector.get(0));
+      assertEquals(101, vector.get(1));
+      assertEquals(102, vector.get(100));
+      assertEquals(103, vector.get(1022));
+      assertEquals(104, vector.get(1023));
+      assertEquals(10000, vector.get(1024));
 
       val = 0;
 
       /* check unset bits/null values */
       for (int i = 2, j = 101; i < 99 || j < 1021; i++, j++) {
         if (i <= 99) {
-          assertTrue(accessor.isNull(i));
+          assertTrue(vector.isNull(i));
         }
         if(j <= 1021) {
-          assertTrue(accessor.isNull(j));
+          assertTrue(vector.isNull(j));
         }
       }
 
@@ -581,7 +579,7 @@ public class TestValueVector {
 
       /* vector data should be zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertTrue("non-null data not expected at index: " + i, accessor.isNull(i));
+        assertTrue("non-null data not expected at index: " + i, vector.isNull(i));
       }
     }
   }
@@ -590,8 +588,6 @@ public class TestValueVector {
   public void testNullableFixedType2() {
     // Create a new value vector for 1024 integers
     try (final NullableFloat4Vector vector = newVector(NullableFloat4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator);) {
-      final NullableFloat4Vector.Mutator mutator = vector.getMutator();
-      final NullableFloat4Vector.Accessor accessor = vector.getAccessor();
       boolean error = false;
       int initialCapacity = 16;
 
@@ -603,17 +599,17 @@ public class TestValueVector {
       assertEquals(initialCapacity, vector.getValueCapacity());
 
       /* populate the vector */
-      mutator.set(0, 100.5f);
-      mutator.set(2, 201.5f);
-      mutator.set(4, 300.3f);
-      mutator.set(6, 423.8f);
-      mutator.set(8, 555.6f);
-      mutator.set(10, 66.6f);
-      mutator.set(12, 78.8f);
-      mutator.set(14, 89.5f);
+      vector.set(0, 100.5f);
+      vector.set(2, 201.5f);
+      vector.set(4, 300.3f);
+      vector.set(6, 423.8f);
+      vector.set(8, 555.6f);
+      vector.set(10, 66.6f);
+      vector.set(12, 78.8f);
+      vector.set(14, 89.5f);
 
       try {
-        mutator.set(16, 90.5f);
+        vector.set(16, 90.5f);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -624,25 +620,25 @@ public class TestValueVector {
       }
 
       /* check vector contents */
-      assertEquals(100.5f, accessor.get(0), 0);
-      assertTrue(accessor.isNull(1));
-      assertEquals(201.5f, accessor.get(2), 0);
-      assertTrue(accessor.isNull(3));
-      assertEquals(300.3f, accessor.get(4), 0);
-      assertTrue(accessor.isNull(5));
-      assertEquals(423.8f, accessor.get(6), 0);
-      assertTrue(accessor.isNull(7));
-      assertEquals(555.6f, accessor.get(8), 0);
-      assertTrue(accessor.isNull(9));
-      assertEquals(66.6f, accessor.get(10), 0);
-      assertTrue(accessor.isNull(11));
-      assertEquals(78.8f, accessor.get(12), 0);
-      assertTrue(accessor.isNull(13));
-      assertEquals(89.5f, accessor.get(14), 0);
-      assertTrue(accessor.isNull(15));
+      assertEquals(100.5f, vector.get(0), 0);
+      assertTrue(vector.isNull(1));
+      assertEquals(201.5f, vector.get(2), 0);
+      assertTrue(vector.isNull(3));
+      assertEquals(300.3f, vector.get(4), 0);
+      assertTrue(vector.isNull(5));
+      assertEquals(423.8f, vector.get(6), 0);
+      assertTrue(vector.isNull(7));
+      assertEquals(555.6f, vector.get(8), 0);
+      assertTrue(vector.isNull(9));
+      assertEquals(66.6f, vector.get(10), 0);
+      assertTrue(vector.isNull(11));
+      assertEquals(78.8f, vector.get(12), 0);
+      assertTrue(vector.isNull(13));
+      assertEquals(89.5f, vector.get(14), 0);
+      assertTrue(vector.isNull(15));
 
       try {
-        accessor.get(16);
+        vector.get(16);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -653,29 +649,28 @@ public class TestValueVector {
       }
 
       /* this should trigger a realloc() */
-      mutator.setSafe(16, 90.5f);
+      vector.setSafe(16, 90.5f);
 
       /* underlying buffer should now be able to store double the number of values */
       assertEquals(initialCapacity * 2, vector.getValueCapacity());
 
       /* vector data should still be intact after realloc */
-      assertEquals(100.5f, accessor.get(0), 0);
-      assertTrue(accessor.isNull(1));
-      assertEquals(201.5f, accessor.get(2), 0);
-      assertTrue(accessor.isNull(3));
-      assertEquals(300.3f, accessor.get(4), 0);
-      assertTrue(accessor.isNull(5));
-      assertEquals(423.8f, accessor.get(6), 0);
-      assertTrue(accessor.isNull(7));
-      assertEquals(555.6f, accessor.get(8), 0);
-      assertTrue(accessor.isNull(9));
-      assertEquals(66.6f, accessor.get(10), 0);
-      assertTrue(accessor.isNull(11));
-      assertEquals(78.8f, accessor.get(12), 0);
-      assertTrue(accessor.isNull(13));
-      assertEquals(89.5f, accessor.get(14), 0);
-      assertTrue(accessor.isNull(15));
-      assertEquals(90.5f, accessor.get(16), 0);
+      assertEquals(100.5f, vector.get(0), 0);
+      assertTrue(vector.isNull(1));
+      assertEquals(201.5f, vector.get(2), 0);
+      assertTrue(vector.isNull(3));
+      assertEquals(300.3f, vector.get(4), 0);
+      assertTrue(vector.isNull(5));
+      assertEquals(423.8f, vector.get(6), 0);
+      assertTrue(vector.isNull(7));
+      assertEquals(555.6f, vector.get(8), 0);
+      assertTrue(vector.isNull(9));
+      assertEquals(66.6f, vector.get(10), 0);
+      assertTrue(vector.isNull(11));
+      assertEquals(78.8f, vector.get(12), 0);
+      assertTrue(vector.isNull(13));
+      assertEquals(89.5f, vector.get(14), 0);
+      assertTrue(vector.isNull(15));
 
       /* reset the vector */
       vector.reset();
@@ -685,7 +680,7 @@ public class TestValueVector {
 
       /* vector data should be zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertTrue("non-null data not expected at index: " + i, accessor.isNull(i));
+        assertTrue("non-null data not expected at index: " + i, vector.isNull(i));
       }
     }
   }
@@ -928,32 +923,30 @@ public class TestValueVector {
 
     // Create a new value vector for 1024 integers.
     try (final NullableVarBinaryVector vector = newNullableVarBinaryVector(EMPTY_SCHEMA_PATH, allocator)) {
-      final NullableVarBinaryVector.Mutator m = vector.getMutator();
       vector.allocateNew(1024 * 10, 1024);
 
-      m.set(0, STR1);
-      m.set(1, STR2);
-      m.set(2, STR3);
-      m.setSafe(3, STR3, 1, STR3.length - 1);
-      m.setSafe(4, STR3, 2, STR3.length - 2);
+      vector.set(0, STR1);
+      vector.set(1, STR2);
+      vector.set(2, STR3);
+      vector.setSafe(3, STR3, 1, STR3.length - 1);
+      vector.setSafe(4, STR3, 2, STR3.length - 2);
       ByteBuffer STR3ByteBuffer = ByteBuffer.wrap(STR3);
-      m.setSafe(5, STR3ByteBuffer, 1, STR3.length - 1);
-      m.setSafe(6, STR3ByteBuffer, 2, STR3.length - 2);
+      vector.setSafe(5, STR3ByteBuffer, 1, STR3.length - 1);
+      vector.setSafe(6, STR3ByteBuffer, 2, STR3.length - 2);
 
       // Check the sample strings.
-      final NullableVarBinaryVector.Accessor accessor = vector.getAccessor();
-      assertArrayEquals(STR1, accessor.get(0));
-      assertArrayEquals(STR2, accessor.get(1));
-      assertArrayEquals(STR3, accessor.get(2));
-      assertArrayEquals(Arrays.copyOfRange(STR3, 1, STR3.length), accessor.get(3));
-      assertArrayEquals(Arrays.copyOfRange(STR3, 2, STR3.length), accessor.get(4));
-      assertArrayEquals(Arrays.copyOfRange(STR3, 1, STR3.length), accessor.get(5));
-      assertArrayEquals(Arrays.copyOfRange(STR3, 2, STR3.length), accessor.get(6));
+      assertArrayEquals(STR1, vector.get(0));
+      assertArrayEquals(STR2, vector.get(1));
+      assertArrayEquals(STR3, vector.get(2));
+      assertArrayEquals(Arrays.copyOfRange(STR3, 1, STR3.length), vector.get(3));
+      assertArrayEquals(Arrays.copyOfRange(STR3, 2, STR3.length), vector.get(4));
+      assertArrayEquals(Arrays.copyOfRange(STR3, 1, STR3.length), vector.get(5));
+      assertArrayEquals(Arrays.copyOfRange(STR3, 2, STR3.length), vector.get(6));
 
       // Ensure null value throws.
       boolean b = false;
       try {
-        vector.getAccessor().get(7);
+        vector.get(7);
       } catch (IllegalStateException e) {
         b = true;
       } finally {
@@ -1064,8 +1057,6 @@ public class TestValueVector {
   @Test /* NullableFloat8Vector */
   public void testReallocAfterVectorTransfer2() {
     try (final NullableFloat8Vector vector = new NullableFloat8Vector(EMPTY_SCHEMA_PATH, allocator)) {
-      final NullableFloat8Vector.Mutator mutator = vector.getMutator();
-      final NullableFloat8Vector.Accessor accessor = vector.getAccessor();
       final int initialDefaultCapacity = 4096;
       boolean error = false;
 
@@ -1076,7 +1067,7 @@ public class TestValueVector {
       double baseValue = 100.375;
 
       for (int i = 0; i < initialDefaultCapacity; i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       /* the above setSafe calls should not have triggered a realloc as
@@ -1085,33 +1076,33 @@ public class TestValueVector {
       assertEquals(initialDefaultCapacity, vector.getValueCapacity());
 
       for (int i = 0; i < initialDefaultCapacity; i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
       /* this should trigger a realloc */
-      mutator.setSafe(initialDefaultCapacity, baseValue + (double)initialDefaultCapacity);
+      vector.setSafe(initialDefaultCapacity, baseValue + (double)initialDefaultCapacity);
       assertEquals(initialDefaultCapacity * 2, vector.getValueCapacity());
 
       for (int i = initialDefaultCapacity + 1; i < (initialDefaultCapacity * 2); i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       for (int i = 0; i < (initialDefaultCapacity * 2); i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
       /* this should trigger a realloc */
-      mutator.setSafe(initialDefaultCapacity * 2, baseValue + (double)(initialDefaultCapacity * 2));
+      vector.setSafe(initialDefaultCapacity * 2, baseValue + (double)(initialDefaultCapacity * 2));
       assertEquals(initialDefaultCapacity * 4, vector.getValueCapacity());
 
       for (int i = (initialDefaultCapacity * 2) + 1; i < (initialDefaultCapacity * 4); i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       for (int i = 0; i < (initialDefaultCapacity * 4); i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
@@ -1123,12 +1114,11 @@ public class TestValueVector {
       transferPair.transfer();
 
       NullableFloat8Vector toVector = (NullableFloat8Vector)transferPair.getTo();
-      final NullableFloat8Vector.Accessor toAccessor = toVector.getAccessor();
 
       /* check toVector contents before realloc */
       for (int i = 0; i < (initialDefaultCapacity * 4); i++) {
-        assertFalse("unexpected null value at index: " + i, toAccessor.isNull(i));
-        double value = toAccessor.get(i);
+        assertFalse("unexpected null value at index: " + i, toVector.isNull(i));
+        double value = toVector.get(i);
         assertEquals("unexpected value at index: " + i, baseValue + (double)i, value, 0);
       }
 
@@ -1138,12 +1128,12 @@ public class TestValueVector {
 
       for (int i = 0; i < (initialDefaultCapacity * 8); i++) {
         if (i < (initialDefaultCapacity * 4)) {
-          assertFalse("unexpected null value at index: " + i, toAccessor.isNull(i));
-          double value = toAccessor.get(i);
+          assertFalse("unexpected null value at index: " + i, toVector.isNull(i));
+          double value = toVector.get(i);
           assertEquals("unexpected value at index: " + i, baseValue + (double)i, value, 0);
         }
         else {
-          assertTrue("unexpected non-null value at index: " + i, toAccessor.isNull(i));
+          assertTrue("unexpected non-null value at index: " + i, toVector.isNull(i));
         }
       }
 
@@ -1351,32 +1341,30 @@ public class TestValueVector {
   public void testReAllocNullableFixedWidthVector() {
     // Create a new value vector for 1024 integers
     try (final NullableFloat4Vector vector = newVector(NullableFloat4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator)) {
-      final NullableFloat4Vector.Mutator m = vector.getMutator();
       vector.allocateNew(1024);
 
       assertEquals(1024, vector.getValueCapacity());
 
       // Put values in indexes that fall within the initial allocation
-      m.setSafe(0, 100.1f);
-      m.setSafe(100, 102.3f);
-      m.setSafe(1023, 104.5f);
+      vector.setSafe(0, 100.1f);
+      vector.setSafe(100, 102.3f);
+      vector.setSafe(1023, 104.5f);
 
       // Now try to put values in space that falls beyond the initial allocation
-      m.setSafe(2000, 105.5f);
+      vector.setSafe(2000, 105.5f);
 
       // Check valueCapacity is more than initial allocation
       assertEquals(1024 * 2, vector.getValueCapacity());
 
-      final NullableFloat4Vector.Accessor accessor = vector.getAccessor();
-      assertEquals(100.1f, accessor.get(0), 0);
-      assertEquals(102.3f, accessor.get(100), 0);
-      assertEquals(104.5f, accessor.get(1023), 0);
-      assertEquals(105.5f, accessor.get(2000), 0);
+      assertEquals(100.1f, vector.get(0), 0);
+      assertEquals(102.3f, vector.get(100), 0);
+      assertEquals(104.5f, vector.get(1023), 0);
+      assertEquals(105.5f, vector.get(2000), 0);
 
       // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors
       // as we don't call setSafe for null values, but we do call setValueCount when all values are inserted into the
       // vector
-      m.setValueCount(vector.getValueCapacity() + 200);
+      vector.setValueCount(vector.getValueCapacity() + 200);
     }
   }
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
index 531a46c..293ffbf 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
@@ -28,6 +28,7 @@ import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.NullableMapVector;
 import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.junit.After;
 import org.junit.Assert;
@@ -115,7 +116,7 @@ public class TestVectorReAlloc {
 
       vector.reAlloc();
       assertEquals(2047, vector.getValueCapacity()); // note: size - 1
-      assertEquals(0, vector.getOffsetVector().getAccessor().get(2014));
+      assertEquals(0, vector.getOffsetBuffer().getInt(2014 * ListVector.OFFSET_WIDTH));
     }
   }
 
@@ -138,7 +139,7 @@ public class TestVectorReAlloc {
 
       vector.reAlloc();
       assertEquals(1024, vector.getValueCapacity());
-      assertNull(vector.getAccessor().getObject(513));
+      assertNull(vector.getObject(513));
     }
   }
 }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
index 0b7928d..3853eec 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
@@ -199,8 +199,10 @@ public class TestVectorUnloadLoad {
     int count = 10;
     ArrowBuf[] values = new ArrowBuf[4];
     for (int i = 0; i < 4; i+=2) {
-      ArrowBuf buf1 = allocator.buffer((int)Math.ceil(count / 8.0));
+      ArrowBuf buf1 = allocator.buffer(BitVectorHelper.getValidityBufferSize(count));
       ArrowBuf buf2 = allocator.buffer(count * 4); // integers
+      buf1.setZero(0, buf1.capacity());
+      buf2.setZero(0, buf2.capacity());
       values[i] = buf1;
       values[i+1] = buf2;
       for (int j = 0; j < count; j++) {
@@ -314,7 +316,7 @@ public class TestVectorUnloadLoad {
 
   public static VectorUnloader newVectorUnloader(FieldVector root) {
     Schema schema = new Schema(root.getField().getChildren());
-    int valueCount = root.getAccessor().getValueCount();
+    int valueCount = root.getValueCount();
     List<FieldVector> fields = root.getChildrenFromFields();
     VectorSchemaRoot vsr = new VectorSchemaRoot(schema.getFields(), fields, valueCount);
     return new VectorUnloader(vsr);
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java b/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
index 97efb7d..b0d6cf5 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
@@ -80,24 +80,23 @@ public class TestPromotableWriter {
 
       writer.end();
 
-      container.getMutator().setValueCount(5);
+      container.setValueCount(5);
 
       final UnionVector uv = v.getChild("A", UnionVector.class);
-      final UnionVector.Accessor accessor = uv.getAccessor();
 
-      assertFalse("0 shouldn't be null", accessor.isNull(0));
-      assertEquals(false, accessor.getObject(0));
+      assertFalse("0 shouldn't be null", uv.isNull(0));
+      assertEquals(false, uv.getObject(0));
 
-      assertFalse("1 shouldn't be null", accessor.isNull(1));
-      assertEquals(true, accessor.getObject(1));
+      assertFalse("1 shouldn't be null", uv.isNull(1));
+      assertEquals(true, uv.getObject(1));
 
-      assertFalse("2 shouldn't be null", accessor.isNull(2));
-      assertEquals(10, accessor.getObject(2));
+      assertFalse("2 shouldn't be null", uv.isNull(2));
+      assertEquals(10, uv.getObject(2));
 
-      assertTrue("3 should be null", accessor.isNull(3));
+      assertTrue("3 should be null", uv.isNull(3));
 
-      assertFalse("4 shouldn't be null", accessor.isNull(4));
-      assertEquals(100, accessor.getObject(4));
+      assertFalse("4 shouldn't be null", uv.isNull(4));
+      assertEquals(100, uv.getObject(4));
 
       container.clear();
       container.allocateNew();
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java b/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
index 856d607..52defdc 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
@@ -456,7 +456,7 @@ public class TestComplexWriter {
         unionWriter.writeFloat4((float) i);
       }
     }
-    vector.getMutator().setValueCount(COUNT);
+    vector.setValueCount(COUNT);
     UnionReader unionReader = new UnionReader(vector);
     for (int i = 0; i < COUNT; i++) {
       unionReader.setPosition(i);
@@ -834,7 +834,7 @@ public class TestComplexWriter {
     TransferPair tp = mapVector.getTransferPair(allocator);
     tp.splitAndTransfer(0, 1);
     MapVector toMapVector = (MapVector) tp.getTo();
-    JsonStringHashMap<?, ?> toMapValue = (JsonStringHashMap<?, ?>) toMapVector.getAccessor().getObject(0);
+    JsonStringHashMap<?, ?> toMapValue = (JsonStringHashMap<?, ?>) toMapVector.getObject(0);
     JsonStringArrayList<?> object = (JsonStringArrayList<?>) toMapValue.get("list");
     assertEquals(1, object.get(0));
     assertEquals(2, object.get(1));
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java b/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
index 60009b0..874ba99 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
@@ -97,14 +97,8 @@ public class BaseFileTest {
 
   protected void validateContent(int count, VectorSchemaRoot root) {
     for (int i = 0; i < count; i++) {
-      FieldVector fv = root.getVector("int");
-      if (fv instanceof NullableIntVector) {
-        Assert.assertEquals(i, fv.getObject(i));
-        Assert.assertEquals(Integer.valueOf(i), fv.getObject(i));
-      } else {
-        Assert.assertEquals(i, fv.getAccessor().getObject(i));
-        Assert.assertEquals(Long.valueOf(i), fv.getAccessor().getObject(i));
-      }
+      Assert.assertEquals(i, root.getVector("int").getObject(i));
+      Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getObject(i));
     }
   }
 
@@ -146,10 +140,9 @@ public class BaseFileTest {
   public void printVectors(List<FieldVector> vectors) {
     for (FieldVector vector : vectors) {
       LOGGER.debug(vector.getField().getName());
-      Accessor accessor = vector.getAccessor();
-      int valueCount = accessor.getValueCount();
+      int valueCount = vector.getValueCount();
       for (int i = 0; i < valueCount; i++) {
-        LOGGER.debug(String.valueOf(accessor.getObject(i)));
+        LOGGER.debug(String.valueOf(vector.getObject(i)));
       }
     }
   }
@@ -159,14 +152,14 @@ public class BaseFileTest {
     printVectors(root.getFieldVectors());
     for (int i = 0; i < count; i++) {
 
-      Object intVal = root.getVector("int").getAccessor().getObject(i);
+      Object intVal = root.getVector("int").getObject(i);
       if (i % 5 != 3) {
         Assert.assertEquals(i, intVal);
       } else {
         Assert.assertNull(intVal);
       }
-      Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getAccessor().getObject(i));
-      Assert.assertEquals(i % 3, ((List<?>) root.getVector("list").getAccessor().getObject(i)).size());
+      Assert.assertEquals(Long.valueOf(i), root.getVector("bigInt").getObject(i));
+      Assert.assertEquals(i % 3, ((List<?>) root.getVector("list").getObject(i)).size());
       NullableTimeStampMilliHolder h = new NullableTimeStampMilliHolder();
       FieldReader mapReader = root.getVector("map").getReader();
       mapReader.setPosition(i);
@@ -209,15 +202,15 @@ public class BaseFileTest {
     Assert.assertEquals(count, root.getRowCount());
     printVectors(root.getFieldVectors());
     for (int i = 0; i < count; i++) {
-      long dateVal = ((NullableDateMilliVector) root.getVector("date")).getAccessor().get(i);
+      long dateVal = ((NullableDateMilliVector) root.getVector("date")).get(i);
       LocalDateTime dt = makeDateTimeFromCount(i);
       LocalDateTime dateExpected = dt.minusMillis(dt.getMillisOfDay());
       Assert.assertEquals(DateUtility.toMillis(dateExpected), dateVal);
-      long timeVal = ((NullableTimeMilliVector) root.getVector("time")).getAccessor().get(i);
+      long timeVal = ((NullableTimeMilliVector) root.getVector("time")).get(i);
       Assert.assertEquals(dt.getMillisOfDay(), timeVal);
-      Object timestampMilliVal = root.getVector("timestamp-milli").getAccessor().getObject(i);
+      Object timestampMilliVal = root.getVector("timestamp-milli").getObject(i);
       Assert.assertEquals(dt, timestampMilliVal);
-      Object timestampMilliTZVal = root.getVector("timestamp-milliTZ").getAccessor().getObject(i);
+      Object timestampMilliTZVal = root.getVector("timestamp-milliTZ").getObject(i);
       Assert.assertEquals(DateUtility.toMillis(dt), timestampMilliTZVal);
     }
   }
@@ -282,7 +275,7 @@ public class BaseFileTest {
     List<Field> fields = ImmutableList.of(encodedVector1A.getField(), encodedVector1B.getField(), encodedVector2.getField());
     List<FieldVector> vectors = ImmutableList.of(encodedVector1A, encodedVector1B, encodedVector2);
 
-    return new VectorSchemaRoot(fields, vectors, encodedVector1A.getAccessor().getValueCount());
+    return new VectorSchemaRoot(fields, vectors, encodedVector1A.getValueCount());
   }
 
   protected void validateFlatDictionary(VectorSchemaRoot root, DictionaryProvider provider) {
@@ -293,14 +286,13 @@ public class BaseFileTest {
     Assert.assertNotNull(encoding1A);
     Assert.assertEquals(1L, encoding1A.getId());
 
-    FieldVector.Accessor accessor = vector1A.getAccessor();
-    Assert.assertEquals(6, accessor.getValueCount());
-    Assert.assertEquals(0, accessor.getObject(0));
-    Assert.assertEquals(1, accessor.getObject(1));
-    Assert.assertEquals(null, accessor.getObject(2));
-    Assert.assertEquals(2, accessor.getObject(3));
-    Assert.assertEquals(1, accessor.getObject(4));
-    Assert.assertEquals(2, accessor.getObject(5));
+    Assert.assertEquals(6, vector1A.getValueCount());
+    Assert.assertEquals(0, vector1A.getObject(0));
+    Assert.assertEquals(1, vector1A.getObject(1));
+    Assert.assertEquals(null, vector1A.getObject(2));
+    Assert.assertEquals(2, vector1A.getObject(3));
+    Assert.assertEquals(1, vector1A.getObject(4));
+    Assert.assertEquals(2, vector1A.getObject(5));
 
     FieldVector vector1B = root.getVector("varcharB");
     Assert.assertNotNull(vector1B);
@@ -310,14 +302,13 @@ public class BaseFileTest {
     Assert.assertTrue(encoding1A.equals(encoding1B));
     Assert.assertEquals(1L, encoding1B.getId());
 
-    accessor = vector1B.getAccessor();
-    Assert.assertEquals(6, accessor.getValueCount());
-    Assert.assertEquals(2, accessor.getObject(0));
-    Assert.assertEquals(1, accessor.getObject(1));
-    Assert.assertEquals(2, accessor.getObject(2));
-    Assert.assertEquals(null, accessor.getObject(3));
-    Assert.assertEquals(1, accessor.getObject(4));
-    Assert.assertEquals(0, accessor.getObject(5));
+    Assert.assertEquals(6, vector1B.getValueCount());
+    Assert.assertEquals(2, vector1B.getObject(0));
+    Assert.assertEquals(1, vector1B.getObject(1));
+    Assert.assertEquals(2, vector1B.getObject(2));
+    Assert.assertEquals(null, vector1B.getObject(3));
+    Assert.assertEquals(1, vector1B.getObject(4));
+    Assert.assertEquals(0, vector1B.getObject(5));
 
     FieldVector vector2 = root.getVector("sizes");
     Assert.assertNotNull(vector2);
@@ -326,30 +317,29 @@ public class BaseFileTest {
     Assert.assertNotNull(encoding2);
     Assert.assertEquals(2L, encoding2.getId());
 
-    accessor = vector2.getAccessor();
-    Assert.assertEquals(6, accessor.getValueCount());
-    Assert.assertEquals(null, accessor.getObject(0));
-    Assert.assertEquals(2, accessor.getObject(1));
-    Assert.assertEquals(1, accessor.getObject(2));
-    Assert.assertEquals(1, accessor.getObject(3));
-    Assert.assertEquals(2, accessor.getObject(4));
-    Assert.assertEquals(null, accessor.getObject(5));
+    Assert.assertEquals(6, vector2.getValueCount());
+    Assert.assertEquals(null, vector2.getObject(0));
+    Assert.assertEquals(2, vector2.getObject(1));
+    Assert.assertEquals(1, vector2.getObject(2));
+    Assert.assertEquals(1, vector2.getObject(3));
+    Assert.assertEquals(2, vector2.getObject(4));
+    Assert.assertEquals(null, vector2.getObject(5));
 
     Dictionary dictionary1 = provider.lookup(1L);
     Assert.assertNotNull(dictionary1);
-    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary1.getVector()).getAccessor();
-    Assert.assertEquals(3, dictionaryAccessor.getValueCount());
-    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
-    Assert.assertEquals(new Text("baz"), dictionaryAccessor.getObject(2));
+    NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary1.getVector());
+    Assert.assertEquals(3, dictionaryVector.getValueCount());
+    Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
+    Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
+    Assert.assertEquals(new Text("baz"), dictionaryVector.getObject(2));
 
     Dictionary dictionary2 = provider.lookup(2L);
     Assert.assertNotNull(dictionary2);
-    dictionaryAccessor = ((NullableVarCharVector) dictionary2.getVector()).getAccessor();
-    Assert.assertEquals(3, dictionaryAccessor.getValueCount());
-    Assert.assertEquals(new Text("micro"), dictionaryAccessor.getObject(0));
-    Assert.assertEquals(new Text("small"), dictionaryAccessor.getObject(1));
-    Assert.assertEquals(new Text("large"), dictionaryAccessor.getObject(2));
+    dictionaryVector = ((NullableVarCharVector) dictionary2.getVector());
+    Assert.assertEquals(3, dictionaryVector.getValueCount());
+    Assert.assertEquals(new Text("micro"), dictionaryVector.getObject(0));
+    Assert.assertEquals(new Text("small"), dictionaryVector.getObject(1));
+    Assert.assertEquals(new Text("large"), dictionaryVector.getObject(2));
   }
 
   protected VectorSchemaRoot writeNestedDictionaryData(BufferAllocator bufferAllocator, DictionaryProvider.MapDictionaryProvider provider) {
@@ -359,7 +349,7 @@ public class BaseFileTest {
     dictionaryVector.allocateNewSafe();
     dictionaryVector.set(0, "foo".getBytes(StandardCharsets.UTF_8));
     dictionaryVector.set(1, "bar".getBytes(StandardCharsets.UTF_8));
-    dictionaryVector.getMutator().setValueCount(2);
+    dictionaryVector.setValueCount(2);
 
     Dictionary dictionary = new Dictionary(dictionaryVector, new DictionaryEncoding(2L, false, null));
     provider.put(dictionary);
@@ -398,18 +388,17 @@ public class BaseFileTest {
     Assert.assertEquals(2L, encoding.getId());
     Assert.assertEquals(new ArrowType.Int(32, true), encoding.getIndexType());
 
-    FieldVector.Accessor accessor = vector.getAccessor();
-    Assert.assertEquals(3, accessor.getValueCount());
-    Assert.assertEquals(Arrays.asList(0, 1), accessor.getObject(0));
-    Assert.assertEquals(Arrays.asList(0), accessor.getObject(1));
-    Assert.assertEquals(Arrays.asList(1), accessor.getObject(2));
+    Assert.assertEquals(3, vector.getValueCount());
+    Assert.assertEquals(Arrays.asList(0, 1), vector.getObject(0));
+    Assert.assertEquals(Arrays.asList(0), vector.getObject(1));
+    Assert.assertEquals(Arrays.asList(1), vector.getObject(2));
 
     Dictionary dictionary = provider.lookup(2L);
     Assert.assertNotNull(dictionary);
-    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary.getVector()).getAccessor();
-    Assert.assertEquals(2, dictionaryAccessor.getValueCount());
-    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
+    NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary.getVector());
+    Assert.assertEquals(2, dictionaryVector.getValueCount());
+    Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
+    Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
   }
 
   protected VectorSchemaRoot writeDecimalData(BufferAllocator bufferAllocator) {
@@ -423,14 +412,14 @@ public class BaseFileTest {
     decimalVector3.allocateNew(count);
 
     for (int i = 0; i < count; i++) {
-      decimalVector1.getMutator().setSafe(i, new BigDecimal(BigInteger.valueOf(i), 3));
-      decimalVector2.getMutator().setSafe(i, new BigDecimal(BigInteger.valueOf(i * (1 << 10)), 2));
-      decimalVector3.getMutator().setSafe(i, new BigDecimal(BigInteger.valueOf(i * 1111111111111111L), 8));
+      decimalVector1.setSafe(i, new BigDecimal(BigInteger.valueOf(i), 3));
+      decimalVector2.setSafe(i, new BigDecimal(BigInteger.valueOf(i * (1 << 10)), 2));
+      decimalVector3.setSafe(i, new BigDecimal(BigInteger.valueOf(i * 1111111111111111L), 8));
     }
 
-    decimalVector1.getMutator().setValueCount(count);
-    decimalVector2.getMutator().setValueCount(count);
-    decimalVector3.getMutator().setValueCount(count);
+    decimalVector1.setValueCount(count);
+    decimalVector2.setValueCount(count);
+    decimalVector3.setValueCount(count);
 
     List<Field> fields = ImmutableList.of(decimalVector1.getField(), decimalVector2.getField(), decimalVector3.getField());
     List<FieldVector> vectors = ImmutableList.<FieldVector>of(decimalVector1, decimalVector2, decimalVector3);
@@ -446,19 +435,19 @@ public class BaseFileTest {
 
     for (int i = 0; i < count; i++) {
       // Verify decimal 1 vector
-      BigDecimal readValue = decimalVector1.getAccessor().getObject(i);
+      BigDecimal readValue = decimalVector1.getObject(i);
       ArrowType.Decimal type = (ArrowType.Decimal) decimalVector1.getField().getType();
       BigDecimal genValue = new BigDecimal(BigInteger.valueOf(i), type.getScale());
       Assert.assertEquals(genValue, readValue);
 
       // Verify decimal 2 vector
-      readValue = decimalVector2.getAccessor().getObject(i);
+      readValue = decimalVector2.getObject(i);
       type = (ArrowType.Decimal) decimalVector2.getField().getType();
       genValue = new BigDecimal(BigInteger.valueOf(i * (1 << 10)), type.getScale());
       Assert.assertEquals(genValue, readValue);
 
       // Verify decimal 3 vector
-      readValue = decimalVector3.getAccessor().getObject(i);
+      readValue = decimalVector3.getObject(i);
       type = (ArrowType.Decimal) decimalVector3.getField().getType();
       genValue = new BigDecimal(BigInteger.valueOf(i * 1111111111111111L), type.getScale());
       Assert.assertEquals(genValue, readValue);
@@ -571,7 +560,7 @@ public class BaseFileTest {
     int numVarBinaryValues = 0;
     for (int i = 0; i < count; i++) {
       expectedArray[i] = (byte) i;
-      Object obj = listVector.getAccessor().getObject(i);
+      Object obj = listVector.getObject(i);
       List<?> objList = (List) obj;
       if (i % 3 == 0) {
         Assert.assertTrue(objList.isEmpty());
@@ -586,10 +575,10 @@ public class BaseFileTest {
     }
 
     // ListVector lastSet should be the index of last value + 1
-    Assert.assertEquals(listVector.getMutator().getLastSet(), count);
+    Assert.assertEquals(listVector.getLastSet(), count);
 
     // NullableVarBinaryVector lastSet should be the index of last value
     NullableVarBinaryVector binaryVector = (NullableVarBinaryVector) listVector.getChildrenFromFields().get(0);
-    Assert.assertEquals(binaryVector.getMutator().getLastSet(), numVarBinaryValues - 1);
+    Assert.assertEquals(binaryVector.getLastSet(), numVarBinaryValues - 1);
   }
 }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
index feae08e..8559969 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
@@ -317,11 +317,11 @@ public class TestArrowFile extends BaseFileTest {
 
     try (VectorSchemaRoot root = VectorSchemaRoot.create(MessageSerializerTest.testSchema(), allocator)) {
       root.getFieldVectors().get(0).allocateNew();
-      NullableTinyIntVector.Mutator mutator = (NullableTinyIntVector.Mutator) root.getFieldVectors().get(0).getMutator();
+      NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
       for (int i = 0; i < 16; i++) {
-        mutator.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
+        vector.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
       }
-      mutator.setValueCount(16);
+      vector.setValueCount(16);
       root.setRowCount(16);
 
       // write file
@@ -368,9 +368,9 @@ public class TestArrowFile extends BaseFileTest {
     NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
     for (int i = 0; i < 16; i++) {
       if (i < 8) {
-        Assert.assertEquals((byte) (i + 1), vector.getAccessor().get(i));
+        Assert.assertEquals((byte) (i + 1), vector.get(i));
       } else {
-        Assert.assertTrue(vector.getAccessor().isNull(i));
+        Assert.assertTrue(vector.isNull(i));
       }
     }
   }
@@ -397,7 +397,7 @@ public class TestArrowFile extends BaseFileTest {
     try (BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
          NullableMapVector vector = (NullableMapVector) field.createVector(originalVectorAllocator)) {
       vector.allocateNewSafe();
-      vector.getMutator().setValueCount(0);
+      vector.setValueCount(0);
 
       List<FieldVector> vectors = ImmutableList.<FieldVector>of(vector);
       VectorSchemaRoot root = new VectorSchemaRoot(originalSchema, vectors, 0);
@@ -578,13 +578,13 @@ public class TestArrowFile extends BaseFileTest {
       parent.allocateNew();
 
       for (int i = 0; i < 10; i++) {
-        tuples.getMutator().setNotNull(i);
-        floats.getMutator().set(i * 2, i + 0.1f);
-        floats.getMutator().set(i * 2 + 1, i + 10.1f);
+        tuples.setNotNull(i);
+        floats.set(i * 2, i + 0.1f);
+        floats.set(i * 2 + 1, i + 10.1f);
         ints.set(i, i);
       }
 
-      parent.getMutator().setValueCount(10);
+      parent.setValueCount(10);
       write(parent, file, stream);
     }
 
@@ -600,8 +600,8 @@ public class TestArrowFile extends BaseFileTest {
         arrowReader.loadRecordBatch(rbBlock);
         Assert.assertEquals(count, root.getRowCount());
         for (int i = 0; i < 10; i++) {
-          Assert.assertEquals(Lists.newArrayList(i + 0.1f, i + 10.1f), root.getVector("float-pairs").getAccessor().getObject(i));
-          Assert.assertEquals(i, root.getVector("ints").getAccessor().getObject(i));
+          Assert.assertEquals(Lists.newArrayList(i + 0.1f, i + 10.1f), root.getVector("float-pairs").getObject(i));
+          Assert.assertEquals(i, root.getVector("ints").getObject(i));
         }
       }
     }
@@ -616,8 +616,8 @@ public class TestArrowFile extends BaseFileTest {
       arrowReader.loadNextBatch();
       Assert.assertEquals(count, root.getRowCount());
       for (int i = 0; i < 10; i++) {
-        Assert.assertEquals(Lists.newArrayList(i + 0.1f, i + 10.1f), root.getVector("float-pairs").getAccessor().getObject(i));
-        Assert.assertEquals(i, root.getVector("ints").getAccessor().getObject(i));
+        Assert.assertEquals(Lists.newArrayList(i + 0.1f, i + 10.1f), root.getVector("float-pairs").getObject(i));
+        Assert.assertEquals(i, root.getVector("ints").getObject(i));
       }
     }
   }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStream.java b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStream.java
index e2efabe..c7e3419 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStream.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStream.java
@@ -70,11 +70,11 @@ public class TestArrowStream extends BaseFileTest {
       int numBatches = 1;
 
       root.getFieldVectors().get(0).allocateNew();
-      NullableTinyIntVector.Mutator mutator = (NullableTinyIntVector.Mutator) root.getFieldVectors().get(0).getMutator();
+      NullableTinyIntVector vector = (NullableTinyIntVector)root.getFieldVectors().get(0);
       for (int i = 0; i < 16; i++) {
-        mutator.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
+        vector.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
       }
-      mutator.setValueCount(16);
+      vector.setValueCount(16);
       root.setRowCount(16);
 
       ByteArrayOutputStream out = new ByteArrayOutputStream();
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStreamPipe.java b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStreamPipe.java
index 4071694..f393733 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStreamPipe.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowStreamPipe.java
@@ -62,13 +62,13 @@ public class TestArrowStreamPipe {
         writer.start();
         for (int j = 0; j < numBatches; j++) {
           root.getFieldVectors().get(0).allocateNew();
-          NullableTinyIntVector.Mutator mutator = (NullableTinyIntVector.Mutator) root.getFieldVectors().get(0).getMutator();
+          NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
           // Send a changing batch id first
-          mutator.set(0, j);
+          vector.set(0, j);
           for (int i = 1; i < 16; i++) {
-            mutator.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
+            vector.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
           }
-          mutator.setValueCount(16);
+          vector.setValueCount(16);
           root.setRowCount(16);
 
           writer.writeBatch();
@@ -117,12 +117,12 @@ public class TestArrowStreamPipe {
             VectorSchemaRoot root = getVectorSchemaRoot();
             Assert.assertEquals(16, root.getRowCount());
             NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
-            Assert.assertEquals((byte) (batchesRead - 1), vector.getAccessor().get(0));
+            Assert.assertEquals((byte) (batchesRead - 1), vector.get(0));
             for (int i = 1; i < 16; i++) {
               if (i < 8) {
-                Assert.assertEquals((byte) (i + 1), vector.getAccessor().get(i));
+                Assert.assertEquals((byte) (i + 1), vector.get(i));
               } else {
-                Assert.assertTrue(vector.getAccessor().isNull(i));
+                Assert.assertTrue(vector.isNull(i));
               }
             }
           }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java b/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
index ee90d34..5c4c48c 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
@@ -280,7 +280,7 @@ public class TestJSONFile extends BaseFileTest {
       // initialize vectors
       try (VectorSchemaRoot root = reader.read();) {
         FieldVector vector = root.getVector("struct_nullable");
-        Assert.assertEquals(7, vector.getAccessor().getValueCount());
+        Assert.assertEquals(7, vector.getValueCount());
       }
     }
   }

-- 
To stop receiving notification emails like this one, please contact
"commits@arrow.apache.org" <co...@arrow.apache.org>.