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

[arrow] 04/04: ARROW-1476: [JAVA] Implement Final ValueVector Updates

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 837150e245823c6f0cd9e16dba89b6d1a0396aa7
Author: siddharth <si...@dremio.com>
AuthorDate: Mon Nov 13 17:20:56 2017 -0800

    ARROW-1476: [JAVA] Implement Final ValueVector Updates
    
    indentation and line length
    
    rebase on master, fix tests
    
    address review comments
    
    integration test fix, some checkstyle fixes
    
    review comments
    
    indentation and javadocs
---
 .../main/codegen/templates/LegacyUnionVector.java  |  356 ----
 .../codegen/templates/NullableValueVectors.java    |  595 ------
 .../src/main/codegen/templates/UnionReader.java    |    2 +-
 .../src/main/codegen/templates/UnionVector.java    |   30 +-
 .../apache/arrow/vector/BaseDataValueVector.java   |    6 +-
 .../arrow/vector/BaseNullableFixedWidthVector.java | 1582 +++++++--------
 .../vector/BaseNullableVariableWidthVector.java    | 2100 +++++++++++---------
 .../org/apache/arrow/vector/BaseValueVector.java   |    4 +-
 .../java/org/apache/arrow/vector/BitVector.java    |    4 +-
 .../org/apache/arrow/vector/BitVectorHelper.java   |  298 +--
 .../apache/arrow/vector/GenerateSampleData.java    |  336 ++++
 .../apache/arrow/vector/NullableBigIntVector.java  |  652 +++---
 .../org/apache/arrow/vector/NullableBitVector.java |  888 +++++----
 .../apache/arrow/vector/NullableDateDayVector.java |  654 +++---
 .../arrow/vector/NullableDateMilliVector.java      |  661 +++---
 .../apache/arrow/vector/NullableDecimalVector.java |  758 +++----
 .../apache/arrow/vector/NullableFloat4Vector.java  |  654 +++---
 .../apache/arrow/vector/NullableFloat8Vector.java  |  654 +++---
 .../org/apache/arrow/vector/NullableIntVector.java |  664 ++++---
 .../arrow/vector/NullableIntervalDayVector.java    |  772 +++----
 .../arrow/vector/NullableIntervalYearVector.java   |  692 +++----
 .../arrow/vector/NullableSmallIntVector.java       |  708 +++----
 .../arrow/vector/NullableTimeMicroVector.java      |  652 +++---
 .../arrow/vector/NullableTimeMilliVector.java      |  656 +++---
 .../arrow/vector/NullableTimeNanoVector.java       |  652 +++---
 .../apache/arrow/vector/NullableTimeSecVector.java |  654 +++---
 .../vector/NullableTimeStampMicroTZVector.java     |  362 ++--
 .../arrow/vector/NullableTimeStampMicroVector.java |  363 ++--
 .../vector/NullableTimeStampMilliTZVector.java     |  360 ++--
 .../arrow/vector/NullableTimeStampMilliVector.java |  361 ++--
 .../vector/NullableTimeStampNanoTZVector.java      |  362 ++--
 .../arrow/vector/NullableTimeStampNanoVector.java  |  363 ++--
 .../arrow/vector/NullableTimeStampSecTZVector.java |  360 ++--
 .../arrow/vector/NullableTimeStampSecVector.java   |  363 ++--
 .../arrow/vector/NullableTimeStampVector.java      |  368 ++--
 .../apache/arrow/vector/NullableTinyIntVector.java |  708 +++----
 .../apache/arrow/vector/NullableUInt1Vector.java   |  554 +++---
 .../apache/arrow/vector/NullableUInt2Vector.java   |  554 +++---
 .../apache/arrow/vector/NullableUInt4Vector.java   |  498 ++---
 .../apache/arrow/vector/NullableUInt8Vector.java   |  498 ++---
 .../arrow/vector/NullableVarBinaryVector.java      |  768 +++----
 .../apache/arrow/vector/NullableVarCharVector.java |  772 +++----
 .../java/org/apache/arrow/vector/ValueVector.java  |    4 +-
 .../org/apache/arrow/vector/VectorUnloader.java    |    3 -
 .../vector/complex/BaseRepeatedValueVector.java    |   36 +-
 .../arrow/vector/complex/FixedSizeListVector.java  |   47 +-
 .../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    |  219 +-
 .../arrow/vector/complex/NullableMapVector.java    |  103 +-
 .../arrow/vector/file/json/JsonFileReader.java     |   53 +-
 .../arrow/vector/file/json/JsonFileWriter.java     |   16 +-
 .../org/apache/arrow/vector/TestUnionVector.java   |   22 +-
 55 files changed, 11713 insertions(+), 13143 deletions(-)

diff --git a/java/vector/src/main/codegen/templates/LegacyUnionVector.java b/java/vector/src/main/codegen/templates/LegacyUnionVector.java
deleted file mode 100644
index b9cb6df..0000000
--- a/java/vector/src/main/codegen/templates/LegacyUnionVector.java
+++ /dev/null
@@ -1,356 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-<@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/NullableValueVectors.java b/java/vector/src/main/codegen/templates/NullableValueVectors.java
deleted file mode 100644
index 8e5b10b..0000000
--- a/java/vector/src/main/codegen/templates/NullableValueVectors.java
+++ /dev/null
@@ -1,595 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-<@pp.dropOutputFile />
-<#list vv.types as type>
-<#list type.minor as minor>
-
-<#assign className = "LegacyNullable${minor.class}Vector" />
-<#assign valuesName = "Nullable${minor.class}Vector" />
-<#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
-
-<@pp.changeOutputFile name="/org/apache/arrow/vector/${className}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.arrow.vector;
-
-import org.apache.arrow.vector.schema.ArrowFieldNode;
-import java.util.Collections;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.arrow.flatbuf.Precision;
-
-/**
- * ${className} implements a vector of values which could be null.  Elements in the vector
- * are first checked against a fixed length vector of boolean values.  Then the element is retrieved
- * from the base class (if not null).
- *
- * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
- */
-@SuppressWarnings("unused")
-@Deprecated
-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[]{};
-
-  private final String bitsField = "$bits$";
-  private final String valuesField = "$values$";
-
-  final BitVector bits = new BitVector(bitsField, allocator);
-  final ${valuesName} values;
-
-  private final Mutator mutator;
-  private final Accessor accessor;
-
-  <#if minor.typeParams??>
-    <#assign typeParams = minor.typeParams?reverse>
-    <#list typeParams as typeParam>
-  private final ${typeParam.type} ${typeParam.name};
-    </#list>
-
-  /**
-   * Assumes the type is nullable and not dictionary encoded
-   * @param name name of the field
-   * @param allocator allocator to use to resize the vector<#list typeParams as typeParam>
-   * @param ${typeParam.name} type parameter ${typeParam.name}</#list>
-   */
-  public ${className}(String name, BufferAllocator allocator<#list typeParams as typeParam>, ${typeParam.type} ${typeParam.name}</#list>) {
-    <#if minor.arrowTypeConstructorParams??>
-       <#assign constructorParams = minor.arrowTypeConstructorParams />
-    <#else>
-       <#assign constructorParams = [] />
-       <#list typeParams as typeParam>
-         <#assign constructorParams = constructorParams + [ typeParam.name ] />
-      </#list>
-    </#if>
-    this(name, FieldType.nullable(new ${minor.arrowType}(${constructorParams?join(", ")})), allocator);
-  }
-  <#else>
-  public ${className}(String name, BufferAllocator allocator) {
-    this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.${minor.class?upper_case}.getType()), allocator);
-  }
-  </#if>
-
-  public ${className}(String name, FieldType fieldType, BufferAllocator allocator) {
-    super(name, allocator);
-    <#if minor.typeParams??>
-    <#assign typeParams = minor.typeParams?reverse>
-    ${minor.arrowType} arrowType = (${minor.arrowType})fieldType.getType();
-    <#list typeParams as typeParam>
-    this.${typeParam.name} = arrowType.get${typeParam.name?cap_first}();
-    </#list>
-    this.values = new ${valuesName}(valuesField, allocator<#list typeParams as typeParam>, ${typeParam.name}</#list>);
-    <#else>
-    this.values = new ${valuesName}(valuesField, allocator);
-    </#if>
-    this.mutator = new Mutator();
-    this.accessor = new Accessor();
-  }
-
-  @Override
-  public List<BufferBacked> getFieldInnerVectors() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getFieldInnerVectors();
-  }
-
-  @Override
-  public void initializeChildrenFromFields(List<Field> children) {
-    if (!children.isEmpty()) {
-      throw new IllegalArgumentException("primitive type vector ${className} can not have children: " + children);
-    }
-  }
-
-  @Override
-  public List<FieldVector> getChildrenFromFields() {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    /* DELEGATE TO NEW VECTOR */
-    values.loadFieldBuffers(fieldNode, ownBuffers);
-  }
-
-  public List<ArrowBuf> getFieldBuffers() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getFieldBuffers();
-  }
-
-  @Override
-  public Field getField() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getField();
-  }
-
-  @Override
-  public MinorType getMinorType() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getMinorType();
-  }
-
-  @Override
-  public FieldReader getReader(){
-    /* DELEGATE TO NEW VECTOR */
-    return values.getReader();
-  }
-
-  @Override
-  public int getValueCapacity(){
-    /* DELEGATE TO NEW VECTOR */
-    return values.getValueCapacity();
-  }
-
-  @Override
-  public ArrowBuf[] getBuffers(boolean clear) {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getBuffers(clear);
-  }
-
-  @Override
-  public void close() {
-    /* DELEGATE TO NEW VECTOR */
-    values.close();
-  }
-
-  @Override
-  public void clear() {
-    /* DELEGATE TO NEW VECTOR */
-    values.clear();
-  }
-
-  @Override
-  public int getBufferSize(){
-    /* DELEGATE TO NEW VECTOR */
-    return values.getBufferSize();
-  }
-
-  @Override
-  public int getBufferSizeFor(final int valueCount) {
-    if (valueCount == 0) {
-      return 0;
-    }
-    /* DELEGATE TO NEW VECTOR */
-    return values.getBufferSizeFor(valueCount);
-  }
-
-  public ArrowBuf getBuffer() {
-    return values.getDataBuffer();
-  }
-
-  public ${valuesName} getValuesVector() {
-    return values;
-  }
-
-  @Override
-  public void setInitialCapacity(int numRecords) {
-    /* DELEGATE TO NEW VECTOR */
-    values.setInitialCapacity(numRecords);
-  }
-
-  @Override
-  public void allocateNew() {
-    /* DELEGATE TO NEW VECTOR */
-    values.allocateNew();
-  }
-
-  @Override
-  public boolean allocateNewSafe() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.allocateNewSafe();
-  }
-
-  @Override
-  public void reAlloc() {
-    /* DELEGATE TO NEW VECTOR */
-    values.reAlloc();
-  }
-
-  public void reset() {
-    /* DELEGATE TO NEW VECTOR */
-    values.reset();
-  }
-
-  <#if type.major == "VarLen">
-  @Override
-  public void allocateNew(int totalBytes, int valueCount) {
-    /* DELEGATE TO NEW VECTOR */
-    values.allocateNew(totalBytes, valueCount);
-  }
-
-  @Override
-  public int getByteCapacity(){
-    return values.getByteCapacity();
-  }
-
-  @Override
-  public int getCurrentSizeInBytes(){
-    return values.getCurrentSizeInBytes();
-  }
-
-  <#else>
-  @Override
-  public void allocateNew(int valueCount) {
-    /* DELEGATE TO NEW VECTOR */
-    values.allocateNew(valueCount);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void zeroVector() {
-    /* DELEGATE TO NEW VECTOR */
-    values.zeroVector();
-  }
-  </#if>
-
-
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getTransferPair(ref, allocator, callBack);
-  }
-
-
-
-  @Override
-  public TransferPair getTransferPair(BufferAllocator allocator){
-    /* DELEGATE TO NEW VECTOR */
-    return values.getTransferPair(allocator);
-  }
-
-
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-    /* DELEGATE TO NEW VECTOR */
-    return values.getTransferPair(ref, allocator);
-  }
-
-
-
-  @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    /* DELEGATE TO NEW VECTOR */
-    return values.makeTransferPair(to);
-  }
-
-
-
-  public void transferTo(${valuesName} target) {
-    /* DELEGATE TO NEW VECTOR */
-    <#if type.major == "VarLen">
-        values.transferTo((BaseNullableVariableWidthVector) target);
-    <#else>
-        values.transferTo((BaseNullableFixedWidthVector) target);
-    </#if>
-  }
-
-  public void splitAndTransferTo(int startIndex, int length, ${valuesName} target) {
-    /* DELEGATE TO NEW VECTOR */
-    <#if type.major == "VarLen">
-        values.splitAndTransferTo(startIndex, length, (BaseNullableVariableWidthVector) target);
-    <#else>
-        values.splitAndTransferTo(startIndex, length, (BaseNullableFixedWidthVector) target);
-    </#if>
-  }
-
-
-
-  @Override
-  public Accessor getAccessor(){
-    return accessor;
-  }
-
-  @Override
-  public Mutator getMutator(){
-    return mutator;
-  }
-
-
-  public void copyFrom(int fromIndex, int thisIndex, ${valuesName} from) {
-    /* DELEGATE TO NEW VECTOR */
-    values.copyFrom(fromIndex, thisIndex, from);
-  }
-
-  public void copyFromSafe(int fromIndex, int thisIndex, ${valuesName} from) {
-    /* DELEGATE TO NEW VECTOR */
-    values.copyFromSafe(fromIndex, thisIndex, from);
-  }
-
-  @Override
-  public long getValidityBufferAddress() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getValidityBufferAddress();
-  }
-
-  @Override
-  public long getDataBufferAddress() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getDataBufferAddress();
-  }
-
-  @Override
-  public long getOffsetBufferAddress() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getOffsetBufferAddress();
-  }
-
-  @Override
-  public ArrowBuf getValidityBuffer() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getValidityBuffer();
-  }
-
-  @Override
-  public ArrowBuf getDataBuffer() {
-    return (values.getDataBuffer());
-  }
-
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    /* DELEGATE TO NEW VECTOR */
-    return values.getOffsetBuffer();
-  }
-
-  public final class Accessor extends BaseDataValueVector.BaseAccessor <#if type.major = "VarLen">implements VariableWidthVector.VariableWidthAccessor</#if> {
-
-    /**
-     * Get the element at the specified position.
-     *
-     * @param  index   position of the value
-     * @return value of the element, if not null
-     */
-    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) {
-      /* DELEGATE TO NEW VECTOR */
-      return values.isNull(index);
-    }
-
-    public int isSet(int index){
-      /* DELEGATE TO NEW VECTOR */
-      return values.isSet(index);
-    }
-
-    <#if type.major == "VarLen">
-    public long getStartEnd(int index){
-        /* DELEGATE TO NEW VECTOR */
-        return values.getStartEnd(index);
-    }
-
-    @Override
-    public int getValueLength(int index) {
-        /* DELEGATE TO NEW VECTOR */
-        return values.getValueLength(index);
-    }
-    </#if>
-
-    public void get(int index, Nullable${minor.class}Holder holder){
-        /* DELEGATE TO NEW VECTOR */
-        values.get(index, holder);
-    }
-
-    @Override
-    public ${friendlyType} getObject(int index) {
-      /* DELEGATE TO NEW VECTOR */
-      return values.getObject(index);
-    }
-
-    <#if minor.class == "IntervalYear" || minor.class == "IntervalDay">
-    public StringBuilder getAsStringBuilder(int index) {
-       /* DELEGATE TO NEW VECTOR */
-       return values.getAsStringBuilder(index);
-    }
-    </#if>
-
-    @Override
-    public int getValueCount(){
-      /* DELEGATE TO NEW VECTOR */
-      return values.getValueCount();
-    }
-
-    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() { }
-
-    public ${valuesName} getVectorWithValues() {
-      return values;
-    }
-
-
-    @Override
-    public void setIndexDefined(int index) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setIndexDefined(index);
-    }
-
-
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param value   array of bytes (or int if smaller than 4 bytes) to write
-     */
-    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);
-    }
-
-
-
-    <#if type.major == "VarLen">
-    public void fillEmpties(int index) {
-      /* DELEGATE TO NEW VECTOR */
-      values.fillEmpties(index);
-    }
-
-    @Override
-    public void setValueLengthSafe(int index, int length) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setValueLengthSafe(index, length);
-    }
-    </#if>
-
-
-    public void setSafe(int index, byte[] value, int start, int length) {
-       /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, value, start, length);
-    }
-
-
-    public void setSafe(int index, ByteBuffer value, int start, int length) {
-       /* DELEGATE TO NEW VECTOR */
-       values.setSafe(index, value, start, length);
-    }
-
-
-    public void setNull(int index) {
-       /* DELEGATE TO NEW VECTOR */
-       values.setNull(index);
-    }
-
-
-    public void set(int index, Nullable${minor.class}Holder holder) {
-      /* DELEGATE TO NEW VECTOR */
-      values.set(index, holder);
-    }
-
-
-    public void set(int index, ${minor.class}Holder holder) {
-        /* DELEGATE TO NEW VECTOR */
-        values.set(index, holder);
-    }
-
-
-    public boolean isSafe(int outIndex) {
-       /* DELEGATE TO NEW VECTOR */
-       return values.isSafe(outIndex);
-    }
-
-
-    <#assign fields = minor.fields!type.fields />
-    public void set(int index, int isSet<#list fields as field>, ${field.type} ${field.name}Field</#list> ){
-      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> ) {
-      values.setSafe(index, isSet<#list fields as field><#if field.include!true >, ${field.name}Field</#if></#list>);
-    }
-
-
-    public void setSafe(int index, Nullable${minor.class}Holder value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, value);
-    }
-
-    public void setSafe(int index, ${minor.class}Holder value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, value);
-    }
-
-
-    <#if !(type.major == "VarLen" || minor.class == "IntervalDay")>
-    public void setSafe(int index, ${minor.javaType!type.javaType} value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, value);
-    }
-    </#if>
-
-
-
-    <#if minor.class == "Decimal">
-    public void set(int index, ${friendlyType} value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.set(index, value);
-    }
-
-    public void setSafe(int index, ${friendlyType} value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setSafe(index, value);
-    }
-    </#if>
-
-
-    @Override
-    public void setValueCount(int valueCount) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setValueCount(valueCount);
-    }
-
-
-    /* THIS METHOD IS PROBABLY NOT NEEDED FOR NEW VECTORS */
-    @Override
-    public void generateTestData(int valueCount) { }
-
-
-    /* MUTATOR RESET IS NOT NEEDED FOR NEW VECTORS */
-    @Override
-    public void reset() { }
-
-
-    <#if type.major == "VarLen">
-    public void setLastSet(int value) {
-      /* DELEGATE TO NEW VECTOR */
-      values.setLastSet(value);
-    }
-
-
-    public int getLastSet() {
-      /* DELEGATE TO NEW VECTOR */
-      return values.getLastSet();
-    }
-    </#if>
-  }
-}
-</#list>
-</#list>
diff --git a/java/vector/src/main/codegen/templates/UnionReader.java b/java/vector/src/main/codegen/templates/UnionReader.java
index fd3a766..98bb7c1 100644
--- a/java/vector/src/main/codegen/templates/UnionReader.java
+++ b/java/vector/src/main/codegen/templates/UnionReader.java
@@ -60,7 +60,7 @@ public class UnionReader extends AbstractFieldReader {
   }
 
   public boolean isSet(){
-    return !data.getAccessor().isNull(idx());
+    return !data.isNull(idx());
   }
 
   public void read(UnionHolder holder) {
diff --git a/java/vector/src/main/codegen/templates/UnionVector.java b/java/vector/src/main/codegen/templates/UnionVector.java
index 3da383f..e44edbd 100644
--- a/java/vector/src/main/codegen/templates/UnionVector.java
+++ b/java/vector/src/main/codegen/templates/UnionVector.java
@@ -63,7 +63,7 @@ public class UnionVector implements FieldVector {
   int valueCount;
 
   MapVector internalMap;
-  ArrowBuf typeBuffer;
+  protected ArrowBuf typeBuffer;
 
   private NullableMapVector mapVector;
   private ListVector listVector;
@@ -121,15 +121,17 @@ public class UnionVector implements FieldVector {
   @Override
   public List<ArrowBuf> getFieldBuffers() {
     List<ArrowBuf> result = new ArrayList<>(1);
-
-    typeBuffer.readerIndex(0);
-    typeBuffer.writerIndex(valueCount * TYPE_WIDTH);
-
+    setReaderAndWriterIndex();
     result.add(typeBuffer);
 
     return result;
   }
 
+  private void setReaderAndWriterIndex() {
+    typeBuffer.readerIndex(0);
+    typeBuffer.writerIndex(valueCount * TYPE_WIDTH);
+  }
+
   @Override
   @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
@@ -232,6 +234,8 @@ public class UnionVector implements FieldVector {
 
   @Override
   public void allocateNew() throws OutOfMemoryException {
+    /* new allocation -- clear the current buffers */
+    clear();
     internalMap.allocateNew();
     try {
       allocateTypeBuffer();
@@ -243,6 +247,8 @@ public class UnionVector implements FieldVector {
 
   @Override
   public boolean allocateNewSafe() {
+    /* new allocation -- clear the current buffers */
+    clear();
     boolean safe = internalMap.allocateNewSafe();
     if (!safe) { return false; }
     try {
@@ -386,6 +392,7 @@ public class UnionVector implements FieldVector {
       to.typeBuffer = typeBuffer.transferOwnership(to.allocator).buffer;
       internalMapVectorTransferPair.transfer();
       to.valueCount = valueCount;
+      clear();
     }
 
     @Override
@@ -460,8 +467,17 @@ public class UnionVector implements FieldVector {
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
     ImmutableList.Builder<ArrowBuf> builder = ImmutableList.builder();
-    builder.add(typeBuffer);
-    builder.add(internalMap.getBuffers(clear));
+    setReaderAndWriterIndex();
+    if (getBufferSize() != 0) {
+      builder.add(typeBuffer);
+      builder.add(internalMap.getBuffers(clear));
+    }
+    if (clear) {
+      valueCount = 0;
+      typeBuffer.retain();
+      typeBuffer.release();
+      typeBuffer = allocator.getEmpty();
+    }
     List<ArrowBuf> list = builder.build();
     return list.toArray(new ArrowBuf[list.size()]);
   }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
index 01340f6..38524ff 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -85,7 +85,7 @@ public abstract class BaseDataValueVector extends BaseValueVector implements Buf
     if (getBufferSize() == 0) {
       out = new ArrowBuf[0];
     } else {
-      out = new ArrowBuf[] {data};
+      out = new ArrowBuf[]{data};
       data.readerIndex(0);
       if (clear) {
         data.retain(1);
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 83dc409..209758e 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -41,806 +41,810 @@ import org.apache.arrow.vector.util.TransferPair;
  * 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;
-
-   protected int valueAllocationSizeInBytes;
-   protected int validityAllocationSizeInBytes;
-
-   protected final Field field;
-   private int allocationMonitor;
-   protected ArrowBuf validityBuffer;
-   protected ArrowBuf valueBuffer;
-   protected int valueCount;
-
-   public BaseNullableFixedWidthVector(final String name, final BufferAllocator allocator,
-                                       FieldType fieldType, final byte typeWidth) {
-      super(name, allocator);
-      this.typeWidth = typeWidth;
-      field = new Field(name, fieldType, null);
-      valueCount = 0;
+        implements FixedWidthVector, FieldVector, NullableVectorDefinitionSetter {
+  private final byte typeWidth;
+
+  protected int valueAllocationSizeInBytes;
+  protected int validityAllocationSizeInBytes;
+
+  protected final Field field;
+  private int allocationMonitor;
+  protected ArrowBuf validityBuffer;
+  protected ArrowBuf valueBuffer;
+  protected int valueCount;
+
+  public BaseNullableFixedWidthVector(final String name, final BufferAllocator allocator,
+                                      FieldType fieldType, final byte typeWidth) {
+    super(name, allocator);
+    this.typeWidth = typeWidth;
+    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:
+   * see if getNullCount() can be made faster -- O(1)
+   */
+
+  /* TODO:
+   * Once the entire hierarchy has been refactored, move common functions
+   * like getNullCount(), splitAndTransferValidityBuffer to top level
+   * base class BaseValueVector.
+   *
+   * Along with this, some class members (validityBuffer) can also be
+   * abstracted out to top level base class.
+   *
+   * Right now BaseValueVector is the top level base class for other
+   * 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.
+   */
+
+
+  @Override
+  @Deprecated
+  public Mutator getMutator() {
+    throw new UnsupportedOperationException("Mutator is not supported for writing to vector");
+  }
+
+  @Override
+  @Deprecated
+  public Accessor getAccessor() {
+    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;
+    if (size > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
+    }
+    valueAllocationSizeInBytes = (int) size;
+    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());
+  }
+
+  private int getValueBufferValueCapacity() {
+    return (int) ((valueBuffer.capacity() * 1.0) / typeWidth);
+  }
+
+  private int getValidityBufferValueCapacity() {
+    return (int) (validityBuffer.capacity() * 8L);
+  }
+
+  /**
+   * 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);
+  }
+
+  /* used to step down the memory allocation */
+  protected void incrementAllocationMonitor() {
+    if (allocationMonitor < 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:
-    *
-    * see if getNullCount() can be made faster -- O(1)
-    */
-
-   /* TODO:
-    * Once the entire hierarchy has been refactored, move common functions
-    * like getNullCount(), splitAndTransferValidityBuffer to top level
-    * base class BaseValueVector.
-    *
-    * Along with this, some class members (validityBuffer) can also be
-    * abstracted out to top level base class.
-    *
-    * Right now BaseValueVector is the top level base class for other
-    * 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.
-    */
-
-
-   @Override
-   @Deprecated
-   public Mutator getMutator() {
-      throw new UnsupportedOperationException("Mutator is not supported for writing to vector");
-   }
-
-   @Override
-   @Deprecated
-   public Accessor getAccessor() {
-      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;
-      if (size > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
-      }
-      valueAllocationSizeInBytes = (int)size;
-      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());
-   }
-
-   private int getValueBufferValueCapacity() {
-      return (int)((valueBuffer.capacity() * 1.0)/typeWidth);
-   }
-
-   private int getValidityBufferValueCapacity() {
-      return (int)(validityBuffer.capacity() * 8L);
-   }
-
-   /**
-    * 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);
-   }
-
-   /* used to step down the memory allocation */
-   protected void incrementAllocationMonitor() {
-      if (allocationMonitor < 0) {
-         allocationMonitor = 0;
-      }
-      allocationMonitor++;
-   }
+    }
+    allocationMonitor++;
+  }
 
-   /* used to step up the memory allocation */
-   protected void decrementAllocationMonitor() {
-      if (allocationMonitor > 0) {
-         allocationMonitor = 0;
-      }
-      allocationMonitor--;
-   }
-
-   /**
-    * Same as {@link #allocateNewSafe()}.
-    */
-   @Override
-   public void allocateNew() {
-      if(!allocateNewSafe()){
-         throw new OutOfMemoryException("Failure while allocating memory.");
-      }
-   }
-
-   /**
-    * 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;
-
-      if (curAllocationSizeValue > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory exceeds limit");
-      }
+  /* used to step up the memory allocation */
+  protected void decrementAllocationMonitor() {
+    if (allocationMonitor > 0) {
+      allocationMonitor = 0;
+    }
+    allocationMonitor--;
+  }
+
+  /**
+   * Same as {@link #allocateNewSafe()}.
+   */
+  @Override
+  public void allocateNew() {
+    if (!allocateNewSafe()) {
+      throw new OutOfMemoryException("Failure while allocating memory.");
+    }
+  }
+
+  /**
+   * 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;
+
+    if (curAllocationSizeValue > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory exceeds limit");
+    }
 
       /* we are doing a new allocation -- release the current buffers */
-      clear();
-
-      try{
-         allocateBytes(curAllocationSizeValue, curAllocationSizeValidity);
-      } catch (Exception e) {
-         e.printStackTrace();
-         clear();
-         return false;
-      }
+    clear();
 
-      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 = getValidityBufferSizeFromCount(valueCount);
-      if (typeWidth == 0) {
-         /* specialized handling for NullableBitVector */
-         valueBufferSize = validityBufferSize;
-      }
-
-      if (allocationMonitor > 10) {
-         /* step down the default memory allocation since we have observed
-          * multiple times that provisioned value capacity was much larger than
-          * actually needed. see setValueCount for more details.
-          */
-         valueBufferSize = Math.max(8, valueBufferSize / 2);
-         validityBufferSize = Math.max(8, validityBufferSize / 2);
-         allocationMonitor = 0;
-      } else if (allocationMonitor < -2) {
-         valueBufferSize = valueBufferSize * 2L;
-         validityBufferSize = validityBufferSize * 2L;
-         allocationMonitor = 0;
-      }
-
-      if (valueBufferSize > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
-      }
-
-      /* we are doing a new allocation -- release the current buffers */
+    try {
+      allocateBytes(curAllocationSizeValue, curAllocationSizeValidity);
+    } catch (Exception e) {
+      e.printStackTrace();
       clear();
-
-      try {
-         allocateBytes(valueBufferSize, validityBufferSize);
-      } catch(Exception e) {
-         e.printStackTrace();
-         clear();
-         throw e;
-      }
-   }
-
-   /**
-    * Actual memory allocation is done by this function. All the calculations
-    * and knowledge about what size to allocate is upto the callers of this
-    * method.
-    * Callers appropriately handle errors if memory allocation fails here.
-    * Callers should also take care of determining that desired size is
-    * within the bounds of max allocation allowed and any other error
-    * conditions.
-    */
-   private void allocateBytes(final long valueBufferSize, final long validityBufferSize) {
-      /* allocate data buffer */
-      int curSize = (int)valueBufferSize;
-      valueBuffer = allocator.buffer(curSize);
-      valueBuffer.readerIndex(0);
-      valueAllocationSizeInBytes = curSize;
-
-      /* allocate validity buffer */
-      allocateValidityBuffer((int)validityBufferSize);
-      initValidityBuffer();
-   }
-
-   /**
-    * 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.
-    */
-   private void allocateValidityBuffer(final int validityBufferSize) {
-      validityBuffer = allocator.buffer(validityBufferSize);
-      validityBuffer.readerIndex(0);
-      validityAllocationSizeInBytes = validityBufferSize;
-      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) + 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) + 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];
+      return false;
+    }
+
+    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 = getValidityBufferSizeFromCount(valueCount);
+    if (typeWidth == 0) {
+      /* specialized handling for NullableBitVector */
+      valueBufferSize = validityBufferSize;
+    }
+
+    if (valueBufferSize > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
+    }
+
+    /* we are doing a new allocation -- release the current buffers */
+    clear();
+
+    try {
+      allocateBytes(valueBufferSize, validityBufferSize);
+    } catch (Exception e) {
+      e.printStackTrace();
+      clear();
+      throw e;
+    }
+  }
+
+  /**
+   * Actual memory allocation is done by this function. All the calculations
+   * and knowledge about what size to allocate is upto the callers of this
+   * method.
+   * Callers appropriately handle errors if memory allocation fails here.
+   * Callers should also take care of determining that desired size is
+   * within the bounds of max allocation allowed and any other error
+   * conditions.
+   */
+  private void allocateBytes(final long valueBufferSize, final long validityBufferSize) {
+    /* allocate data buffer */
+    int curSize = (int) valueBufferSize;
+    valueBuffer = allocator.buffer(curSize);
+    valueBuffer.readerIndex(0);
+    valueAllocationSizeInBytes = curSize;
+    /* allocate validity buffer */
+    allocateValidityBuffer((int) validityBufferSize);
+    zeroVector();
+  }
+
+  /**
+   * 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.
+   */
+  private void allocateValidityBuffer(final int validityBufferSize) {
+    validityBuffer = allocator.buffer(validityBufferSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = validityBufferSize;
+  }
+
+  /**
+   * 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) + 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) + 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;
+    setReaderAndWriterIndex();
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = new ArrowBuf[2];
       buffers[0] = validityBuffer;
       buffers[1] = valueBuffer;
-      if (clear) {
-         for (final ArrowBuf buffer:buffers) {
-            buffer.retain(1);
-         }
-         clear();
+    }
+    if (clear) {
+      for (final ArrowBuf buffer : buffers) {
+        buffer.retain(1);
       }
-      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
-                                   : 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, buffer, 0, currentBufferCapacity);
-      final int halfNewCapacity = newBuf.capacity() / 2;
-      newBuf.setZero(halfNewCapacity, halfNewCapacity);
-      buffer.release(1);
-      buffer = newBuf;
-      if (dataBuffer) {
-         valueAllocationSizeInBytes = (int)newAllocationSize;
-      }
-      else {
-         validityAllocationSizeInBytes = (int)newAllocationSize;
-      }
-
-      return buffer;
-   }
-
-   @Override
-   @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()) {
-         throw new IllegalArgumentException("primitive type vector can not have children");
-      }
-   }
-
-   /**
-    * 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) {
-         throw new IllegalArgumentException("Illegal buffer count, expected " + 2 + ", got: " + ownBuffers.size());
-      }
-
-      ArrowBuf bitBuffer = ownBuffers.get(0);
-      ArrowBuf dataBuffer = ownBuffers.get(1);
-
-      validityBuffer.release();
-      validityBuffer = bitBuffer.retain(allocator);
-      valueBuffer.release();
-      valueBuffer = dataBuffer.retain(allocator);
-
-      valueCount = fieldNode.getLength();
-
-      valueAllocationSizeInBytes = valueBuffer.capacity();
-      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);
+      clear();
+    }
+    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
+            : 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, buffer, 0, currentBufferCapacity);
+    final int halfNewCapacity = newBuf.capacity() / 2;
+    newBuf.setZero(halfNewCapacity, halfNewCapacity);
+    buffer.release(1);
+    buffer = newBuf;
+    if (dataBuffer) {
+      valueAllocationSizeInBytes = (int) newAllocationSize;
+    } else {
+      validityAllocationSizeInBytes = (int) newAllocationSize;
+    }
+
+    return buffer;
+  }
+
+  @Override
+  @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()) {
+      throw new IllegalArgumentException("primitive type vector can not have children");
+    }
+  }
+
+  /**
+   * 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) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 2 + ", got: " + ownBuffers.size());
+    }
+
+    ArrowBuf bitBuffer = ownBuffers.get(0);
+    ArrowBuf dataBuffer = ownBuffers.get(1);
+
+    validityBuffer.release();
+    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
+    valueBuffer.release();
+    valueBuffer = dataBuffer.retain(allocator);
+
+    valueCount = fieldNode.getLength();
+
+    valueAllocationSizeInBytes = valueBuffer.capacity();
+    validityAllocationSizeInBytes = validityBuffer.capacity();
+  }
+
+  /**
+   * Get the buffers belonging to this vector
+   * @return the inner buffers.
+   */
+  public List<ArrowBuf> getFieldBuffers() {
+    List<ArrowBuf> result = new ArrayList<>(2);
+    setReaderAndWriterIndex();
+    result.add(validityBuffer);
+    result.add(valueBuffer);
+
+    return result;
+  }
+
+  /**
+   * Set the reader and writer indexes for the inner buffers.
+   */
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    valueBuffer.readerIndex(0);
+    if (valueCount == 0) {
+      validityBuffer.writerIndex(0);
+      valueBuffer.writerIndex(0);
+    } else {
       validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
-      valueBuffer.readerIndex(0);
       if (typeWidth == 0) {
-         /* specialized handling for NullableBitVector */
-         valueBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+        /* specialized handling for NullableBitVector */
+        valueBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
       } else {
-         valueBuffer.writerIndex(valueCount * typeWidth);
+        valueBuffer.writerIndex(valueCount * typeWidth);
       }
-
-      result.add(validityBuffer);
-      result.add(valueBuffer);
-
-      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();
-      target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
-      target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer;
-      target.valueCount = valueCount;
-      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");
-      target.clear();
-      splitAndTransferValidityBuffer(startIndex, length, target);
-      splitAndTransferValueBuffer(startIndex, length, target);
-      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;
-      final int sliceLength = length * typeWidth;
-      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 = 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(this.validityBuffer,
-                       firstByteSource + i, offset);
-               byte b2 = BitVectorHelper.getBitsFromNextByte(this.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(this.validityBuffer,
-                       firstByteSource + byteSizeTarget - 1, offset);
-               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
-                       firstByteSource + byteSizeTarget, offset);
-
-               target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
-            }
-            else {
-               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
-                       firstByteSource + byteSizeTarget - 1, offset);
-               target.validityBuffer.setByte(byteSizeTarget - 1, b1);
-            }
-         }
-      }
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          common getters and setters                            *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * Get the number of elements that are null in the vector
-    *
-    * @return the number of null elements.
-    */
-   public int getNullCount() {
-      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.
-    *
-    * @return valueCount for the vector
-    */
-   public int getValueCount(){
-      return valueCount;
-   }
-
-   /**
-    * Set value count for the vector.
-    *
-    * @param valueCount  value count to set
-    */
-   public void setValueCount(int valueCount) {
-      this.valueCount = valueCount;
-      final int currentValueCapacity = getValueCapacity();
-      while (valueCount > getValueCapacity()) {
-         reAlloc();
-      }
-      /*
-       * We are trying to understand the pattern of memory allocation.
-       * If initially, the user did vector.allocateNew(), we would have
-       * allocated memory of default size (4096 * type width).
-       * Later on user invokes setValueCount(count).
-       *
-       * If the existing value capacity is twice as large as the
-       * valueCount, we know that we over-provisioned memory in the
-       * first place when default memory allocation was done because user
-       * really needs a much less value count in the vector.
-       *
-       * We record this by bumping up the allocationMonitor. If this pattern
-       * happens for certain number of times and allocationMonitor
-       * reaches the threshold (internal hardcoded) value, subsequent
-       * call to allocateNew() will take care of stepping down the
-       * default memory allocation size.
-       *
-       * Another case would be under-provisioning the initial memory and
-       * thus going through a lot of realloc(). Here the goal is to
-       * see if we can minimize the number of reallocations. Again the
-       * state is recorded in allocationMonitor by decrementing it
-       * (negative value). If a threshold is hit, realloc will try to
-       * allocate more memory in order to possibly avoid a future realloc.
-       * This case is also applicable to setSafe() methods which can trigger
-       * a realloc() and thus we record the state there as well.
-       */
-      if (valueCount > 0) {
-         if (currentValueCapacity >= (valueCount * 2)) {
-            incrementAllocationMonitor();
-         } else if (currentValueCapacity <= (valueCount/2)) {
-            decrementAllocationMonitor();
-         }
+    }
+  }
+
+  /**
+   * 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();
+    target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
+    target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer;
+    target.valueCount = valueCount;
+    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");
+    target.clear();
+    splitAndTransferValidityBuffer(startIndex, length, target);
+    splitAndTransferValueBuffer(startIndex, length, target);
+    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;
+    final int sliceLength = length * typeWidth;
+    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 = 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(this.validityBuffer,
+                  firstByteSource + i, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(this.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(this.validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
+                  firstByteSource + byteSizeTarget, offset);
+
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+        } else {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1);
+        }
       }
-   }
-
-   /**
-    * Check if the given index is within the current value capacity
-    * of the vector
-    *
-    * @param index  position to check
-    * @return true if index is within the current value capacity
-    */
-   public boolean isSafe(int index) {
-      return index < getValueCapacity();
-   }
-
-   /**
-    * Check if element at given index is null.
-    *
-    * @param index  position of element
-    * @return true if element at given index is null, false otherwise
-    */
-   public boolean isNull(int index) {
-      return (isSet(index) == 0);
-   }
-
-   /**
-    * Same as {@link #isNull(int)}.
-    *
-    * @param index  position of element
-    * @return 1 if element at given index is not null, 0 otherwise
-    */
-   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));
-   }
-
-   /**
-    * 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);
-   }
-
-   public void set(int index, byte[] value, int start, int length) {
-      throw new UnsupportedOperationException();
-   }
-
-   public void setSafe(int index, byte[] value, int start, int length) {
-      throw new UnsupportedOperationException();
-   }
-
-   public void set(int index, ByteBuffer value, int start, int length) {
-      throw new UnsupportedOperationException();
-   }
-
-   public void setSafe(int index, ByteBuffer value, int start, int length) {
-      throw new UnsupportedOperationException();
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                helper methods for setters                      *
-    *                                                                *
-    ******************************************************************/
-
-
-   protected void handleSafe(int index) {
-      while (index >= getValueCapacity()) {
-         decrementAllocationMonitor();
-         reAlloc();
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          common getters and setters                            *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * Get the number of elements that are null in the vector
+   *
+   * @return the number of null elements.
+   */
+  public int getNullCount() {
+    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.
+   *
+   * @return valueCount for the vector
+   */
+  public int getValueCount() {
+    return valueCount;
+  }
+
+  /**
+   * Set value count for the vector.
+   *
+   * @param valueCount  value count to set
+   */
+  public void setValueCount(int valueCount) {
+    this.valueCount = valueCount;
+    final int currentValueCapacity = getValueCapacity();
+    while (valueCount > getValueCapacity()) {
+      reAlloc();
+    }
+    /*
+     * We are trying to understand the pattern of memory allocation.
+     * If initially, the user did vector.allocateNew(), we would have
+     * allocated memory of default size (4096 * type width).
+     * Later on user invokes setValueCount(count).
+     *
+     * If the existing value capacity is twice as large as the
+     * valueCount, we know that we over-provisioned memory in the
+     * first place when default memory allocation was done because user
+     * really needs a much less value count in the vector.
+     *
+     * We record this by bumping up the allocationMonitor. If this pattern
+     * happens for certain number of times and allocationMonitor
+     * reaches the threshold (internal hardcoded) value, subsequent
+     * call to allocateNew() will take care of stepping down the
+     * default memory allocation size.
+     *
+     * Another case would be under-provisioning the initial memory and
+     * thus going through a lot of realloc(). Here the goal is to
+     * see if we can minimize the number of reallocations. Again the
+     * state is recorded in allocationMonitor by decrementing it
+     * (negative value). If a threshold is hit, realloc will try to
+     * allocate more memory in order to possibly avoid a future realloc.
+     * This case is also applicable to setSafe() methods which can trigger
+     * a realloc() and thus we record the state there as well.
+     */
+    if (valueCount > 0) {
+      if (currentValueCapacity >= (valueCount * 2)) {
+        incrementAllocationMonitor();
+      } else if (currentValueCapacity <= (valueCount / 2)) {
+        decrementAllocationMonitor();
       }
-   }
+    }
+    setReaderAndWriterIndex();
+  }
+
+  /**
+   * Check if the given index is within the current value capacity
+   * of the vector
+   *
+   * @param index  position to check
+   * @return true if index is within the current value capacity
+   */
+  public boolean isSafe(int index) {
+    return index < getValueCapacity();
+  }
+
+  /**
+   * Check if element at given index is null.
+   *
+   * @param index  position of element
+   * @return true if element at given index is null, false otherwise
+   */
+  public boolean isNull(int index) {
+    return (isSet(index) == 0);
+  }
+
+  /**
+   * Same as {@link #isNull(int)}.
+   *
+   * @param index  position of element
+   * @return 1 if element at given index is not null, 0 otherwise
+   */
+  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));
+  }
+
+  /**
+   * Mark the particular position in the vector as non-null.
+   *
+   * @param index position of the element.
+   */
+  @Override
+  public void setIndexDefined(int index) {
+    handleSafe(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+  }
+
+  public void set(int index, byte[] value, int start, int length) {
+    throw new UnsupportedOperationException();
+  }
+
+  public void setSafe(int index, byte[] value, int start, int length) {
+    throw new UnsupportedOperationException();
+  }
+
+  public void set(int index, ByteBuffer value, int start, int length) {
+    throw new UnsupportedOperationException();
+  }
+
+  public void setSafe(int index, ByteBuffer value, int start, int length) {
+    throw new UnsupportedOperationException();
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                helper methods for setters                      *
+   *                                                                *
+   ******************************************************************/
+
+
+  protected void handleSafe(int index) {
+    while (index >= getValueCapacity()) {
+      decrementAllocationMonitor();
+      reAlloc();
+    }
+  }
 }
\ No newline at end of file
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 47daeee..edf4987 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -24,6 +24,7 @@ import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.OutOfMemoryException;
 import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.NullableMapVector;
 import org.apache.arrow.vector.schema.ArrowFieldNode;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
@@ -31,928 +32,1219 @@ import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
 public abstract class BaseNullableVariableWidthVector extends BaseValueVector
-        implements VariableWidthVector, FieldVector {
-   private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
-   private static final int INITIAL_BYTE_COUNT = INITIAL_VALUE_ALLOCATION * DEFAULT_RECORD_BYTE_COUNT;
-
-   private int valueAllocationSizeInBytes;
-   private int validityAllocationSizeInBytes;
-   private int offsetAllocationSizeInBytes;
-
-   /* protected members */
-   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;
-   protected ArrowBuf offsetBuffer;
-   protected int valueCount;
-   protected int lastSet;
-   protected final Field field;
-   private boolean cleared;
-
-   public BaseNullableVariableWidthVector(final String name, final BufferAllocator allocator,
-                                          FieldType fieldType) {
-      super(name, allocator);
-      valueAllocationSizeInBytes = INITIAL_BYTE_COUNT;
-      validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
-      offsetAllocationSizeInBytes = (INITIAL_VALUE_ALLOCATION) * OFFSET_WIDTH;
-      field = new Field(name, fieldType, null);
-      valueCount = 0;
-      lastSet = -1;
-      offsetBuffer = allocator.getEmpty();
-      validityBuffer = allocator.getEmpty();
-      valueBuffer = allocator.getEmpty();
-      cleared = false;
-   }
-
-   /* TODO:
-    *
-    * see if getNullCount() can be made faster -- O(1)
-    */
+        implements VariableWidthVector, FieldVector, NullableVectorDefinitionSetter {
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+  private static final int INITIAL_BYTE_COUNT = INITIAL_VALUE_ALLOCATION * DEFAULT_RECORD_BYTE_COUNT;
+
+  private int valueAllocationSizeInBytes;
+  private int validityAllocationSizeInBytes;
+  private int offsetAllocationSizeInBytes;
+
+  /* protected members */
+  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;
+  protected ArrowBuf offsetBuffer;
+  protected int valueCount;
+  protected int lastSet;
+  protected final Field field;
+  private boolean cleared;
+
+  public BaseNullableVariableWidthVector(final String name, final BufferAllocator allocator,
+                                         FieldType fieldType) {
+    super(name, allocator);
+    valueAllocationSizeInBytes = INITIAL_BYTE_COUNT;
+    validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
+    offsetAllocationSizeInBytes = (INITIAL_VALUE_ALLOCATION) * OFFSET_WIDTH;
+    field = new Field(name, fieldType, null);
+    valueCount = 0;
+    lastSet = -1;
+    offsetBuffer = allocator.getEmpty();
+    validityBuffer = allocator.getEmpty();
+    valueBuffer = allocator.getEmpty();
+    cleared = false;
+  }
 
   /* TODO:
-    * Once the entire hierarchy has been refactored, move common functions
-    * like getNullCount(), splitAndTransferValidityBuffer to top level
-    * base class BaseValueVector.
-    *
-    * Along with this, some class members (validityBuffer) can also be
-    * abstracted out to top level base class.
-    *
-    * Right now BaseValueVector is the top level base class for other
-    * 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.
-    */
-
-
-   @Override
-   @Deprecated
-   public VariableWidthMutator getMutator() {
-      throw new  UnsupportedOperationException("Mutator is not supported for writing into vector");
-   }
-
-   @Override
-   @Deprecated
-   public VariableWidthAccessor getAccessor() {
-      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;
-      if (size > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
-      }
-      valueAllocationSizeInBytes = (int)size;
-      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());
-   }
-
-   private int getValidityBufferValueCapacity() {
-      return (int)(validityBuffer.capacity() * 8L);
-   }
-
-   private int getOffsetBufferValueCapacity() {
-      return (int)((offsetBuffer.capacity() * 1.0)/OFFSET_WIDTH);
-   }
-
-   /**
-    * 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);
-      valueBuffer = releaseBuffer(valueBuffer);
-      offsetBuffer = releaseBuffer(offsetBuffer);
-      cleared = true;
-      lastSet = -1;
-      valueCount = 0;
-   }
-
-   @Override
-   @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()) {
-         throw new IllegalArgumentException("primitive type vector can not have children");
-      }
-   }
-
-   /**
-    * 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);
-      ArrowBuf offBuffer = ownBuffers.get(1);
-      ArrowBuf dataBuffer = ownBuffers.get(2);
-
-      validityBuffer.release();
-      validityBuffer = bitBuffer.retain(allocator);
-      offsetBuffer.release();
-      offsetBuffer = offBuffer.retain(allocator);
-      valueBuffer.release();
-      valueBuffer = dataBuffer.retain(allocator);
-
-      lastSet = fieldNode.getLength() - 1;
-      valueCount = fieldNode.getLength();
-   }
-
-   /**
-    * Get the buffers belonging to this vector
-    * @return the inner buffers.
-    */
-   public List<ArrowBuf> getFieldBuffers() {
-      List<ArrowBuf> result = new ArrayList<>(3);
+   * see if getNullCount() can be made faster -- O(1)
+   */
+
+  /* TODO:
+   * Once the entire hierarchy has been refactored, move common functions
+   * like getNullCount(), splitAndTransferValidityBuffer to top level
+   * base class BaseValueVector.
+   *
+   * Along with this, some class members (validityBuffer) can also be
+   * abstracted out to top level base class.
+   *
+   * Right now BaseValueVector is the top level base class for other
+   * 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.
+   */
+
+  @Override
+  @Deprecated
+  public VariableWidthMutator getMutator() {
+    throw new UnsupportedOperationException("Mutator is not supported for writing into vector");
+  }
+
+  @Override
+  @Deprecated
+  public VariableWidthAccessor getAccessor() {
+    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;
+    if (size > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
+    }
+    valueAllocationSizeInBytes = (int) size;
+    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());
+  }
+
+  private int getValidityBufferValueCapacity() {
+    return (int) (validityBuffer.capacity() * 8L);
+  }
+
+  private int getOffsetBufferValueCapacity() {
+    return (int) ((offsetBuffer.capacity() * 1.0) / OFFSET_WIDTH);
+  }
+
+  /**
+   * 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);
+    valueBuffer = releaseBuffer(valueBuffer);
+    offsetBuffer = releaseBuffer(offsetBuffer);
+    cleared = true;
+    lastSet = -1;
+    valueCount = 0;
+  }
+
+  @Override
+  @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()) {
+      throw new IllegalArgumentException("primitive type vector can not have children");
+    }
+  }
+
+  /**
+   * 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);
+    ArrowBuf offBuffer = ownBuffers.get(1);
+    ArrowBuf dataBuffer = ownBuffers.get(2);
+
+    validityBuffer.release();
+    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
+    offsetBuffer.release();
+    offsetBuffer = offBuffer.retain(allocator);
+    valueBuffer.release();
+    valueBuffer = dataBuffer.retain(allocator);
+
+    lastSet = fieldNode.getLength() - 1;
+    valueCount = fieldNode.getLength();
+  }
+
+  /**
+   * Get the buffers belonging to this vector
+   * @return the inner buffers.
+   */
+  public List<ArrowBuf> getFieldBuffers() {
+    List<ArrowBuf> result = new ArrayList<>(3);
+    setReaderAndWriterIndex();
+    result.add(validityBuffer);
+    result.add(offsetBuffer);
+    result.add(valueBuffer);
+
+    return result;
+  }
+
+  /**
+   * Set the reader and writer indexes for the inner buffers.
+   */
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    offsetBuffer.readerIndex(0);
+    valueBuffer.readerIndex(0);
+    if (valueCount == 0) {
+      validityBuffer.writerIndex(0);
+      offsetBuffer.writerIndex(0);
+      valueBuffer.writerIndex(0);
+    } else {
       final int lastDataOffset = getstartOffset(valueCount);
-      validityBuffer.readerIndex(0);
       validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
-      offsetBuffer.readerIndex(0);
       offsetBuffer.writerIndex((valueCount + 1) * OFFSET_WIDTH);
-      valueBuffer.readerIndex(0);
       valueBuffer.writerIndex(lastDataOffset);
-
-      result.add(validityBuffer);
-      result.add(offsetBuffer);
-      result.add(valueBuffer);
-
-      return result;
-   }
-
-   /**
-    * Same as {@link #allocateNewSafe()}.
-    */
-   @Override
-   public void allocateNew() {
-      if(!allocateNewSafe()){
-         throw new OutOfMemoryException("Failure while allocating memory.");
-      }
-   }
-
-   /**
-    * 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;
-      long curAllocationSizeValidity = validityAllocationSizeInBytes;
-      long curAllocationSizeOffset = offsetAllocationSizeInBytes;
-
-      if (curAllocationSizeValue > MAX_ALLOCATION_SIZE ||
-              curAllocationSizeOffset > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory exceeds limit");
-      }
-
-      /* we are doing a new allocation -- release the current buffers */
+    }
+  }
+
+  /**
+   * Same as {@link #allocateNewSafe()}.
+   */
+  @Override
+  public void allocateNew() {
+    if (!allocateNewSafe()) {
+      throw new OutOfMemoryException("Failure while allocating memory.");
+    }
+  }
+
+  /**
+   * 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;
+    long curAllocationSizeValidity = validityAllocationSizeInBytes;
+    long curAllocationSizeOffset = offsetAllocationSizeInBytes;
+
+    if (curAllocationSizeValue > MAX_ALLOCATION_SIZE ||
+            curAllocationSizeOffset > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory exceeds limit");
+    }
+
+    /* we are doing a new allocation -- release the current buffers */
+    clear();
+
+    try {
+      allocateBytes(curAllocationSizeValue, curAllocationSizeValidity, curAllocationSizeOffset);
+    } catch (Exception e) {
+      e.printStackTrace();
       clear();
-
-      try {
-         allocateBytes(curAllocationSizeValue, curAllocationSizeValidity, curAllocationSizeOffset);
-      } catch (Exception e) {
-         e.printStackTrace();
-         clear();
-         return false;
-      }
-
-      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 = getValidityBufferSizeFromCount(valueCount);
-
-      if (totalBytes > MAX_ALLOCATION_SIZE ||
-              offsetBufferSize > MAX_ALLOCATION_SIZE) {
-         throw new OversizedAllocationException("Requested amount of memory exceeds limit");
-      }
-
-      /* we are doing a new allocation -- release the current buffers */
+      return false;
+    }
+
+    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 = getValidityBufferSizeFromCount(valueCount);
+
+    if (totalBytes > MAX_ALLOCATION_SIZE ||
+            offsetBufferSize > MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Requested amount of memory exceeds limit");
+    }
+
+    /* we are doing a new allocation -- release the current buffers */
+    clear();
+
+    try {
+      allocateBytes(totalBytes, validityBufferSize, offsetBufferSize);
+    } catch (Exception e) {
+      e.printStackTrace();
       clear();
-
-      try {
-         allocateBytes(totalBytes, validityBufferSize, offsetBufferSize);
-      } catch (Exception e) {
-         e.printStackTrace();
-         clear();
-      }
-   }
-
-   /* allocate the inner buffers */
-   private void allocateBytes(final long valueBufferSize, final long validityBufferSize,
-                              final long offsetBufferSize) {
-      /* allocate data buffer */
-      int curSize = (int)valueBufferSize;
-      valueBuffer = allocator.buffer(curSize);
-      valueBuffer.readerIndex(0);
-      valueAllocationSizeInBytes = curSize;
-      allocateValidityBuffer(validityBufferSize);
-      allocateOffsetBuffer(offsetBufferSize);
-   }
-
-   /* allocate offset buffer */
-   private void allocateOffsetBuffer(final long size) {
-      final int curSize = (int)size;
-      offsetBuffer = allocator.buffer(curSize);
-      offsetBuffer.readerIndex(0);
-      offsetAllocationSizeInBytes = curSize;
-      initOffsetBuffer();
-   }
-
-   /* allocate validity buffer */
-   private void allocateValidityBuffer(final long size) {
-      final int curSize = (int)size;
-      validityBuffer = allocator.buffer(curSize);
-      validityBuffer.readerIndex(0);
-      validityAllocationSizeInBytes = curSize;
-      initValidityBuffer();
-   }
-
-   /**
-    * Resize the vector to increase the capacity. The internal behavior is to
-    * double the current value capacity.
-    */
-   public void reAlloc() {
-      reallocValueBuffer();
-      reallocValidityAndOffsetBuffers();
-   }
-
-   protected void reallocValueBuffer() {
-      long baseSize = valueAllocationSizeInBytes;
-      final int currentBufferCapacity = valueBuffer.capacity();
-
-      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, valueBuffer, 0, currentBufferCapacity);
-      valueBuffer.release();
-      valueBuffer = newBuf;
-      valueAllocationSizeInBytes = (int)newAllocationSize;
-   }
-
-   protected void reallocValidityAndOffsetBuffers() {
-      offsetBuffer = reallocBufferHelper(offsetBuffer, true);
-      validityBuffer = reallocBufferHelper(validityBuffer, false);
-   }
-
-   /* 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
-              : 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, buffer, 0, currentBufferCapacity);
-      final int halfNewCapacity = newBuf.capacity() / 2;
-      newBuf.setZero(halfNewCapacity, halfNewCapacity);
-      buffer.release(1);
-      buffer = newBuf;
-      if (offsetBuffer) {
-         offsetAllocationSizeInBytes = (int)newAllocationSize;
-      }
-      else {
-         validityAllocationSizeInBytes = (int)newAllocationSize;
-      }
-
-      return buffer;
-   }
-
-   /**
-    * Get the size (number of bytes) of underlying data buffer.
-    * @return
-    */
-   @Override
-   public int getByteCapacity(){
-      return valueBuffer.capacity();
-   }
-
-   @Override
-   public int getCurrentSizeInBytes(){
+    }
+  }
+
+  /* allocate the inner buffers */
+  private void allocateBytes(final long valueBufferSize, final long validityBufferSize,
+                             final long offsetBufferSize) {
+    /* allocate data buffer */
+    int curSize = (int) valueBufferSize;
+    valueBuffer = allocator.buffer(curSize);
+    valueBuffer.readerIndex(0);
+    valueAllocationSizeInBytes = curSize;
+    allocateValidityBuffer(validityBufferSize);
+    allocateOffsetBuffer(offsetBufferSize);
+  }
+
+  /* allocate offset buffer */
+  private void allocateOffsetBuffer(final long size) {
+    final int curSize = (int) size;
+    offsetBuffer = allocator.buffer(curSize);
+    offsetBuffer.readerIndex(0);
+    offsetAllocationSizeInBytes = curSize;
+    initOffsetBuffer();
+  }
+
+  /* allocate validity buffer */
+  private void allocateValidityBuffer(final long size) {
+    final int curSize = (int) size;
+    validityBuffer = allocator.buffer(curSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = curSize;
+    initValidityBuffer();
+  }
+
+  /**
+   * Resize the vector to increase the capacity. The internal behavior is to
+   * double the current value capacity.
+   */
+  public void reAlloc() {
+    reallocDataBuffer();
+    reallocValidityAndOffsetBuffers();
+  }
+
+  /**
+   * Reallocate the data buffer. Data Buffer stores the actual data for
+   * VARCHAR or VARBINARY elements in the vector. The behavior is to double
+   * the size of buffer.
+   * @throws OversizedAllocationException if the desired new size is more than
+   *                                      max allowed
+   * @throws OutOfMemoryException if the internal memory allocation fails
+   */
+  public void reallocDataBuffer() {
+    long baseSize = valueAllocationSizeInBytes;
+    final int currentBufferCapacity = valueBuffer.capacity();
+
+    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, valueBuffer, 0, currentBufferCapacity);
+    valueBuffer.release();
+    valueBuffer = newBuf;
+    valueAllocationSizeInBytes = (int) newAllocationSize;
+  }
+
+  /**
+   * Reallocate the validity and offset buffers for this vector. Validity
+   * buffer is used to track the NULL or NON-NULL nature of elements in
+   * the vector and offset buffer is used to store the lengths of variable
+   * width elements in the vector.
+   *
+   * Note that data buffer for variable length vectors moves independent
+   * of the companion validity and offset buffers. This is in
+   * contrast to what we have for fixed width vectors.
+   *
+   * So even though we may have setup an initial capacity of 1024
+   * elements in the vector, it is quite possible
+   * that we need to reAlloc() the data buffer when we are setting
+   * the 5th element in the vector simply because previous
+   * variable length elements have exhausted the buffer capacity.
+   * However, we really don't need to reAlloc() validity and
+   * offset buffers until we try to set the 1025th element
+   * This is why we do a separate check for safe methods to
+   * determine which buffer needs reallocation.
+   * @throws OversizedAllocationException if the desired new size is more than
+   *                                      max allowed
+   * @throws OutOfMemoryException if the internal memory allocation fails
+   */
+  public void reallocValidityAndOffsetBuffers() {
+    offsetBuffer = reallocBufferHelper(offsetBuffer, true);
+    validityBuffer = reallocBufferHelper(validityBuffer, false);
+  }
+
+  /* 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
+            : 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, buffer, 0, currentBufferCapacity);
+    final int halfNewCapacity = newBuf.capacity() / 2;
+    newBuf.setZero(halfNewCapacity, halfNewCapacity);
+    buffer.release(1);
+    buffer = newBuf;
+    if (offsetBuffer) {
+      offsetAllocationSizeInBytes = (int) newAllocationSize;
+    } else {
+      validityAllocationSizeInBytes = (int) newAllocationSize;
+    }
+
+    return buffer;
+  }
+
+  /**
+   * Get the size (number of bytes) of underlying data buffer.
+   * @return
+   */
+  @Override
+  public int getByteCapacity() {
+    return valueBuffer.capacity();
+  }
+
+  @Override
+  public int getCurrentSizeInBytes() {
       /* TODO */
+    return 0;
+  }
+
+  /**
+   * Get the size (number of bytes) of underlying buffers used by this
+   * vector
+   * @return size of underlying buffers.
+   */
+  @Override
+  public int getBufferSize() {
+    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;
-   }
-
-   /**
-    * Get the size (number of bytes) of underlying buffers used by this
-    * vector
-    * @return size of underlying buffers.
-    */
-   @Override
-   public int getBufferSize() {
-      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 = getValidityBufferSizeFromCount(valueCount);
-      final int offsetBufferSize = (valueCount + 1) * OFFSET_WIDTH;
+    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];
+    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;
+    setReaderAndWriterIndex();
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = new ArrowBuf[3];
       buffers[0] = validityBuffer;
       buffers[1] = offsetBuffer;
-      buffers[1] = valueBuffer;
-      if (clear) {
-         for (final ArrowBuf buffer:buffers) {
-            buffer.retain(1);
-         }
-         clear();
+      buffers[2] = valueBuffer;
+    }
+    if (clear) {
+      for (final ArrowBuf buffer : buffers) {
+        buffer.retain(1);
       }
-      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();
-      target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
-      target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer;
-      target.offsetBuffer = offsetBuffer.transferOwnership(target.allocator).buffer;
-      target.valueCount = valueCount;
-      target.setLastSet(lastSet);
       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");
-      target.clear();
-      splitAndTransferValidityBuffer(startIndex, length, target);
-      splitAndTransferOffsetBuffer(startIndex, length, target);
-      target.setLastSet(length - 1);
-      target.setValueCount(length);
-   }
-
-   /*
-    * 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);
-      final int end = offsetBuffer.getInt((startIndex + length) * OFFSET_WIDTH);
-      final int dataLength = end - start;
-      target.allocateOffsetBuffer((length + 1) * OFFSET_WIDTH);
-      for (int i = 0; i < length + 1; i++) {
-         final int relativeSourceOffset = offsetBuffer.getInt((startIndex + i) * OFFSET_WIDTH) - start;
-         target.offsetBuffer.setInt(i * OFFSET_WIDTH, relativeSourceOffset);
+    }
+    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();
+    target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
+    target.valueBuffer = valueBuffer.transferOwnership(target.allocator).buffer;
+    target.offsetBuffer = offsetBuffer.transferOwnership(target.allocator).buffer;
+    target.setLastSet(this.lastSet);
+    if (this.valueCount > 0) {
+      target.setValueCount(this.valueCount);
+    }
+    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");
+    target.clear();
+    splitAndTransferValidityBuffer(startIndex, length, target);
+    splitAndTransferOffsetBuffer(startIndex, length, target);
+    target.setLastSet(length - 1);
+    if (this.valueCount > 0) {
+      target.setValueCount(this.valueCount);
+    }
+  }
+
+  /*
+   * 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);
+    final int end = offsetBuffer.getInt((startIndex + length) * OFFSET_WIDTH);
+    final int dataLength = end - start;
+    target.allocateOffsetBuffer((length + 1) * OFFSET_WIDTH);
+    for (int i = 0; i < length + 1; i++) {
+      final int relativeSourceOffset = offsetBuffer.getInt((startIndex + i) * OFFSET_WIDTH) - start;
+      target.offsetBuffer.setInt(i * OFFSET_WIDTH, relativeSourceOffset);
+    }
+    target.valueBuffer = valueBuffer.slice(start, dataLength).transferOwnership(target.allocator).buffer;
+  }
+
+  /*
+   * 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 = 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(this.validityBuffer, firstByteSource + i, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(this.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(this.validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
+                  firstByteSource + byteSizeTarget, offset);
+
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+        } else {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1);
+        }
       }
-      target.valueBuffer = valueBuffer.slice(start, dataLength).transferOwnership(target.allocator).buffer;
-   }
-
-   /*
-    * 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 = 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(this.validityBuffer, firstByteSource + i, offset);
-               byte b2 = BitVectorHelper.getBitsFromNextByte(this.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(this.validityBuffer,
-                       firstByteSource + byteSizeTarget - 1, offset);
-               byte b2 = BitVectorHelper.getBitsFromNextByte(this.validityBuffer,
-                       firstByteSource + byteSizeTarget, offset);
-
-               target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
-            }
-            else {
-               byte b1 = BitVectorHelper.getBitsFromCurrentByte(this.validityBuffer,
-                       firstByteSource + byteSizeTarget - 1, offset);
-               target.validityBuffer.setByte(byteSizeTarget - 1, b1);
-            }
-         }
-      }
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                common getters and setters                      *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * Get the number of elements that are null in the vector
-    *
-    * @return the number of null elements.
-    */
-   public int getNullCount() {
-      return BitVectorHelper.getNullCount(validityBuffer, valueCount);
-   }
-
-   /**
-    * Check if the given index is within the current value capacity
-    * of the vector
-    *
-    * @param index  position to check
-    * @return true if index is within the current value capacity
-    */
-   public boolean isSafe(int index) {
-      return index < getValueCapacity();
-   }
-
-   /**
-    * Check if element at given index is null.
-    *
-    * @param index  position of element
-    * @return true if element at given index is null
-    */
-   public boolean isNull(int index) {
-      return (isSet(index) == 0);
-   }
-
-   /**
-    * Same as {@link #isNull(int)}.
-    *
-    * @param index  position of element
-    * @return 1 if element at given index is not null, 0 otherwise
-    */
-   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));
-   }
-
-   /**
-    * Get the value count of vector. This will always be zero unless
-    * setValueCount(int) has been called prior to calling this.
-    *
-    * @return valueCount for the vector
-    */
-   public int getValueCount(){
-      return valueCount;
-   }
-
-   /**
-    * Sets the value count for the vector
-    *
-    * @param valueCount   value count
-    */
-   public void setValueCount(int valueCount) {
-      assert valueCount >= 0;
-      this.valueCount = valueCount;
-      while (valueCount > getValueCapacity()) {
-         reallocValidityAndOffsetBuffers();
-      }
-      fillHoles(valueCount);
-      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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                helper methods for setters                      *
-    *                                                                *
-    ******************************************************************/
-
-
-   protected final void fillHoles(int index) {
-      for (int i = lastSet + 1; i < index; i++) {
-         setBytes(i, emptyByteArray, 0, emptyByteArray.length);
-      }
-      lastSet = index - 1;
-   }
-
-   protected final void setBytes(int index, byte[] value, int start, int length) {
-      /* end offset of current last element in the vector. this will
-       * be the start offset of new element we are trying to store.
-       */
-      final int startOffset = getstartOffset(index);
-      /* set new end offset */
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-      /* store the var length data in value buffer */
-      valueBuffer.setBytes(startOffset, value, start, length);
-   }
-
-   protected final int getstartOffset(int index) {
-      return offsetBuffer.getInt(index * OFFSET_WIDTH);
-   }
-
-   protected final void handleSafe(int index, int dataLength) {
-      /*
-       * IMPORTANT:
-       * value buffer for variable length vectors moves independent
-       * of the companion validity and offset buffers. This is in
-       * contrast to what we have for fixed width vectors.
-       *
-       * Here there is no concept of getValueCapacity() in the
-       * data stream. getValueCapacity() is applicable only to validity
-       * and offset buffers.
-       *
-       * So even though we may have setup an initial capacity of 1024
-       * elements in the vector, it is quite possible
-       * that we need to reAlloc() the data buffer when we are setting
-       * the 5th element in the vector simply because previous
-       * variable length elements have exhausted the buffer capacity.
-       * However, we really don't need to reAlloc() validity and
-       * offset buffers until we try to set the 1025th element
-       * This is why we do a separate check for safe methods to
-       * determine which buffer needs reallocation.
-       */
-      while (index >= getValueCapacity()) {
-         reallocValidityAndOffsetBuffers();
-      }
-      final int startOffset = getstartOffset(index);
-      while (valueBuffer.capacity() < (startOffset + dataLength)) {
-         reallocValueBuffer();
-      }
-   }
-
-   /**
-    * 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;
-   }
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                common getters and setters                      *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * Get the number of elements that are null in the vector
+   *
+   * @return the number of null elements.
+   */
+  public int getNullCount() {
+    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
+  }
+
+  /**
+   * Check if the given index is within the current value capacity
+   * of the vector
+   *
+   * @param index  position to check
+   * @return true if index is within the current value capacity
+   */
+  public boolean isSafe(int index) {
+    return index < getValueCapacity();
+  }
+
+  /**
+   * Check if element at given index is null.
+   *
+   * @param index  position of element
+   * @return true if element at given index is null
+   */
+  public boolean isNull(int index) {
+    return (isSet(index) == 0);
+  }
+
+  /**
+   * Same as {@link #isNull(int)}.
+   *
+   * @param index  position of element
+   * @return 1 if element at given index is not null, 0 otherwise
+   */
+  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));
+  }
+
+  /**
+   * Get the value count of vector. This will always be zero unless
+   * setValueCount(int) has been called prior to calling this.
+   *
+   * @return valueCount for the vector
+   */
+  public int getValueCount() {
+    return valueCount;
+  }
+
+  /**
+   * Sets the value count for the vector
+   *
+   * @param valueCount   value count
+   */
+  public void setValueCount(int valueCount) {
+    assert valueCount >= 0;
+    this.valueCount = valueCount;
+    while (valueCount > getValueCapacity()) {
+      reallocValidityAndOffsetBuffers();
+    }
+    fillHoles(valueCount);
+    lastSet = valueCount - 1;
+    setReaderAndWriterIndex();
+  }
+
+  /**
+   * 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 offsetBuffer.getLong(index * OFFSET_WIDTH);
+  }
+
+  /**
+   * Mark the particular position in the vector as non-null.
+   *
+   * @param index position of the element.
+   */
+  @Override
+  public void setIndexDefined(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      validityBuffer = reallocBufferHelper(validityBuffer, false);
+    }
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+  }
+
+  /**
+   * Sets the value length for an element.
+   *
+   * @param index   position of the element to set
+   * @param length  length of the element
+   */
+  public void setValueLengthSafe(int index, int length) {
+    assert index >= 0;
+    handleSafe(index, length);
+    fillHoles(index);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    lastSet = index;
+  }
+
+  /**
+   * Get the variable length element at specified index as Text.
+   *
+   * @param index   position of element to get
+   * @return greater than 0 length for non-null element, 0 otherwise
+   */
+  public int getValueLength(int index) {
+    assert index >= 0;
+    if (isSet(index) == 0) {
+      return 0;
+    }
+    final int startOffset = getstartOffset(index);
+    final int dataLength =
+            offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
+    return dataLength;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the supplied
+   * byte array. This is same as using {@link #set(int, byte[], int, int)}
+   * with start as 0 and length as value.length
+   *
+   * @param index   position of the element to set
+   * @param value   array of bytes to write
+   */
+  public void set(int index, byte[] value) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setBytes(index, value, 0, value.length);
+    lastSet = index;
+  }
+
+  /**
+   * Same as {@link #set(int, byte[])} 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 value   array of bytes to write
+   */
+  public void setSafe(int index, byte[] value) {
+    assert index >= 0;
+    fillEmpties(index);
+    handleSafe(index, value.length);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setBytes(index, value, 0, value.length);
+    lastSet = index;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the supplied
+   * byte array.
+   *
+   * @param index   position of the element to set
+   * @param value   array of bytes to write
+   * @param start   start index in array of bytes
+   * @param length  length of data in array of bytes
+   */
+  public void set(int index, byte[] value, int start, int length) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setBytes(index, value, start, length);
+    lastSet = index;
+  }
+
+  /**
+   * Same as {@link #set(int, byte[], int, int)} 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 value   array of bytes to write
+   * @param start   start index in array of bytes
+   * @param length  length of data in array of bytes
+   */
+  public void setSafe(int index, byte[] value, int start, int length) {
+    assert index >= 0;
+    fillEmpties(index);
+    handleSafe(index, length);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setBytes(index, value, start, length);
+    lastSet = index;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the
+   * content in supplied ByteBuffer
+   *
+   * @param index   position of the element to set
+   * @param value   ByteBuffer with data
+   * @param start   start index in ByteBuffer
+   * @param length  length of data in ByteBuffer
+   */
+  public void set(int index, ByteBuffer value, int start, int length) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    valueBuffer.setBytes(startOffset, value, start, length);
+    lastSet = index;
+  }
+
+  /**
+   * Same as {@link #set(int, ByteBuffer, int, int)} 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 value   ByteBuffer with data
+   * @param start   start index in ByteBuffer
+   * @param length  length of data in ByteBuffer
+   */
+  public void setSafe(int index, ByteBuffer value, int start, int length) {
+    assert index >= 0;
+    fillEmpties(index);
+    handleSafe(index, length);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    valueBuffer.setBytes(startOffset, value, start, length);
+    lastSet = index;
+  }
+
+  /**
+   * Set the element at the given index to null.
+   *
+   * @param index   position of element
+   */
+  public void setNull(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      validityBuffer = reallocBufferHelper(validityBuffer, false);
+    }
+    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 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;
+    final int dataLength = end - start;
+    fillHoles(index);
+    BitVectorHelper.setValidityBit(validityBuffer, index, isSet);
+    final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, buffer, start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * 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;
+    final int dataLength = end - start;
+    fillEmpties(index);
+    handleSafe(index, end);
+    BitVectorHelper.setValidityBit(validityBuffer, index, isSet);
+    final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, buffer, start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * 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 start start position of data in buffer
+   * @param length length 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 start, int length, ArrowBuf buffer) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    final ArrowBuf bb = buffer.slice(start, length);
+    valueBuffer.setBytes(startOffset, bb);
+    lastSet = index;
+  }
+
+  /**
+   * 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 start start position of data in buffer
+   * @param length length 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 start, int length, ArrowBuf buffer) {
+    assert index >= 0;
+    fillEmpties(index);
+    handleSafe(index, length);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = offsetBuffer.getInt(index * OFFSET_WIDTH);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    final ArrowBuf bb = buffer.slice(start, length);
+    valueBuffer.setBytes(startOffset, bb);
+    lastSet = index;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                helper methods for setters                      *
+   *                                                                *
+   ******************************************************************/
+
+
+  protected final void fillHoles(int index) {
+    for (int i = lastSet + 1; i < index; i++) {
+      setBytes(i, emptyByteArray, 0, emptyByteArray.length);
+    }
+    lastSet = index - 1;
+  }
+
+  protected final void setBytes(int index, byte[] value, int start, int length) {
+    /* end offset of current last element in the vector. this will
+     * be the start offset of new element we are trying to store.
+     */
+    final int startOffset = getstartOffset(index);
+    /* set new end offset */
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
+    /* store the var length data in value buffer */
+    valueBuffer.setBytes(startOffset, value, start, length);
+  }
+
+  protected final int getstartOffset(int index) {
+    return offsetBuffer.getInt(index * OFFSET_WIDTH);
+  }
+
+  protected final void handleSafe(int index, int dataLength) {
+    /*
+     * IMPORTANT:
+     * value buffer for variable length vectors moves independent
+     * of the companion validity and offset buffers. This is in
+     * contrast to what we have for fixed width vectors.
+     *
+     * Here there is no concept of getValueCapacity() in the
+     * data stream. getValueCapacity() is applicable only to validity
+     * and offset buffers.
+     *
+     * So even though we may have setup an initial capacity of 1024
+     * elements in the vector, it is quite possible
+     * that we need to reAlloc() the data buffer when we are setting
+     * the 5th element in the vector simply because previous
+     * variable length elements have exhausted the buffer capacity.
+     * However, we really don't need to reAlloc() validity and
+     * offset buffers until we try to set the 1025th element
+     * This is why we do a separate check for safe methods to
+     * determine which buffer needs reallocation.
+     */
+    while (index >= getValueCapacity()) {
+      reallocValidityAndOffsetBuffers();
+    }
+    final int startOffset = getstartOffset(index);
+    while (valueBuffer.capacity() < (startOffset + dataLength)) {
+      reallocDataBuffer();
+    }
+  }
+
+  /**
+   * 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 1b96782..fc0ab3e 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
index 591d13c..c6d404e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BitVector.java
@@ -251,12 +251,12 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
 
   @Override
   public Mutator getMutator() {
-    return new Mutator();
+    return mutator;
   }
 
   @Override
   public Accessor getAccessor() {
-    return new Accessor();
+    return accessor;
   }
 
   @Override
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 69325b5..23252ca 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -20,6 +20,7 @@ package org.apache.arrow.vector;
 
 import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.schema.ArrowFieldNode;
 
 /**
  * Helper class for performing generic operations on a bit vector buffer.
@@ -27,139 +28,174 @@ import org.apache.arrow.memory.BufferAllocator;
  */
 public class BitVectorHelper {
 
-   /**
-    * Get the index of byte corresponding to bit index in validity buffer
-    */
-   public static int byteIndex(int absoluteBitIndex) {
-      return absoluteBitIndex >> 3;
-   }
-
-   /**
-    * Get the relative index of bit within the byte in validity buffer
-    */
-   public static int bitIndex(int absoluteBitIndex) {
-      return absoluteBitIndex & 7;
-   }
-
-   /**
-    * 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);
-      final byte bitMask = (byte) (1L << bitIndex);
+  /**
+   * Get the index of byte corresponding to bit index in validity buffer
+   */
+  public static int byteIndex(int absoluteBitIndex) {
+    return absoluteBitIndex >> 3;
+  }
+
+  /**
+   * Get the relative index of bit within the byte in validity buffer
+   */
+  public static int bitIndex(int absoluteBitIndex) {
+    return absoluteBitIndex & 7;
+  }
+
+  /**
+   * 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);
+    final byte bitMask = (byte) (1L << bitIndex);
+    currentByte |= bitMask;
+    validityBuffer.setByte(byteIndex, currentByte);
+  }
+
+  /**
+   * 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);
+    final byte bitMask = (byte) (1L << bitIndex);
+    if (value != 0) {
       currentByte |= bitMask;
-      validityBuffer.setByte(byteIndex, currentByte);
-   }
-
-   /**
-    * 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);
-      final byte bitMask = (byte) (1L << bitIndex);
-      if (value != 0) {
-         currentByte |= bitMask;
-      } else {
-         currentByte -= (bitMask & currentByte);
-      }
-      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));
+    } else {
+      currentByte -= (bitMask & currentByte);
+    }
+    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)));
+  }
+
+  public static ArrowBuf loadValidityBuffer(final ArrowFieldNode fieldNode,
+                                            final ArrowBuf sourceValidityBuffer,
+                                            final BufferAllocator allocator) {
+    final int valueCount = fieldNode.getLength();
+    ArrowBuf newBuffer = null;
+    /* either all NULLs or all non-NULLs */
+    if (fieldNode.getNullCount() == 0 || fieldNode.getNullCount() == valueCount) {
+      newBuffer = allocator.buffer(getValidityBufferSize(valueCount));
+      newBuffer.setZero(0, newBuffer.capacity());
+      if (fieldNode.getNullCount() != 0) {
+        /* all NULLs */
+        return newBuffer;
       }
-      setValidityBit(validityBuffer, index, value);
-      if (index == (valueCount - 1)) {
-         validityBuffer.writerIndex(getValidityBufferSize(valueCount));
+      /* all non-NULLs */
+      int fullBytesCount = valueCount / 8;
+      for (int i = 0; i < fullBytesCount; ++i) {
+        newBuffer.setByte(i, 0xFF);
       }
-
-      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 remainder = valueCount % 8;
+      if (remainder > 0) {
+        byte bitMask = (byte) (0xFFL >>> ((8 - remainder) & 7));
+        newBuffer.setByte(fullBytesCount, bitMask);
       }
-      int nullCount = (sizeInBytes * 8) - count;
-      /* if the valueCount is not a multiple of 8,
-       * the bits on the right were counted as null bits.
+    } else {
+      /* mixed byte pattern -- create another ArrowBuf associated with the
+       * target allocator
        */
-      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);
-   }
+      newBuffer = sourceValidityBuffer.retain(allocator);
+    }
 
-   public static byte getBitsFromNextByte(ArrowBuf data, int index, int offset) {
-      return (byte)((data.getByte(index) << (8 - offset)));
-   }
+    return newBuffer;
+  }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/GenerateSampleData.java b/java/vector/src/main/java/org/apache/arrow/vector/GenerateSampleData.java
new file mode 100644
index 0000000..806beb5
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/GenerateSampleData.java
@@ -0,0 +1,336 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 java.math.BigDecimal;
+import java.nio.charset.Charset;
+
+/**
+ * Helper class to generate test data for Nullable fixed and variable
+ * width scalar vectors. Previous implementations of java vector classes
+ * provided generateTestData(now deprecated) API to populate the vector
+ * with sample data. This class should be used for that purpose.
+ */
+public class GenerateSampleData {
+
+  public static void generateTestData(final ValueVector vector, final int valueCount) {
+    if (vector instanceof NullableIntVector) {
+      writeIntData((NullableIntVector) vector, valueCount);
+    } else if (vector instanceof NullableDecimalVector) {
+      writeDecimalData((NullableDecimalVector) vector, valueCount);
+    } else if (vector instanceof NullableBitVector) {
+      writeBooleanData((NullableBitVector) vector, valueCount);
+    } else if (vector instanceof NullableVarCharVector) {
+      writeVarCharData((NullableVarCharVector) vector, valueCount);
+    } else if (vector instanceof NullableVarBinaryVector) {
+      writeVarBinaryData((NullableVarBinaryVector) vector, valueCount);
+    } else if (vector instanceof NullableBigIntVector) {
+      writeBigIntData((NullableBigIntVector) vector, valueCount);
+    } else if (vector instanceof NullableFloat4Vector) {
+      writeFloatData((NullableFloat4Vector) vector, valueCount);
+    } else if (vector instanceof NullableFloat8Vector) {
+      writeDoubleData((NullableFloat8Vector) vector, valueCount);
+    } else if (vector instanceof NullableDateDayVector) {
+      writeDateDayData((NullableDateDayVector) vector, valueCount);
+    } else if (vector instanceof NullableDateMilliVector) {
+      writeDateMilliData((NullableDateMilliVector) vector, valueCount);
+    } else if (vector instanceof NullableIntervalDayVector) {
+      writeIntervalDayData((NullableIntervalDayVector) vector, valueCount);
+    } else if (vector instanceof NullableIntervalYearVector) {
+      writeIntervalYearData((NullableIntervalYearVector) vector, valueCount);
+    } else if (vector instanceof NullableSmallIntVector) {
+      writeSmallIntData((NullableSmallIntVector) vector, valueCount);
+    } else if (vector instanceof NullableTinyIntVector) {
+      writeTinyIntData((NullableTinyIntVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeMicroVector) {
+      writeTimeMicroData((NullableTimeMicroVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeMilliVector) {
+      writeTimeMilliData((NullableTimeMilliVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeNanoVector) {
+      writeTimeNanoData((NullableTimeNanoVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeSecVector) {
+      writeTimeSecData((NullableTimeSecVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampSecVector) {
+      writeTimeStampData((NullableTimeStampSecVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampMicroVector) {
+      writeTimeStampData((NullableTimeStampMicroVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampMilliVector) {
+      writeTimeStampData((NullableTimeStampMilliVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampNanoVector) {
+      writeTimeStampData((NullableTimeStampNanoVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampSecTZVector) {
+      writeTimeStampData((NullableTimeStampSecTZVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampMicroTZVector) {
+      writeTimeStampData((NullableTimeStampMicroTZVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampMilliTZVector) {
+      writeTimeStampData((NullableTimeStampMilliTZVector) vector, valueCount);
+    } else if (vector instanceof NullableTimeStampNanoTZVector) {
+      writeTimeStampData((NullableTimeStampNanoTZVector) vector, valueCount);
+    }
+  }
+
+  private static void writeTimeStampData(NullableTimeStampVector vector, int valueCount) {
+    final long even = 100000;
+    final long odd = 200000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeDecimalData(NullableDecimalVector vector, int valueCount) {
+    final BigDecimal even = new BigDecimal(0.0543278923);
+    final BigDecimal odd = new BigDecimal(2.0543278923);
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeIntData(NullableIntVector vector, int valueCount) {
+    final int even = 1000;
+    final int odd = 2000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeBooleanData(NullableBitVector vector, int valueCount) {
+    final int even = 0;
+    final int odd = 1;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeIntervalYearData(NullableIntervalYearVector vector, int valueCount) {
+    final int even = 1;
+    final int odd = 2;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeIntervalDayData(NullableIntervalDayVector vector, int valueCount) {
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, 1, 50);
+      } else {
+        vector.setSafe(i, 2, 100);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeTimeSecData(NullableTimeSecVector vector, int valueCount) {
+    final int even = 500;
+    final int odd = 900;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeTimeMilliData(NullableTimeMilliVector vector, int valueCount) {
+    final int even = 1000;
+    final int odd = 2000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeTimeMicroData(NullableTimeMicroVector vector, int valueCount) {
+    final long even = 1000000000;
+    final long odd = 2000000000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+
+  }
+
+  private static void writeTimeNanoData(NullableTimeNanoVector vector, int valueCount) {
+    final long even = 1000000000;
+    final long odd = 2000000000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeDateDayData(NullableDateDayVector vector, int valueCount) {
+    final int even = 1000;
+    final int odd = 2000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeDateMilliData(NullableDateMilliVector vector, int valueCount) {
+    final long even = 1000000000;
+    final long odd = 2000000000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeSmallIntData(NullableSmallIntVector vector, int valueCount) {
+    final short even = 10;
+    final short odd = 20;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeTinyIntData(NullableTinyIntVector vector, int valueCount) {
+    final byte even = 1;
+    final byte odd = 2;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeBigIntData(NullableBigIntVector vector, int valueCount) {
+    final long even = 1000000000;
+    final long odd = 2000000000;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeFloatData(NullableFloat4Vector vector, int valueCount) {
+    final float even = 20.3f;
+    final float odd = 40.2f;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeDoubleData(NullableFloat8Vector vector, int valueCount) {
+    final double even = 20.2373;
+    final double odd = 40.2378;
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeVarBinaryData(NullableVarBinaryVector vector, int valueCount) {
+    Charset utf8Charset = Charset.forName("UTF-8");
+    final byte[] even = "AAAAA1".getBytes(utf8Charset);
+    final byte[] odd = "BBBBBBBBB2".getBytes(utf8Charset);
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+
+  private static void writeVarCharData(NullableVarCharVector vector, int valueCount) {
+    Charset utf8Charset = Charset.forName("UTF-8");
+    final byte[] even = "AAAAA1".getBytes(utf8Charset);
+    final byte[] odd = "BBBBBBBBB2".getBytes(utf8Charset);
+    for (int i = 0; i < valueCount; i++) {
+      if (i % 2 == 0) {
+        vector.setSafe(i, even);
+      } else {
+        vector.setSafe(i, odd);
+      }
+    }
+    vector.setValueCount(valueCount);
+  }
+}
+
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
index 2534273..eca6592 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,335 +34,335 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableBigIntVector extends BaseNullableFixedWidthVector {
-   public static final byte TYPE_WIDTH = 8;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.BIGINT;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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");
-      }
+  public static final byte TYPE_WIDTH = 8;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.BIGINT;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableBigIntHolder 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 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, NullableBigIntHolder 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);
-      }
-   }
-
-   /**
-    * 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));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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, NullableBigIntHolder 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, BigIntHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableBigIntHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableBigIntHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, BigIntHolder 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);
+    } 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, BigIntHolder 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, 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, NullableBigIntHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, BigIntHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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((NullableBigIntVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableBigIntVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableBigIntVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableBigIntVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableBigIntVector 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, NullableBigIntVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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((NullableBigIntVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableBigIntVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableBigIntVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableBigIntVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableBigIntVector 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, 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
index eceaa61..bd363f5 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.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
@@ -7,15 +6,15 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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;
 
@@ -39,150 +38,160 @@ import java.util.List;
  * 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) {
+  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 {
+        if (destBuffer != null) {
+          destBuffer.release();
+        }
+        destBuffer = sourceBuffer.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());
+        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);
+        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));
-            }
+          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
@@ -193,310 +202,315 @@ public class NullableBitVector extends BaseNullableFixedWidthVector {
              * (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);
+        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);
+        }
       }
-   }
-
-   /**
-    * 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) {
+    }
+
+    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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+        BitVectorHelper.setValidityBitToOne(valueBuffer, index);
       } else {
-         BitVectorHelper.setValidityBit(valueBuffer, index, 0);
+        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);
+    } 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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/NullableDateDayVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
index e6b5b59..c5ddc9e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -35,336 +35,336 @@ import org.slf4j.Logger;
  * maintained to track which elements in the vector are null.
  */
 public class NullableDateDayVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.DATEDAY;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  private static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.DATEDAY;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableDateDayHolder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Integer getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getInt(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, NullableDateDayHolder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Integer 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, NullableDateDayVector from) {
-      if (from.isSet(fromIndex) != 0) {
-         set(thisIndex, from.get(fromIndex));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableDateDayHolder 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, DateDayHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableDateDayHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableDateDayHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, DateDayHolder 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);
+    } 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, DateDayHolder 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, 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, NullableDateDayHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, DateDayHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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((NullableDateDayVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableDateDayVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableDateDayVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableDateDayVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableDateDayVector 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, NullableDateDayVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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((NullableDateDayVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableDateDayVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableDateDayVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableDateDayVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableDateDayVector 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, 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
index 8e15100..7e37055 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -37,338 +37,339 @@ import org.slf4j.Logger;
  * 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){
+  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 = valueBuffer.getLong(index * TYPE_WIDTH);
+      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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
-   }
-
-   /**
-    * 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);
+    } 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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
index 2a611c6..dcc5510 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -37,383 +37,391 @@ import java.math.BigDecimal;
  * maintained to track which elements in the vector are null.
  */
 public class NullableDecimalVector extends BaseNullableFixedWidthVector {
-   public 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){
+  public 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);
+      BitVectorHelper.setValidityBitToOne(validityBuffer, thisIndex);
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * Return scale for the decimal value
+   */
+  public int getScale() {
+    return scale;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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);
-   }
-
-   /**
-    * 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);
+    } 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);
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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/NullableFloat4Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
index 3ba5cfc..834f16c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,336 +34,336 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
-   public static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.FLOAT4;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 float get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  public static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.FLOAT4;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 float get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getFloat(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, NullableFloat4Holder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getFloat(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Float getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getFloat(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, NullableFloat4Holder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getFloat(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Float 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, NullableFloat4Vector from) {
-      if (from.isSet(fromIndex) != 0) {
-         set(thisIndex, from.get(fromIndex));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, float value) {
-      valueBuffer.setFloat(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, float 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, NullableFloat4Holder 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, Float4Holder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, float value) {
+    valueBuffer.setFloat(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, float 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, NullableFloat4Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
-   }
-
-   /**
-    * 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, float value) {
-      handleSafe(index);
-      set(index, value);
-   }
-
-   /**
-    * 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, NullableFloat4Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, Float4Holder 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);
+    } 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, Float4Holder holder) {
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setValue(index, holder.value);
+  }
+
+  /**
+   * 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, float value) {
+    handleSafe(index);
+    set(index, value);
+  }
+
+  /**
+   * 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, NullableFloat4Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, Float4Holder 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, float value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       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, float value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   /**
-    * 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, value);
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      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((NullableFloat4Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableFloat4Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableFloat4Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableFloat4Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableFloat4Vector 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, NullableFloat4Vector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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, value);
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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((NullableFloat4Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableFloat4Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableFloat4Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableFloat4Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableFloat4Vector 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, NullableFloat4Vector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
index 2fb96a4..b054d9b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,336 +34,336 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
-   public static final byte TYPE_WIDTH = 8;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.FLOAT8;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 double get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  public static final byte TYPE_WIDTH = 8;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.FLOAT8;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 double get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getDouble(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, NullableFloat8Holder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getDouble(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Double getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getDouble(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, NullableFloat8Holder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getDouble(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Double 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, NullableFloat8Vector from) {
-      if (from.isSet(fromIndex) != 0) {
-         set(thisIndex, from.get(fromIndex));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, double value) {
-      valueBuffer.setDouble(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, double 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, NullableFloat8Holder 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, Float8Holder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, double value) {
+    valueBuffer.setDouble(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, double 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, NullableFloat8Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       BitVectorHelper.setValidityBitToOne(validityBuffer, index);
       setValue(index, holder.value);
-   }
-
-   /**
-    * 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, double value) {
-      handleSafe(index);
-      set(index, value);
-   }
-
-   /**
-    * 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, NullableFloat8Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, Float8Holder 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);
+    } 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, Float8Holder holder) {
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    setValue(index, holder.value);
+  }
+
+  /**
+   * 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, double value) {
+    handleSafe(index);
+    set(index, value);
+  }
+
+  /**
+   * 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, NullableFloat8Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, Float8Holder 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, double value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       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, double value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   /**
-    * 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, value);
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      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((NullableFloat8Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableFloat8Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableFloat8Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableFloat8Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableFloat8Vector 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, NullableFloat8Vector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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, value);
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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((NullableFloat8Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableFloat8Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableFloat8Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableFloat8Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableFloat8Vector 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, 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 93deacb..e5dfbce 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,336 +34,346 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableIntVector extends BaseNullableFixedWidthVector {
-   public 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);
-   }
-
-   /**
-    * 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;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  public 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);
+  }
+
+  /**
+   * 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;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableIntHolder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index position of element
+   * @return element at given index
+   */
+  public Integer getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getInt(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, NullableIntHolder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Integer 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, 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableIntHolder 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, IntHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableIntHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, NullableIntHolder)} 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 {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * Same as {@link #set(int, IntHolder)} 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){
-      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);
+    } 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, IntHolder 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, NullableIntHolder)} 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 {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * Same as {@link #set(int, IntHolder)} 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) {
+    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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableIntVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableIntVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableIntVector 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, NullableIntVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * Given a data buffer, get the value stored at a particular position
+   * in the vector.
+   * <p>
+   * 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);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableIntVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableIntVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableIntVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableIntVector 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, NullableIntVector.this);
+    }
+  }
 }
\ No newline at end of file
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
index 174b722..8d2fb56 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -36,394 +36,394 @@ import org.joda.time.Period;
  * 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;
-      }
+  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;
-      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){
+      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);
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
-   }
-
-   /**
-    * 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);
+    } 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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/NullableIntervalYearVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
index 303286d..ac121d2 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,354 +34,354 @@ import org.joda.time.Period;
  * (bit vector) is maintained to track which elements in the vector are null.
  */
 public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.INTERVALYEAR;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
-      return valueBuffer.getInt(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, NullableIntervalYearHolder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
-   }
-
-   /**
-    * 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 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);
-      }
-   }
-
-   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));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableIntervalYearHolder 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, IntervalYearHolder holder){
+  private static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.INTERVALYEAR;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableIntervalYearHolder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+  }
+
+  /**
+   * 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 interval = valueBuffer.getInt(index * TYPE_WIDTH);
+      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);
+    }
+  }
+
+  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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableIntervalYearHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableIntervalYearHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, IntervalYearHolder 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);
+    } 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, IntervalYearHolder 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, 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, NullableIntervalYearHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, IntervalYearHolder 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);
-   }
-
-   /**
-    * 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((NullableIntervalYearVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableIntervalYearVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableIntervalYearVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableIntervalYearVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableIntervalYearVector 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, NullableIntervalYearVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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((NullableIntervalYearVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableIntervalYearVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableIntervalYearVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableIntervalYearVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableIntervalYearVector 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, 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
index ed33718..5dd1492 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,363 +34,363 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
-   public 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");
-      }
+  public 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 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){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
-   }
-
-   /**
-    * 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);
+    } 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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/NullableTimeMicroVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
index d7ea322..25da4f6 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -36,335 +36,335 @@ import org.slf4j.Logger;
  * vector are null.
  */
 public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 8;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.TIMEMICRO;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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");
-      }
+  private static final byte TYPE_WIDTH = 8;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.TIMEMICRO;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableTimeMicroHolder 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 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, NullableTimeMicroHolder 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);
-      }
-   }
-
-   /**
-    * 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));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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, NullableTimeMicroHolder 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, TimeMicroHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableTimeMicroHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableTimeMicroHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, TimeMicroHolder 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);
+    } 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, TimeMicroHolder 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, 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, NullableTimeMicroHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, TimeMicroHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      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((NullableTimeMicroVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableTimeMicroVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableTimeMicroVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableTimeMicroVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableTimeMicroVector 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, NullableTimeMicroVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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((NullableTimeMicroVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableTimeMicroVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableTimeMicroVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableTimeMicroVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableTimeMicroVector 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, NullableTimeMicroVector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
index 07d8abb..7d2d5d1 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -36,336 +36,336 @@ import org.slf4j.Logger;
  * (bit vector) is maintained to track which elements in the vector are null.
  */
 public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.TIMEMILLI;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
-      return valueBuffer.getInt(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, NullableTimeMilliHolder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(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;
-      }
-      org.joda.time.LocalDateTime ldt = new org.joda.time.LocalDateTime(get(index),
-              org.joda.time.DateTimeZone.UTC);
-      return ldt;
-   }
-
-   /**
-    * 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));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableTimeMilliHolder 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, TimeMilliHolder holder){
+  private static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.TIMEMILLI;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableTimeMilliHolder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(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;
+    }
+    org.joda.time.LocalDateTime ldt = new org.joda.time.LocalDateTime(get(index),
+            org.joda.time.DateTimeZone.UTC);
+    return ldt;
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableTimeMilliHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableTimeMilliHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, TimeMilliHolder 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);
+    } 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, TimeMilliHolder 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, 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, NullableTimeMilliHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, TimeMilliHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /**
-    * 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((NullableTimeMilliVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableTimeMilliVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableTimeMilliVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableTimeMilliVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableTimeMilliVector 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, NullableTimeMilliVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /**
+   * 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((NullableTimeMilliVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableTimeMilliVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableTimeMilliVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableTimeMilliVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableTimeMilliVector 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, NullableTimeMilliVector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
index 947b239..e93a63f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,335 +34,335 @@ import org.apache.arrow.vector.util.TransferPair;
  * (bit vector) is maintained to track which elements in the vector are null.
  */
 public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 8;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.TIMENANO;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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");
-      }
+  private static final byte TYPE_WIDTH = 8;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.TIMENANO;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableTimeNanoHolder 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 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, NullableTimeNanoHolder 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);
-      }
-   }
-
-   /**
-    * 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));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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, NullableTimeNanoHolder 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, TimeNanoHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableTimeNanoHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableTimeNanoHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, TimeNanoHolder 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);
+    } 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, TimeNanoHolder 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, 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, NullableTimeNanoHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, TimeNanoHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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((NullableTimeNanoVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableTimeNanoVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableTimeNanoVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableTimeNanoVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableTimeNanoVector 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, NullableTimeNanoVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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((NullableTimeNanoVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableTimeNanoVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableTimeNanoVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableTimeNanoVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableTimeNanoVector 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, NullableTimeNanoVector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
index 0a3cfaa..8329fa7 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,336 +34,336 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.TIMESEC;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  private static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.TIMESEC;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableTimeSecHolder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Integer getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getInt(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, NullableTimeSecHolder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Integer 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, NullableTimeSecVector from) {
-      if (from.isSet(fromIndex) != 0) {
-         set(thisIndex, from.get(fromIndex));
-      }
-   }
-
-   /**
-    * 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);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableTimeSecHolder 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, TimeSecHolder holder){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableTimeSecHolder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableTimeSecHolder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, TimeSecHolder 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);
+    } 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, TimeSecHolder 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, 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, NullableTimeSecHolder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, TimeSecHolder 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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((NullableTimeSecVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableTimeSecVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableTimeSecVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableTimeSecVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableTimeSecVector 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, NullableTimeSecVector.this);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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((NullableTimeSecVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableTimeSecVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableTimeSecVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableTimeSecVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableTimeSecVector 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, 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
index 0d25eff..c7ad141 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,186 +34,184 @@ import org.apache.arrow.vector.util.TransferPair;
  * (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){
+  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 valueBuffer.getLong(index * TYPE_WIDTH);
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 41f7a11..5a34b19 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,187 +34,186 @@ import org.joda.time.LocalDateTime;
  * (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 {
+  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){
+      final long micros = valueBuffer.getLong(index * TYPE_WIDTH);
+      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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 6376b08..f9736ae 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,185 +34,183 @@ import org.apache.arrow.vector.util.TransferPair;
  * (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){
+  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 valueBuffer.getLong(index * TYPE_WIDTH);
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index ff581d2..fc38a6b 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,185 +34,184 @@ import org.joda.time.LocalDateTime;
  * (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){
+  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 = valueBuffer.getLong(index * TYPE_WIDTH);
+      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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 6c4445d..8eb4846 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,186 +34,184 @@ import org.apache.arrow.vector.util.TransferPair;
  * (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){
+  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 valueBuffer.getLong(index * TYPE_WIDTH);
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 096913e..6cc5c90 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,186 +34,185 @@ import org.joda.time.LocalDateTime;
  * (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){
+  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 = valueBuffer.getLong(index * TYPE_WIDTH);
+      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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 8e5ba72..0593ac7 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -33,185 +33,183 @@ import org.apache.arrow.vector.util.TransferPair;
  * (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){
+  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 valueBuffer.getLong(index * TYPE_WIDTH);
+    }
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index bc800e0..1afdb30 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,186 +34,185 @@ import org.joda.time.LocalDateTime;
  * 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){
+  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 = valueBuffer.getLong(index * TYPE_WIDTH);
+      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);
-   }
-
-   /**
-    * 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);
-   }
+    } 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
index 1bf2abc..d65cb83 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -29,189 +29,191 @@ import org.apache.arrow.vector.util.TransferPair;
  * 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);
+  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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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
index ccbfa32..106359d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,363 +34,363 @@ import org.apache.arrow.vector.util.TransferPair;
  * 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");
-      }
+  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 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){
+    }
+  }
+
+  /**
+   * 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));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  /**
+   * 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);
-   }
-
-   /**
-    * 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);
+    } 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);
-   }
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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);
-      }
-   }
+    }
+  }
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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/NullableUInt1Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
index 4d20f80..52f6c15 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -33,286 +33,286 @@ import org.apache.arrow.vector.util.TransferPair;
  * maintained to track which elements in the vector are null.
  */
 public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 1;
-   private final FieldReader reader;
-
-   public NullableUInt1Vector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT1.getType()),
-              allocator);
-   }
-
-   public NullableUInt1Vector(String name, FieldType fieldType, BufferAllocator allocator) {
-      super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new UInt1ReaderImpl(NullableUInt1Vector.this);
-   }
-
-   @Override
-   public FieldReader getReader(){
-      return reader;
-   }
-
-   @Override
-   public Types.MinorType getMinorType() {
-      return Types.MinorType.UINT1;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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");
-      }
+  private static final byte TYPE_WIDTH = 1;
+  private final FieldReader reader;
+
+  public NullableUInt1Vector(String name, BufferAllocator allocator) {
+    this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT1.getType()),
+            allocator);
+  }
+
+  public NullableUInt1Vector(String name, FieldType fieldType, BufferAllocator allocator) {
+    super(name, allocator, fieldType, TYPE_WIDTH);
+    reader = new UInt1ReaderImpl(NullableUInt1Vector.this);
+  }
+
+  @Override
+  public FieldReader getReader() {
+    return reader;
+  }
+
+  @Override
+  public Types.MinorType getMinorType() {
+    return Types.MinorType.UINT1;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableUInt1Holder 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 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, NullableUInt1Holder 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);
-      }
-   }
-
-   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, NullableUInt1Vector 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, NullableUInt1Holder 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, UInt1Holder holder){
+    }
+  }
+
+  public void copyFrom(int fromIndex, int thisIndex, NullableUInt1Vector from) {
+    if (from.isSet(fromIndex) != 0) {
+      set(thisIndex, from.get(fromIndex));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt1Vector 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, NullableUInt1Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableUInt1Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, UInt1Holder 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);
+    } 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, UInt1Holder 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, 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, NullableUInt1Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, UInt1Holder 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);
+  }
+
+  public void set(int index, int isSet, byte value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-   }
-
-   public void set(int index, int isSet, byte value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   public void setSafe(int index, int isSet, byte value) {
-      handleSafe(index);
-      set(index, isSet, value);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      vector transfer                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   @Override
-   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-      return new TransferImpl(ref, allocator);
-   }
-
-   @Override
-   public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableUInt1Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableUInt1Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableUInt1Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableUInt1Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableUInt1Vector 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, NullableUInt1Vector.this);
-      }
-   }
+    }
+  }
+
+  public void setSafe(int index, int isSet, byte value) {
+    handleSafe(index);
+    set(index, isSet, value);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      vector transfer                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  @Override
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+    return new TransferImpl(ref, allocator);
+  }
+
+  @Override
+  public TransferPair makeTransferPair(ValueVector to) {
+    return new TransferImpl((NullableUInt1Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableUInt1Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableUInt1Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableUInt1Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableUInt1Vector 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, NullableUInt1Vector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
index b140b9f..42ba6e1 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,286 +34,286 @@ import org.slf4j.Logger;
  * maintained to track which elements in the vector are null.
  */
 public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 2;
-   private final FieldReader reader;
-
-   public NullableUInt2Vector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT2.getType()),
-              allocator);
-   }
-
-   public NullableUInt2Vector(String name, FieldType fieldType, BufferAllocator allocator) {
-      super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new UInt2ReaderImpl(NullableUInt2Vector.this);
-   }
-
-   @Override
-   public FieldReader getReader(){
-      return reader;
-   }
-
-   @Override
-   public Types.MinorType getMinorType() {
-      return Types.MinorType.UINT2;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 char get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  private static final byte TYPE_WIDTH = 2;
+  private final FieldReader reader;
+
+  public NullableUInt2Vector(String name, BufferAllocator allocator) {
+    this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT2.getType()),
+            allocator);
+  }
+
+  public NullableUInt2Vector(String name, FieldType fieldType, BufferAllocator allocator) {
+    super(name, allocator, fieldType, TYPE_WIDTH);
+    reader = new UInt2ReaderImpl(NullableUInt2Vector.this);
+  }
+
+  @Override
+  public FieldReader getReader() {
+    return reader;
+  }
+
+  @Override
+  public Types.MinorType getMinorType() {
+    return Types.MinorType.UINT2;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 char get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getChar(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, NullableUInt2Holder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getChar(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Character getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getChar(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, NullableUInt2Holder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getChar(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Character getObject(int index) {
-      if (isSet(index) == 0) {
-         return null;
-      } else {
-         return get(index);
-      }
-   }
-
-   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, NullableUInt2Vector from) {
-      handleSafe(thisIndex);
-      copyFrom(fromIndex, thisIndex, from);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setChar(index * TYPE_WIDTH, value);
-   }
-
-   private void setValue(int index, char value) {
-      valueBuffer.setChar(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, 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.
-    *
-    * @param index   position of element
-    * @param holder  nullable data holder for value of element
-    */
-   public void set(int index, NullableUInt2Holder 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, UInt2Holder holder){
+    }
+  }
+
+  public void copyFrom(int fromIndex, int thisIndex, NullableUInt2Vector from) {
+    if (from.isSet(fromIndex) != 0) {
+      set(thisIndex, from.get(fromIndex));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt2Vector from) {
+    handleSafe(thisIndex);
+    copyFrom(fromIndex, thisIndex, from);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setChar(index * TYPE_WIDTH, value);
+  }
+
+  private void setValue(int index, char value) {
+    valueBuffer.setChar(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, 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.
+   *
+   * @param index   position of element
+   * @param holder  nullable data holder for value of element
+   */
+  public void set(int index, NullableUInt2Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableUInt2Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, UInt2Holder 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);
+    } 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, UInt2Holder 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, 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, NullableUInt2Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, UInt2Holder 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);
+  }
+
+  public void set(int index, int isSet, char value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-   }
-
-   public void set(int index, int isSet, char value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   public void setSafe(int index, int isSet, char value) {
-      handleSafe(index);
-      set(index, isSet, value);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      vector transfer                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   @Override
-   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-      return new TransferImpl(ref, allocator);
-   }
-
-   @Override
-   public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableUInt2Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableUInt2Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableUInt2Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableUInt2Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableUInt2Vector 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, NullableUInt2Vector.this);
-      }
-   }
+    }
+  }
+
+  public void setSafe(int index, int isSet, char value) {
+    handleSafe(index);
+    set(index, isSet, value);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      vector transfer                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  @Override
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+    return new TransferImpl(ref, allocator);
+  }
+
+  @Override
+  public TransferPair makeTransferPair(ValueVector to) {
+    return new TransferImpl((NullableUInt2Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableUInt2Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableUInt2Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableUInt2Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableUInt2Vector 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, NullableUInt2Vector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
index fc8cb63..1ec2149 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,258 +34,258 @@ import org.slf4j.Logger;
  * maintained to track which elements in the vector are null.
  */
 public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 4;
-   private final FieldReader reader;
-
-   public NullableUInt4Vector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT4.getType()),
-              allocator);
-   }
-
-   public NullableUInt4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
-      super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new UInt4ReaderImpl(NullableUInt4Vector.this);
-   }
-
-   @Override
-   public FieldReader getReader(){
-      return reader;
-   }
-
-   @Override
-   public Types.MinorType getMinorType() {
-      return Types.MinorType.UINT4;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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 int get(int index) throws IllegalStateException {
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
+  private static final byte TYPE_WIDTH = 4;
+  private final FieldReader reader;
+
+  public NullableUInt4Vector(String name, BufferAllocator allocator) {
+    this(name, FieldType.nullable(org.apache.arrow.vector.types.Types.MinorType.UINT4.getType()),
+            allocator);
+  }
+
+  public NullableUInt4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
+    super(name, allocator, fieldType, TYPE_WIDTH);
+    reader = new UInt4ReaderImpl(NullableUInt4Vector.this);
+  }
+
+  @Override
+  public FieldReader getReader() {
+    return reader;
+  }
+
+  @Override
+  public Types.MinorType getMinorType() {
+    return Types.MinorType.UINT4;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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 int get(int index) throws IllegalStateException {
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    return valueBuffer.getInt(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, NullableUInt4Holder holder) {
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
+  }
+
+  /**
+   * Same as {@link #get(int)}.
+   *
+   * @param index   position of element
+   * @return element at given index
+   */
+  public Integer getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
       return valueBuffer.getInt(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, NullableUInt4Holder holder){
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      holder.isSet = 1;
-      holder.value = valueBuffer.getInt(index * TYPE_WIDTH);
-   }
-
-   /**
-    * Same as {@link #get(int)}.
-    *
-    * @param index   position of element
-    * @return element at given index
-    */
-   public Integer getObject(int index) {
-      if (isSet(index) == 0) {
-         return null;
-      } else {
-         return get(index);
-      }
-   }
-
-   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, NullableUInt4Vector from) {
-      handleSafe(thisIndex);
-      copyFrom(fromIndex, thisIndex, from);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   private void setValue(int index, int value) {
-      valueBuffer.setInt(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 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, NullableUInt4Holder 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, UInt4Holder holder){
+    }
+  }
+
+  public void copyFrom(int fromIndex, int thisIndex, NullableUInt4Vector from) {
+    if (from.isSet(fromIndex) != 0) {
+      set(thisIndex, from.get(fromIndex));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt4Vector from) {
+    handleSafe(thisIndex);
+    copyFrom(fromIndex, thisIndex, from);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  private void setValue(int index, int value) {
+    valueBuffer.setInt(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 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, NullableUInt4Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableUInt4Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, UInt4Holder 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);
+    } 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, UInt4Holder 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, 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, NullableUInt4Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, UInt4Holder 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);
+  }
+
+  public void set(int index, int isSet, int value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-   }
-
-   public void set(int index, int isSet, int value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   public void setSafe(int index, int isSet, int value) {
-      handleSafe(index);
-      set(index, isSet, value);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      vector transfer                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   @Override
-   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-      return new TransferImpl(ref, allocator);
-   }
-
-   @Override
-   public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableUInt4Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableUInt4Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableUInt4Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableUInt4Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableUInt4Vector 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, NullableUInt4Vector.this);
-      }
-   }
+    }
+  }
+
+  public void setSafe(int index, int isSet, int value) {
+    handleSafe(index);
+    set(index, isSet, value);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      vector transfer                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  @Override
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+    return new TransferImpl(ref, allocator);
+  }
+
+  @Override
+  public TransferPair makeTransferPair(ValueVector to) {
+    return new TransferImpl((NullableUInt4Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableUInt4Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableUInt4Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableUInt4Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableUInt4Vector 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, NullableUInt4Vector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
index 2110cf0..408a810 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -34,258 +34,258 @@ import org.slf4j.Logger;
  * maintained to track which elements in the vector are null.
  */
 public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
-   private static final byte TYPE_WIDTH = 8;
-   private final FieldReader reader;
-
-   public NullableUInt8Vector(String name, BufferAllocator allocator) {
-      this(name, FieldType.nullable(Types.MinorType.UINT8.getType()),
-              allocator);
-   }
-
-   public NullableUInt8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
-      super(name, allocator, fieldType, TYPE_WIDTH);
-      reader = new UInt8ReaderImpl(NullableUInt8Vector.this);
-   }
-
-   @Override
-   public FieldReader getReader(){
-      return reader;
-   }
-
-   @Override
-   public Types.MinorType getMinorType() {
-      return Types.MinorType.UINT8;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          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");
-      }
+  private static final byte TYPE_WIDTH = 8;
+  private final FieldReader reader;
+
+  public NullableUInt8Vector(String name, BufferAllocator allocator) {
+    this(name, FieldType.nullable(Types.MinorType.UINT8.getType()),
+            allocator);
+  }
+
+  public NullableUInt8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
+    super(name, allocator, fieldType, TYPE_WIDTH);
+    reader = new UInt8ReaderImpl(NullableUInt8Vector.this);
+  }
+
+  @Override
+  public FieldReader getReader() {
+    return reader;
+  }
+
+  @Override
+  public Types.MinorType getMinorType() {
+    return Types.MinorType.UINT8;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          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, NullableUInt8Holder 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 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, NullableUInt8Holder 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);
-      }
-   }
-
-   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, NullableUInt8Vector 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, NullableUInt8Holder 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, UInt8Holder holder){
+    }
+  }
+
+  public void copyFrom(int fromIndex, int thisIndex, NullableUInt8Vector from) {
+    if (from.isSet(fromIndex) != 0) {
+      set(thisIndex, from.get(fromIndex));
+    } else {
+      BitVectorHelper.setValidityBit(validityBuffer, thisIndex, 0);
+    }
+  }
+
+  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt8Vector 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, NullableUInt8Holder holder) throws IllegalArgumentException {
+    if (holder.isSet < 0) {
+      throw new IllegalArgumentException();
+    } else if (holder.isSet > 0) {
       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, 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, NullableUInt8Holder holder) throws IllegalArgumentException {
-      handleSafe(index);
-      set(index, holder);
-   }
-
-   /**
-    * 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, UInt8Holder 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);
+    } 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, UInt8Holder 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, 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, NullableUInt8Holder holder) throws IllegalArgumentException {
+    handleSafe(index);
+    set(index, holder);
+  }
+
+  /**
+   * 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, UInt8Holder 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);
+  }
+
+  public void set(int index, int isSet, long value) {
+    if (isSet > 0) {
+      set(index, value);
+    } else {
       BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-   }
-
-   public void set(int index, int isSet, long value) {
-      if (isSet > 0) {
-         set(index, value);
-      } else {
-         BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-      }
-   }
-
-   public void setSafe(int index, int isSet, long value) {
-      handleSafe(index);
-      set(index, isSet, value);
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *                      vector transfer                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   @Override
-   public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-      return new TransferImpl(ref, allocator);
-   }
-
-   @Override
-   public TransferPair makeTransferPair(ValueVector to) {
-      return new TransferImpl((NullableUInt8Vector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableUInt8Vector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableUInt8Vector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableUInt8Vector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableUInt8Vector 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, NullableUInt8Vector.this);
-      }
-   }
+    }
+  }
+
+  public void setSafe(int index, int isSet, long value) {
+    handleSafe(index);
+    set(index, isSet, value);
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      vector transfer                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  @Override
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+    return new TransferImpl(ref, allocator);
+  }
+
+  @Override
+  public TransferPair makeTransferPair(ValueVector to) {
+    return new TransferImpl((NullableUInt8Vector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableUInt8Vector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableUInt8Vector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableUInt8Vector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableUInt8Vector 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, NullableUInt8Vector.this);
+    }
+  }
 }
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
index 330e067..1dfe891 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -36,479 +36,293 @@ import java.nio.ByteBuffer;
  * to track which elements in the vector are null.
  */
 public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
-   private final FieldReader reader;
-
-   /**
-    * 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);
-   }
-
-   /**
-    * 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 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.VARBINARY;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value getter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * Get the variable length element at specified index as byte array.
-    *
-    * @param index   position of element to get
-    * @return array of bytes for non-null element, null otherwise
-    */
-   public byte[] get(int index) {
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      final byte[] result = new byte[dataLength];
-      valueBuffer.getBytes(startOffset, result, 0, dataLength);
-      return result;
-   }
-
-   /**
-    * Get the variable length element at specified index as Text.
-    *
-    * @param index   position of element to get
-    * @return byte array for non-null element, null otherwise
-    */
-   public byte[] getObject(int index) {
-      byte[] b;
-      try {
-         b = get(index);
-      } catch (IllegalStateException e) {
-         return null;
-      }
-      return b;
-   }
-
-   /**
-    * Get the variable length element at specified index as Text.
-    *
-    * @param index   position of element to get
-    * @return greater than 0 length for non-null element, 0 otherwise
-    */
-   public int getValueLength(int index) {
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         return 0;
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      return dataLength;
-   }
-
-   /**
-    * Get the variable length element at specified index and sets the state
-    * in provided holder.
-    *
-    * @param index   position of element to get
-    * @param holder  data holder to be populated by this function
-    */
-   public void get(int index, NullableVarBinaryHolder holder){
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      holder.isSet = 1;
-      holder.start = startOffset;
-      holder.end = dataLength;
-      holder.buffer = valueBuffer;
-   }
-
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * 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));
-         lastSet = thisIndex;
-      }
-   }
-
-   /**
-    * 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));
-         lastSet = thisIndex;
-      }
-   }
-
-
-   /**
-    * Set the variable length element at the specified index to the supplied
-    * byte array. This is same as using {@link #set(int, byte[], int, int)}
-    * with start as 0 and length as value.length
-    *
-    * @param index   position of the element to set
-    * @param value   array of bytes to write
-    */
-   public void set(int index, byte[] value) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, 0, value.length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, byte[])} 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 value   array of bytes to write
-    */
-   public void setSafe(int index, byte[] value) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, value.length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, 0, value.length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the supplied
-    * byte array.
-    *
-    * @param index   position of the element to set
-    * @param value   array of bytes to write
-    * @param start   start index in array of bytes
-    * @param length  length of data in array of bytes
-    */
-   public void set(int index, byte[] value, int start, int length) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, byte[], int, int)} 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 value   array of bytes to write
-    * @param start   start index in array of bytes
-    * @param length  length of data in array of bytes
-    */
-   public void setSafe(int index, byte[] value, int start, int length) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the
-    * content in supplied ByteBuffer
-    *
-    * @param index   position of the element to set
-    * @param value   ByteBuffer with data
-    * @param start   start index in ByteBuffer
-    * @param length  length of data in ByteBuffer
-    */
-   public void set(int index, ByteBuffer value, int start, int length) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-      valueBuffer.setBytes(startOffset, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, ByteBuffer, int, int)} 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 value   ByteBuffer with data
-    * @param start   start index in ByteBuffer
-    * @param length  length of data in ByteBuffer
-    */
-   public void setSafe(int index, ByteBuffer value, int start, int length) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-      valueBuffer.setBytes(startOffset, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the data
-    * buffer supplied in the holder
-    *
-    * @param index   position of the element to set
-    * @param holder  holder that carries data buffer.
-    */
-   public void set(int index, VarBinaryHolder holder) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int dataLength = holder.end - holder.start;
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * 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, VarBinaryHolder holder) {
-      assert index >= 0;
-      final int dataLength = holder.end - holder.start;
-      fillEmpties(index);
-      handleSafe(index, dataLength);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the data
-    * buffer supplied in the holder
-    *
-    * @param index   position of the element to set
-    * @param holder  holder that carries data buffer.
-    */
-   public void set(int index, NullableVarBinaryHolder holder) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
-      final int dataLength = holder.end - holder.start;
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * 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, NullableVarBinaryHolder holder) {
-      assert index >= 0;
-      final int dataLength = holder.end - holder.start;
-      fillEmpties(index);
-      handleSafe(index, dataLength);
-      BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Sets the value length for an element.
-    *
-    * @param index   position of the element to set
-    * @param length  length of the element
-    */
-   public void setValueLengthSafe(int index, int length) {
-      assert index >= 0;
-      handleSafe(index, length);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-   }
-
-   /**
-    * Set the element at the given index to null.
-    *
-    * @param index   position of element
-    */
-   public void setNull(int index){
-      handleSafe(index, 0);
-      /* 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 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 + end);
-      final ArrowBuf bb = buffer.slice(start, end);
-      valueBuffer.setBytes(startOffset, bb);
-      lastSet = index;
-   }
-
-   /**
-    * 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, end);
-      set(index, isSet, start, end, 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((NullableVarBinaryVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableVarBinaryVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableVarBinaryVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableVarBinaryVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableVarBinaryVector 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, NullableVarBinaryVector.this);
-      }
-   }
+  private final FieldReader reader;
+
+  /**
+   * 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);
+  }
+
+  /**
+   * 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 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.VARBINARY;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value getter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * Get the variable length element at specified index as byte array.
+   *
+   * @param index   position of element to get
+   * @return array of bytes for non-null element, null otherwise
+   */
+  public byte[] get(int index) {
+    assert index >= 0;
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    final int startOffset = getstartOffset(index);
+    final int dataLength =
+            offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
+    final byte[] result = new byte[dataLength];
+    valueBuffer.getBytes(startOffset, result, 0, dataLength);
+    return result;
+  }
+
+  /**
+   * Get the variable length element at specified index as Text.
+   *
+   * @param index   position of element to get
+   * @return byte array for non-null element, null otherwise
+   */
+  public byte[] getObject(int index) {
+    byte[] b;
+    try {
+      b = get(index);
+    } catch (IllegalStateException e) {
+      return null;
+    }
+    return b;
+  }
+
+  /**
+   * Get the variable length element at specified index and sets the state
+   * in provided holder.
+   *
+   * @param index   position of element to get
+   * @param holder  data holder to be populated by this function
+   */
+  public void get(int index, NullableVarBinaryHolder holder) {
+    assert index >= 0;
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.start = getstartOffset(index);
+    holder.end = offsetBuffer.getInt((index + 1) * OFFSET_WIDTH);
+    holder.buffer = valueBuffer;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * 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) {
+    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+    final int length = end - start;
+    fillHoles(thisIndex);
+    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
+    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    lastSet = thisIndex;
+  }
+
+  /**
+   * 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) {
+    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+    final int length = end - start;
+    handleSafe(thisIndex, length);
+    fillHoles(thisIndex);
+    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
+    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    lastSet = thisIndex;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the data
+   * buffer supplied in the holder
+   *
+   * @param index   position of the element to set
+   * @param holder  holder that carries data buffer.
+   */
+  public void set(int index, VarBinaryHolder holder) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int dataLength = holder.end - holder.start;
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * 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, VarBinaryHolder holder) {
+    assert index >= 0;
+    final int dataLength = holder.end - holder.start;
+    fillEmpties(index);
+    handleSafe(index, dataLength);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the data
+   * buffer supplied in the holder
+   *
+   * @param index   position of the element to set
+   * @param holder  holder that carries data buffer.
+   */
+  public void set(int index, NullableVarBinaryHolder holder) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
+    final int dataLength = holder.end - holder.start;
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * 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, NullableVarBinaryHolder holder) {
+    assert index >= 0;
+    final int dataLength = holder.end - holder.start;
+    fillEmpties(index);
+    handleSafe(index, dataLength);
+    BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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((NullableVarBinaryVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableVarBinaryVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableVarBinaryVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableVarBinaryVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableVarBinaryVector 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, 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 0b03164..31e409c 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -37,481 +37,295 @@ import java.nio.ByteBuffer;
  * to track which elements in the vector are null.
  */
 public class NullableVarCharVector extends BaseNullableVariableWidthVector {
-   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);
-   }
-
-   /**
-    * 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;
-   }
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value getter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * Get the variable length element at specified index as byte array.
-    *
-    * @param index   position of element to get
-    * @return array of bytes for non-null element, null otherwise
-    */
-   public byte[] get(int index) {
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         throw new IllegalStateException("Value at index is null");
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      final byte[] result = new byte[dataLength];
-      valueBuffer.getBytes(startOffset, result, 0, dataLength);
-      return result;
-   }
-
-   /**
-    * 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
-    */
-   public Text getObject(int index) {
-      Text result = new Text();
-      byte[] b;
-      try {
-         b = get(index);
-      } catch (IllegalStateException e) {
-         return null;
-      }
-      result.set(b);
-      return result;
-   }
-
-   /**
-    * Get the variable length element at specified index as Text.
-    *
-    * @param index   position of element to get
-    * @return greater than 0 length for non-null element, 0 otherwise
-    */
-   public int getValueLength(int index) {
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         return 0;
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      return dataLength;
-   }
-
-   /**
-    * Get the variable length element at specified index and sets the state
-    * in provided holder.
-    *
-    * @param index   position of element to get
-    * @param holder  data holder to be populated by this function
-    */
-   public void get(int index, NullableVarCharHolder holder){
-      assert index >= 0;
-      if(isSet(index) == 0) {
-         holder.isSet = 0;
-         return;
-      }
-      final int startOffset = getstartOffset(index);
-      final int dataLength =
-              offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
-      holder.isSet = 1;
-      holder.start = startOffset;
-      holder.end = dataLength;
-      holder.buffer = valueBuffer;
-   }
-
-
-
-   /******************************************************************
-    *                                                                *
-    *          vector value setter methods                           *
-    *                                                                *
-    ******************************************************************/
-
-
-   /**
-    * 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) {
-         set(thisIndex, from.get(fromIndex));
-         lastSet = thisIndex;
-      }
-   }
-
-   /**
-    * 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) {
-         setSafe(thisIndex, from.get(fromIndex));
-         lastSet = thisIndex;
-      }
-   }
-
-
-   /**
-    * Set the variable length element at the specified index to the supplied
-    * byte array. This is same as using {@link #set(int, byte[], int, int)}
-    * with start as 0 and length as value.length
-    *
-    * @param index   position of the element to set
-    * @param value   array of bytes to write
-    */
-   public void set(int index, byte[] value) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, 0, value.length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, byte[])} 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 value   array of bytes to write
-    */
-   public void setSafe(int index, byte[] value) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, value.length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, 0, value.length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the supplied
-    * byte array.
-    *
-    * @param index   position of the element to set
-    * @param value   array of bytes to write
-    * @param start   start index in array of bytes
-    * @param length  length of data in array of bytes
-    */
-   public void set(int index, byte[] value, int start, int length) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, byte[], int, int)} 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 value   array of bytes to write
-    * @param start   start index in array of bytes
-    * @param length  length of data in array of bytes
-    */
-   public void setSafe(int index, byte[] value, int start, int length) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      setBytes(index, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the
-    * content in supplied ByteBuffer
-    *
-    * @param index   position of the element to set
-    * @param value   ByteBuffer with data
-    * @param start   start index in ByteBuffer
-    * @param length  length of data in ByteBuffer
-    */
-   public void set(int index, ByteBuffer value, int start, int length) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-      valueBuffer.setBytes(startOffset, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, ByteBuffer, int, int)} 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 value   ByteBuffer with data
-    * @param start   start index in ByteBuffer
-    * @param length  length of data in ByteBuffer
-    */
-   public void setSafe(int index, ByteBuffer value, int start, int length) {
-      assert index >= 0;
-      fillEmpties(index);
-      handleSafe(index, length);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-      valueBuffer.setBytes(startOffset, value, start, length);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the data
-    * buffer supplied in the holder
-    *
-    * @param index   position of the element to set
-    * @param holder  holder that carries data buffer.
-    */
-   public void set(int index, VarCharHolder holder) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int dataLength = holder.end - holder.start;
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, VarCharHolder)} 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) {
-      assert index >= 0;
-      final int dataLength = holder.end - holder.start;
-      fillEmpties(index);
-      handleSafe(index, dataLength);
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Set the variable length element at the specified index to the data
-    * buffer supplied in the holder
-    *
-    * @param index   position of the element to set
-    * @param holder  holder that carries data buffer.
-    */
-   public void set(int index, NullableVarCharHolder holder) {
-      assert index >= 0;
-      fillHoles(index);
-      BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
-      final int dataLength = holder.end - holder.start;
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Same as {@link #set(int, NullableVarCharHolder)} 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) {
-      assert index >= 0;
-      final int dataLength = holder.end - holder.start;
-      fillEmpties(index);
-      handleSafe(index, dataLength);
-      BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
-      valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
-      lastSet = index;
-   }
-
-   /**
-    * Sets the value length for an element.
-    *
-    * @param index   position of the element to set
-    * @param length  length of the element
-    */
-   public void setValueLengthSafe(int index, int length) {
-      assert index >= 0;
-      handleSafe(index, length);
-      final int startOffset = getstartOffset(index);
-      offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + length);
-   }
-
-   /**
-    * Set the element at the given index to null.
-    *
-    * @param index   position of element
-    */
-   public void setNull(int index){
-      handleSafe(index, 0);
-      /* 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 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 + end);
-      final ArrowBuf bb = buffer.slice(start, end);
-      valueBuffer.setBytes(startOffset, bb);
-      lastSet = index;
-   }
-
-   /**
-    * 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, end);
-      set(index, isSet, start, end, 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((NullableVarCharVector)to);
-   }
-
-   private class TransferImpl implements TransferPair {
-      NullableVarCharVector to;
-
-      public TransferImpl(String ref, BufferAllocator allocator){
-         to = new NullableVarCharVector(ref, field.getFieldType(), allocator);
-      }
-
-      public TransferImpl(NullableVarCharVector to){
-         this.to = to;
-      }
-
-      @Override
-      public NullableVarCharVector 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, NullableVarCharVector.this);
-      }
-   }
+  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);
+  }
+
+  /**
+   * 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;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value getter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * Get the variable length element at specified index as byte array.
+   *
+   * @param index   position of element to get
+   * @return array of bytes for non-null element, null otherwise
+   */
+  public byte[] get(int index) {
+    assert index >= 0;
+    if (isSet(index) == 0) {
+      throw new IllegalStateException("Value at index is null");
+    }
+    final int startOffset = getstartOffset(index);
+    final int dataLength =
+            offsetBuffer.getInt((index + 1) * OFFSET_WIDTH) - startOffset;
+    final byte[] result = new byte[dataLength];
+    valueBuffer.getBytes(startOffset, result, 0, dataLength);
+    return result;
+  }
+
+  /**
+   * 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
+   */
+  public Text getObject(int index) {
+    Text result = new Text();
+    byte[] b;
+    try {
+      b = get(index);
+    } catch (IllegalStateException e) {
+      return null;
+    }
+    result.set(b);
+    return result;
+  }
+
+  /**
+   * Get the variable length element at specified index and sets the state
+   * in provided holder.
+   *
+   * @param index   position of element to get
+   * @param holder  data holder to be populated by this function
+   */
+  public void get(int index, NullableVarCharHolder holder) {
+    assert index >= 0;
+    if (isSet(index) == 0) {
+      holder.isSet = 0;
+      return;
+    }
+    holder.isSet = 1;
+    holder.start = getstartOffset(index);
+    holder.end = offsetBuffer.getInt((index + 1) * OFFSET_WIDTH);
+    holder.buffer = valueBuffer;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
+
+  /**
+   * 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) {
+    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+    final int length = end - start;
+    fillHoles(thisIndex);
+    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
+    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    lastSet = thisIndex;
+  }
+
+  /**
+   * 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) {
+    final int start = from.offsetBuffer.getInt(fromIndex * OFFSET_WIDTH);
+    final int end = from.offsetBuffer.getInt((fromIndex + 1) * OFFSET_WIDTH);
+    final int length = end - start;
+    handleSafe(thisIndex, length);
+    fillHoles(thisIndex);
+    BitVectorHelper.setValidityBit(this.validityBuffer, thisIndex, from.isSet(fromIndex));
+    final int copyStart = offsetBuffer.getInt(thisIndex * OFFSET_WIDTH);
+    from.valueBuffer.getBytes(start, this.valueBuffer, copyStart, length);
+    offsetBuffer.setInt((thisIndex + 1) * OFFSET_WIDTH, copyStart + length);
+    lastSet = thisIndex;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the data
+   * buffer supplied in the holder
+   *
+   * @param index   position of the element to set
+   * @param holder  holder that carries data buffer.
+   */
+  public void set(int index, VarCharHolder holder) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int dataLength = holder.end - holder.start;
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * Same as {@link #set(int, VarCharHolder)} 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) {
+    assert index >= 0;
+    final int dataLength = holder.end - holder.start;
+    fillEmpties(index);
+    handleSafe(index, dataLength);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * Set the variable length element at the specified index to the data
+   * buffer supplied in the holder
+   *
+   * @param index   position of the element to set
+   * @param holder  holder that carries data buffer.
+   */
+  public void set(int index, NullableVarCharHolder holder) {
+    assert index >= 0;
+    fillHoles(index);
+    BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
+    final int dataLength = holder.end - holder.start;
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+  /**
+   * Same as {@link #set(int, NullableVarCharHolder)} 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) {
+    assert index >= 0;
+    final int dataLength = holder.end - holder.start;
+    fillEmpties(index);
+    handleSafe(index, dataLength);
+    BitVectorHelper.setValidityBit(validityBuffer, index, holder.isSet);
+    final int startOffset = getstartOffset(index);
+    offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, startOffset + dataLength);
+    valueBuffer.setBytes(startOffset, holder.buffer, holder.start, dataLength);
+    lastSet = index;
+  }
+
+
+  /******************************************************************
+   *                                                                *
+   *                      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((NullableVarCharVector) to);
+  }
+
+  private class TransferImpl implements TransferPair {
+    NullableVarCharVector to;
+
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new NullableVarCharVector(ref, field.getFieldType(), allocator);
+    }
+
+    public TransferImpl(NullableVarCharVector to) {
+      this.to = to;
+    }
+
+    @Override
+    public NullableVarCharVector 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, NullableVarCharVector.this);
+    }
+  }
 }
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 dc74afa..0d3acf1 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
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 288afc6..fd96773 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
@@ -22,13 +22,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import io.netty.buffer.ArrowBuf;
-import org.apache.arrow.vector.ValueVector.Accessor;
 import org.apache.arrow.vector.schema.ArrowFieldNode;
 import org.apache.arrow.vector.schema.ArrowRecordBatch;
 import org.apache.arrow.vector.schema.ArrowVectorType;
 
-import javax.annotation.Nullable;
-
 public class VectorUnloader {
 
   private final VectorSchemaRoot root;
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 3eab30a..4648d07 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -74,8 +74,7 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
       e.printStackTrace();
       clear();
       return false;
-    }
-    finally {
+    } finally {
       if (!dataAlloc) {
         clear();
       }
@@ -84,7 +83,7 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
   }
 
   protected void allocateOffsetBuffer(final long size) {
-    final int curSize = (int)size;
+    final int curSize = (int) size;
     offsetBuffer = allocator.buffer(curSize);
     offsetBuffer.readerIndex(0);
     offsetAllocationSizeInBytes = curSize;
@@ -101,8 +100,8 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
     final int currentBufferCapacity = offsetBuffer.capacity();
     long baseSize = offsetAllocationSizeInBytes;
 
-    if (baseSize < (long)currentBufferCapacity) {
-      baseSize = (long)currentBufferCapacity;
+    if (baseSize < (long) currentBufferCapacity) {
+      baseSize = (long) currentBufferCapacity;
     }
 
     long newAllocationSize = baseSize * 2L;
@@ -112,13 +111,13 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
       throw new OversizedAllocationException("Unable to expand the buffer");
     }
 
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    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;
+    offsetAllocationSizeInBytes = (int) newAllocationSize;
   }
 
   @Override
@@ -147,8 +146,8 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
     return Math.min(vector.getValueCapacity(), offsetValueCapacity);
   }
 
-  private int getOffsetBufferValueCapacity() {
-    return (int)((offsetBuffer.capacity() * 1.0)/OFFSET_WIDTH);
+  protected int getOffsetBufferValueCapacity() {
+    return (int) ((offsetBuffer.capacity() * 1.0) / OFFSET_WIDTH);
   }
 
   @Override
@@ -183,8 +182,13 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer},
-            vector.getBuffers(false), ArrowBuf.class);
+    final ArrowBuf[] buffers;
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer}, vector.getBuffers(false),
+              ArrowBuf.class);
+    }
     if (clear) {
       for (ArrowBuf buffer : buffers) {
         buffer.retain();
@@ -208,15 +212,15 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
       // 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)) {
+              // 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());
+              fieldType.getType().getTypeID(), vector.getField().getType().getTypeID());
       throw new SchemaChangeRuntimeException(msg);
     }
 
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 bb091f6..6713b1c 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
@@ -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
@@ -7,15 +6,15 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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;
 
@@ -125,7 +124,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     ArrowBuf bitBuffer = ownBuffers.get(0);
 
     validityBuffer.release();
-    validityBuffer = bitBuffer.retain(allocator);
+    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
     valueCount = fieldNode.getLength();
 
     validityAllocationSizeInBytes = validityBuffer.capacity();
@@ -134,15 +133,17 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
   @Override
   public List<ArrowBuf> getFieldBuffers() {
     List<ArrowBuf> result = new ArrayList<>(1);
-
-    validityBuffer.readerIndex(0);
-    validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
-
+    setReaderAndWriterIndex();
     result.add(validityBuffer);
 
     return result;
   }
 
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+  }
+
   @Override
   @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
@@ -182,6 +183,8 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
      */
     boolean success = false;
     try {
+      /* we are doing a new allocation -- release the current buffers */
+      clear();
       /* allocate validity buffer */
       allocateValidityBuffer(validityAllocationSizeInBytes);
       success = vector.allocateNewSafe();
@@ -196,7 +199,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
   }
 
   private void allocateValidityBuffer(final long size) {
-    final int curSize = (int)size;
+    final int curSize = (int) size;
     validityBuffer = allocator.buffer(curSize);
     validityBuffer.readerIndex(0);
     validityAllocationSizeInBytes = curSize;
@@ -213,8 +216,8 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     final int currentBufferCapacity = validityBuffer.capacity();
     long baseSize = validityAllocationSizeInBytes;
 
-    if (baseSize < (long)currentBufferCapacity) {
-      baseSize = (long)currentBufferCapacity;
+    if (baseSize < (long) currentBufferCapacity) {
+      baseSize = (long) currentBufferCapacity;
     }
 
     long newAllocationSize = baseSize * 2L;
@@ -224,12 +227,12 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
       throw new OversizedAllocationException("Unable to expand the buffer");
     }
 
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    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;
+    validityAllocationSizeInBytes = (int) newAllocationSize;
   }
 
   public FieldVector getDataVector() {
@@ -282,7 +285,14 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
 
   @Override
   public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, vector.getBuffers(false), ArrowBuf.class);
+    setReaderAndWriterIndex();
+    final ArrowBuf[] buffers;
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, vector.getBuffers(false),
+              ArrowBuf.class);
+    }
     if (clear) {
       for (ArrowBuf buffer : buffers) {
         buffer.retain();
@@ -311,7 +321,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
     // returned vector must have the same field
     if (!Objects.equals(vector.getField().getType(), type.getType())) {
       final String msg = String.format("Inner vector type mismatch. Requested type: [%s], actual type: [%s]",
-          type.getType(), vector.getField().getType());
+              type.getType(), vector.getField().getType());
       throw new SchemaChangeRuntimeException(msg);
     }
 
@@ -400,7 +410,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
   }
 
   private int getValidityBufferValueCapacity() {
-    return (int)(validityBuffer.capacity() * 8L);
+    return (int) (validityBuffer.capacity() * 8L);
   }
 
   public void setNull(int index) {
@@ -462,6 +472,7 @@ public class FixedSizeListVector extends BaseValueVector implements FieldVector,
       dataPair.transfer();
       to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer;
       to.setValueCount(valueCount);
+      clear();
     }
 
     @Override
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
deleted file mode 100644
index 435d3d8..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyBaseRepeatedValueVector.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.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
deleted file mode 100644
index 0843907..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyFixedSizeListVector.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*******************************************************************************
-
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.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
deleted file mode 100644
index 732699a..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyListVector.java
+++ /dev/null
@@ -1,288 +0,0 @@
-/*******************************************************************************
-
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-
-package org.apache.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
deleted file mode 100644
index 77556ab..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/LegacyMapVector.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.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 ea57720..afe86a6 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
@@ -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
@@ -7,15 +6,15 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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;
 
@@ -60,7 +59,7 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return new ListVector(name, allocator, FieldType.nullable(ArrowType.List.INSTANCE), null);
   }
 
-  private ArrowBuf validityBuffer;
+  protected ArrowBuf validityBuffer;
   private UnionListReader reader;
   private CallBack callBack;
   private final FieldType fieldType;
@@ -108,6 +107,13 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return singletonList(getDataVector());
   }
 
+  /**
+   * 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) {
@@ -129,20 +135,35 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     valueCount = fieldNode.getLength();
   }
 
+  /**
+   * Get the buffers belonging to this vector
+   * @return the inner buffers.
+   */
   @Override
   public List<ArrowBuf> getFieldBuffers() {
     List<ArrowBuf> result = new ArrayList<>(2);
-    validityBuffer.readerIndex(0);
-    validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
-    offsetBuffer.readerIndex(0);
-    offsetBuffer.writerIndex((valueCount + 1) * OFFSET_WIDTH);
-
+    setReaderAndWriterIndex();
     result.add(validityBuffer);
     result.add(offsetBuffer);
 
     return result;
   }
 
+  /**
+   * Set the reader and writer indexes for the inner buffers.
+   */
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    offsetBuffer.readerIndex(0);
+    if (valueCount == 0) {
+      validityBuffer.writerIndex(0);
+      offsetBuffer.writerIndex(0);
+    } else {
+      validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
+      offsetBuffer.writerIndex((valueCount + 1) * OFFSET_WIDTH);
+    }
+  }
+
   @Override
   @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
@@ -153,16 +174,28 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return new UnionListWriter(this);
   }
 
+  /**
+   * Same as {@link #allocateNewSafe()}.
+   */
   @Override
   public void allocateNew() throws OutOfMemoryException {
-   if (!allocateNewSafe()) {
-     throw new OutOfMemoryException("Failure while allocating memory");
-   }
+    if (!allocateNewSafe()) {
+      throw new OutOfMemoryException("Failure while allocating memory");
+    }
   }
 
+  /**
+   * 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.
+   *
+   * @return false if memory allocation fails, true otherwise.
+   */
   public boolean allocateNewSafe() {
     boolean success = false;
     try {
+      /* we are doing a new allocation -- release the current buffers */
+      clear();
       /* allocate validity buffer */
       allocateValidityBuffer(validityAllocationSizeInBytes);
       /* allocate offset and data buffer */
@@ -177,13 +210,17 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   private void allocateValidityBuffer(final long size) {
-    final int curSize = (int)size;
+    final int curSize = (int) size;
     validityBuffer = allocator.buffer(curSize);
     validityBuffer.readerIndex(0);
     validityAllocationSizeInBytes = curSize;
     validityBuffer.setZero(0, validityBuffer.capacity());
   }
 
+  /**
+   * Resize the vector to increase the capacity. The internal behavior is to
+   * double the current value capacity.
+   */
   @Override
   public void reAlloc() {
     /* reallocate the validity buffer */
@@ -201,8 +238,8 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     final int currentBufferCapacity = validityBuffer.capacity();
     long baseSize = validityAllocationSizeInBytes;
 
-    if (baseSize < (long)currentBufferCapacity) {
-      baseSize = (long)currentBufferCapacity;
+    if (baseSize < (long) currentBufferCapacity) {
+      baseSize = (long) currentBufferCapacity;
     }
 
     long newAllocationSize = baseSize * 2L;
@@ -212,19 +249,34 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
       throw new OversizedAllocationException("Unable to expand the buffer");
     }
 
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    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;
+    validityAllocationSizeInBytes = (int) newAllocationSize;
   }
 
+  /**
+   * Same as {@link #copyFrom(int, int, ListVector)} except that
+   * it handles the case when the capacity of the vector needs to be expanded
+   * before copy.
+   * @param inIndex position to copy from in source vector
+   * @param outIndex position to copy to in this vector
+   * @param from source vector
+   */
   public void copyFromSafe(int inIndex, int outIndex, ListVector from) {
     copyFrom(inIndex, outIndex, from);
   }
 
+  /**
+   * Copy a cell value from a particular index in source vector to a particular
+   * position in this vector
+   * @param inIndex position to copy from in source vector
+   * @param outIndex position to copy to in this vector
+   * @param from source vector
+   */
   public void copyFrom(int inIndex, int outIndex, ListVector from) {
     FieldReader in = from.getReader();
     in.setPosition(inIndex);
@@ -233,6 +285,10 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     ComplexCopier.copy(in, out);
   }
 
+  /**
+   * Get the inner data vector for this list vector
+   * @return data vector
+   */
   @Override
   public FieldVector getDataVector() {
     return vector;
@@ -269,7 +325,9 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   @Override
-  public ArrowBuf getValidityBuffer() { return validityBuffer; }
+  public ArrowBuf getValidityBuffer() {
+    return validityBuffer;
+  }
 
   @Override
   public ArrowBuf getDataBuffer() {
@@ -277,7 +335,9 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   @Override
-  public ArrowBuf getOffsetBuffer() { return offsetBuffer; }
+  public ArrowBuf getOffsetBuffer() {
+    return offsetBuffer;
+  }
 
   private class TransferImpl implements TransferPair {
 
@@ -297,15 +357,30 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
       dataTransferPair = getDataVector().makeTransferPair(to.getDataVector());
     }
 
+    /**
+     * 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.
+     */
     @Override
     public void transfer() {
+      to.clear();
       dataTransferPair.transfer();
       to.validityBuffer = validityBuffer.transferOwnership(to.allocator).buffer;
       to.offsetBuffer = offsetBuffer.transferOwnership(to.allocator).buffer;
       to.lastSet = lastSet;
-      to.setValueCount(valueCount);
+      if (valueCount > 0) {
+        to.setValueCount(valueCount);
+      }
+      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.
+     */
     @Override
     public void splitAndTransfer(int startIndex, int length) {
       final int startPoint = offsetBuffer.getInt(startIndex * OFFSET_WIDTH);
@@ -343,8 +418,7 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
           }
           target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
           target.validityBuffer.retain(1);
-        }
-        else {
+        } else {
           /* Copy data
            * When the first bit starts from the middle of a byte (offset != 0),
            * copy data from src BitVector.
@@ -369,15 +443,14 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
            * (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) {
+          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 {
+          } else {
             byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
                     firstByteSource + byteSizeTarget - 1, offset);
             target.validityBuffer.setByte(byteSizeTarget - 1, b1);
@@ -408,9 +481,14 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return result;
   }
 
+  /**
+   * Get the size (number of bytes) of underlying buffers used by this
+   * vector
+   * @return size of underlying buffers.
+   */
   @Override
   public int getBufferSize() {
-    if (getValueCount() == 0) {
+    if (valueCount == 0) {
       return 0;
     }
     final int offsetBufferSize = (valueCount + 1) * OFFSET_WIDTH;
@@ -435,10 +513,28 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     lastSet = 0;
   }
 
+  /**
+   * 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 = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer}, ObjectArrays.concat(new ArrowBuf[] {validityBuffer},
-        vector.getBuffers(false), ArrowBuf.class), ArrowBuf.class);
+    setReaderAndWriterIndex();
+    final ArrowBuf[] buffers;
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = ObjectArrays.concat(new ArrowBuf[]{offsetBuffer},
+              ObjectArrays.concat(new ArrowBuf[]{validityBuffer},
+                      vector.getBuffers(false), ArrowBuf.class), ArrowBuf.class);
+    }
     if (clear) {
       for (ArrowBuf buffer : buffers) {
         buffer.retain();
@@ -459,6 +555,11 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return vector;
   }
 
+  /**
+   * Get the element in the list vector at a particular index
+   * @param index position of the element
+   * @return Object at given position
+   */
   @Override
   public Object getObject(int index) {
     if (isSet(index) == 0) {
@@ -475,11 +576,23 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return vals;
   }
 
+  /**
+   * Check if element at given index is null.
+   *
+   * @param index  position of element
+   * @return true if element at given index is null, false otherwise
+   */
   @Override
   public boolean isNull(int index) {
     return (isSet(index) == 0);
   }
 
+  /**
+   * Same as {@link #isNull(int)}.
+   *
+   * @param index  position of element
+   * @return 1 if element at given index is not null, 0 otherwise
+   */
   public int isSet(int index) {
     final int byteIndex = index >> 3;
     final byte b = validityBuffer.getByte(byteIndex);
@@ -487,29 +600,57 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     return Long.bitCount(b & (1L << bitIndex));
   }
 
+  /**
+   * Get the number of elements that are null in the vector
+   *
+   * @return the number of null elements.
+   */
   @Override
   public int getNullCount() {
     return BitVectorHelper.getNullCount(validityBuffer, valueCount);
   }
 
+  /**
+   * Get the current value capacity for the vector
+   * @return number of elements that vector can hold.
+   */
+  @Override
+  public int getValueCapacity() {
+    return Math.min(getValidityBufferValueCapacity(), super.getValueCapacity());
+  }
+
+  private int getValidityAndOffsetValueCapacity() {
+    final int offsetValueCapacity = Math.max(getOffsetBufferValueCapacity() - 1, 0);
+    return Math.min(offsetValueCapacity, getValidityBufferValueCapacity());
+  }
+
+  private int getValidityBufferValueCapacity() {
+    return (int) (validityBuffer.capacity() * 8L);
+  }
+
   public void setNotNull(int index) {
-    if (index >= getValueCapacity()) {
+    while (index >= getValidityAndOffsetValueCapacity()) {
       reallocValidityAndOffsetBuffers();
     }
     BitVectorHelper.setValidityBitToOne(validityBuffer, index);
     lastSet = index + 1;
   }
 
+  /**
+   * Start a new value in the list vector
+   *
+   * @param index index of the value to start
+   */
   @Override
   public int startNewValue(int index) {
-    if (index >= getValueCapacity()) {
+    while (index >= getValidityAndOffsetValueCapacity()) {
       reallocValidityAndOffsetBuffers();
     }
     for (int i = lastSet; i <= index; i++) {
       final int currentOffset = offsetBuffer.getInt(i * OFFSET_WIDTH);
       offsetBuffer.setInt((i + 1) * OFFSET_WIDTH, currentOffset);
     }
-    setNotNull(index);
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
     lastSet = index + 1;
     return offsetBuffer.getInt(lastSet * OFFSET_WIDTH);
   }
@@ -525,12 +666,17 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     offsetBuffer.setInt((index + 1) * OFFSET_WIDTH, currentOffset + size);
   }
 
+  /**
+   * Sets the value count for the vector
+   *
+   * @param valueCount   value count
+   */
   @Override
   public void setValueCount(int valueCount) {
     this.valueCount = valueCount;
     if (valueCount > 0) {
-      while (valueCount > getValueCapacity()) {
-        /* realloc the inner buffers if needed */
+      while (valueCount > getValidityAndOffsetValueCapacity()) {
+        /* check if validity and offset buffers need to be re-allocated */
         reallocValidityAndOffsetBuffers();
       }
       for (int i = lastSet; i < valueCount; i++) {
@@ -542,6 +688,9 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
     /* valueCount for the data vector is the current end offset */
     final int childValueCount = (valueCount == 0) ? 0 :
             offsetBuffer.getInt(valueCount * OFFSET_WIDTH);
+    /* set the value count of data vector and this will take care of
+     * checking whether data buffer needs to be reallocated.
+     */
     vector.setValueCount(childValueCount);
   }
 
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 a1478c5..f95302f 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
@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
  * 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.
@@ -54,7 +54,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
   private final NullableMapReaderImpl reader = new NullableMapReaderImpl(this);
   private final NullableMapWriter writer = new NullableMapWriter(this);
 
-  private ArrowBuf validityBuffer;
+  protected ArrowBuf validityBuffer;
   private int validityAllocationSizeInBytes;
 
   // deprecated, use FieldType or static constructor instead
@@ -91,7 +91,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
     ArrowBuf bitBuffer = ownBuffers.get(0);
 
     validityBuffer.release();
-    validityBuffer = bitBuffer.retain(allocator);
+    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
     valueCount = fieldNode.getLength();
     validityAllocationSizeInBytes = validityBuffer.capacity();
   }
@@ -99,14 +99,17 @@ public class NullableMapVector extends MapVector implements FieldVector {
   @Override
   public List<ArrowBuf> getFieldBuffers() {
     List<ArrowBuf> result = new ArrayList<>(1);
-
-    validityBuffer.readerIndex(0);
-    validityBuffer.writerIndex(BitVectorHelper.getValidityBufferSize(valueCount));
+    setReaderAndWriterIndex();
     result.add(validityBuffer);
 
     return result;
   }
 
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(BitVectorHelper.getValidityBufferSize(valueCount));
+  }
+
   @Override
   @Deprecated
   public List<BufferBacked> getFieldInnerVectors() {
@@ -156,6 +159,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
       target.clear();
       target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
       super.transfer();
+      clear();
     }
 
     @Override
@@ -193,8 +197,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
         }
         target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
         target.validityBuffer.retain(1);
-      }
-      else {
+      } else {
         /* Copy data
          * When the first bit starts from the middle of a byte (offset != 0),
          * copy data from src BitVector.
@@ -219,15 +222,14 @@ public class NullableMapVector extends MapVector implements FieldVector {
          * (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) {
+        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 {
+        } else {
           byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
                   firstByteSource + byteSizeTarget - 1, offset);
           target.validityBuffer.setByte(byteSizeTarget - 1, b1);
@@ -236,55 +238,108 @@ public class NullableMapVector extends MapVector implements FieldVector {
     }
   }
 
+  /**
+   * Get the value capacity of the internal validity buffer.
+   * @return number of elements that validity buffer can hold
+   */
   private int getValidityBufferValueCapacity() {
-    return (int)(validityBuffer.capacity() * 8L);
+    return (int) (validityBuffer.capacity() * 8L);
   }
 
+  /**
+   * Get the current value capacity for the vector
+   * @return number of elements that vector can hold.
+   */
   @Override
   public int getValueCapacity() {
     return Math.min(getValidityBufferValueCapacity(),
             super.getValueCapacity());
   }
 
+  /**
+   * 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) {
+    setReaderAndWriterIndex();
+    final ArrowBuf[] buffers;
+    if (getBufferSize() == 0) {
+      buffers = new ArrowBuf[0];
+    } else {
+      buffers = ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, super.getBuffers(false),
+              ArrowBuf.class);
+    }
     if (clear) {
-      validityBuffer.retain(1);
+      for (ArrowBuf buffer : buffers) {
+        buffer.retain();
+      }
+      clear();
     }
-    return ObjectArrays.concat(new ArrowBuf[]{validityBuffer}, super.getBuffers(clear), ArrowBuf.class);
+
+    return buffers;
   }
 
+  /**
+   * Close the vector and release the associated buffers.
+   */
   @Override
   public void close() {
     clearValidityBuffer();
     super.close();
   }
 
+  /**
+   * Same as {@link #close()}
+   */
   @Override
   public void clear() {
     clearValidityBuffer();
     super.clear();
   }
 
+  /**
+   * Release the validity buffer
+   */
   private void clearValidityBuffer() {
     validityBuffer.release();
     validityBuffer = allocator.getEmpty();
   }
 
+  /**
+   * 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; }
+    if (valueCount == 0) {
+      return 0;
+    }
     return super.getBufferSize() +
             BitVectorHelper.getValidityBufferSize(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;
     }
     return super.getBufferSizeFor(valueCount)
-        + BitVectorHelper.getValidityBufferSize(valueCount);
+            + BitVectorHelper.getValidityBufferSize(valueCount);
   }
 
   @Override
@@ -302,7 +357,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
      */
     boolean success = false;
     try {
-      clearValidityBuffer();
+      clear();
       allocateValidityBuffer(validityAllocationSizeInBytes);
       success = super.allocateNewSafe();
     } finally {
@@ -315,7 +370,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
   }
 
   private void allocateValidityBuffer(final long size) {
-    final int curSize = (int)size;
+    final int curSize = (int) size;
     validityBuffer = allocator.buffer(curSize);
     validityBuffer.readerIndex(0);
     validityAllocationSizeInBytes = curSize;
@@ -333,8 +388,8 @@ public class NullableMapVector extends MapVector implements FieldVector {
     final int currentBufferCapacity = validityBuffer.capacity();
     long baseSize = validityAllocationSizeInBytes;
 
-    if (baseSize < (long)currentBufferCapacity) {
-      baseSize = (long)currentBufferCapacity;
+    if (baseSize < (long) currentBufferCapacity) {
+      baseSize = (long) currentBufferCapacity;
     }
 
     long newAllocationSize = baseSize * 2L;
@@ -344,12 +399,12 @@ public class NullableMapVector extends MapVector implements FieldVector {
       throw new OversizedAllocationException("Unable to expand the buffer");
     }
 
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
+    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;
+    validityAllocationSizeInBytes = (int) newAllocationSize;
   }
 
   @Override
@@ -431,7 +486,7 @@ public class NullableMapVector extends MapVector implements FieldVector {
   @Override
   public void setValueCount(int valueCount) {
     assert valueCount >= 0;
-    while (valueCount > getValueCapacity()) {
+    while (valueCount > getValidityBufferValueCapacity()) {
       /* realloc the inner buffers if needed */
       reallocValidityBuffer();
     }
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 d3c8b99..8017b38 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
@@ -27,7 +27,8 @@ 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;
@@ -192,7 +193,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
   private abstract class BufferReader {
     abstract protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException;
 
-    final ArrowBuf readBuffer(BufferAllocator allocator, int count) throws IOException {
+    ArrowBuf readBuffer(BufferAllocator allocator, int count) throws IOException {
       readToken(START_ARRAY);
       ArrowBuf buf = read(allocator, count);
       readToken(END_ARRAY);
@@ -201,8 +202,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
   }
 
   private class BufferHelper {
-     BufferReader BIT = new BufferReader() {
-
+    BufferReader BIT = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
         final int bufferSize = BitVectorHelper.getValidityBufferSize(count);
@@ -216,6 +216,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
           BitVectorHelper.setValidityBit(buf, i, parser.readValueAs(Boolean.class) ? 1 : 0);
         }
 
+        buf.writerIndex(bufferSize);
         return buf;
       }
     };
@@ -223,7 +224,8 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT1 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableTinyIntVector.TYPE_WIDTH);
+        final int size = count * NullableTinyIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -237,7 +239,8 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT2 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableSmallIntVector.TYPE_WIDTH);
+        final int size = count * NullableSmallIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -249,10 +252,10 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     };
 
     BufferReader INT4 = new BufferReader() {
-
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableIntVector.TYPE_WIDTH);
+        final int size = count * NullableIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -264,10 +267,10 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     };
 
     BufferReader INT8 = new BufferReader() {
-
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableBigIntVector.TYPE_WIDTH);
+        final int size = count * NullableBigIntVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -279,10 +282,10 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     };
 
     BufferReader FLOAT4 = new BufferReader() {
-
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableFloat4Vector.TYPE_WIDTH);
+        final int size = count * NullableFloat4Vector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -294,10 +297,10 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     };
 
     BufferReader FLOAT8 = new BufferReader() {
-
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableFloat8Vector.TYPE_WIDTH);
+        final int size = count * NullableFloat8Vector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
@@ -309,17 +312,18 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     };
 
     BufferReader DECIMAL = new BufferReader() {
-
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        ArrowBuf buf = allocator.buffer(count * NullableDecimalVector.TYPE_WIDTH);
+        final int size = count * NullableDecimalVector.TYPE_WIDTH;
+        ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
           parser.nextToken();
-          final byte[] value = decodeHexSafe(parser.getValueAsString());
-          DecimalUtility.writeByteArrayToArrowBuf(value, buf, i);
+          BigDecimal decimalValue = new BigDecimal(parser.readValueAs(String.class));
+          DecimalUtility.writeBigDecimalToArrowBuf(decimalValue, buf, i);
         }
 
+        buf.writerIndex(size);
         return buf;
       }
     };
@@ -369,10 +373,10 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
         return buf;
       }
     };
-
   }
 
-  private ArrowBuf readBuffer(BufferAllocator allocator, ArrowVectorType bufferType, Types.MinorType type, int count) throws IOException {
+  private ArrowBuf readIntoBuffer(BufferAllocator allocator, ArrowVectorType bufferType,
+                                  Types.MinorType type, int count) throws IOException {
     ArrowBuf buf;
 
     BufferHelper helper = new BufferHelper();
@@ -461,6 +465,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     }
 
     buf = reader.readBuffer(allocator, count);
+
     assert buf != null;
     return buf;
   }
@@ -506,12 +511,14 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
           innerBufferValueCount = valueCount + 1;
         }
 
-        vectorBuffers[v] = readBuffer(allocator, vectorType, vector.getMinorType(), innerBufferValueCount);
+        vectorBuffers[v] = readIntoBuffer(allocator, vectorType, vector.getMinorType(), innerBufferValueCount);
       }
 
-      vector.loadFieldBuffers(new ArrowFieldNode(valueCount, 0), Arrays.asList(vectorBuffers));
+      final int nullCount = BitVectorHelper.getNullCount(vectorBuffers[0], valueCount);
+      final ArrowFieldNode fieldNode = new ArrowFieldNode(valueCount, nullCount);
+      vector.loadFieldBuffers(fieldNode, Arrays.asList(vectorBuffers));
 
-      // read child vectors, if any
+      /* read child vectors (if any) */
       List<Field> fields = field.getChildren();
       if (!fields.isEmpty()) {
         List<FieldVector> vectorChildren = vector.getChildrenFromFields();
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 7921cd3..0c8507b 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
@@ -22,6 +22,7 @@ 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;
@@ -159,6 +160,8 @@ public class JsonFileWriter implements AutoCloseable {
       generator.writeObjectField("name", field.getName());
       int valueCount = vector.getValueCount();
       generator.writeObjectField("count", valueCount);
+      final int scale = (vector instanceof NullableDecimalVector) ?
+                            ((NullableDecimalVector) vector).getScale() : 0;
       for (int v = 0; v < vectorTypes.size(); v++) {
         ArrowVectorType vectorType = vectorTypes.get(v);
         ArrowBuf vectorBuffer = vectorBuffers.get(v);
@@ -167,9 +170,9 @@ public class JsonFileWriter implements AutoCloseable {
         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);
+            writeValueToGenerator(vectorType, vectorBuffer, vectorBuffers.get(v-1), vector, i, scale);
           } else {
-            writeValueToGenerator(vectorType, vectorBuffer, null, vector, i);
+            writeValueToGenerator(vectorType, vectorBuffer, null, vector, i, scale);
           }
         }
         generator.writeEndArray();
@@ -193,7 +196,8 @@ public class JsonFileWriter implements AutoCloseable {
   }
 
   private void writeValueToGenerator(ArrowVectorType bufferType, ArrowBuf buffer,
-                                     ArrowBuf offsetBuffer, FieldVector vector, int index) throws IOException {
+                                     ArrowBuf offsetBuffer, FieldVector vector,
+                                     final int index, final int scale) throws IOException {
     if (bufferType.equals(TYPE)) {
       generator.writeNumber(buffer.getByte(index * NullableTinyIntVector.TYPE_WIDTH));
     } else if (bufferType.equals(OFFSET)) {
@@ -279,9 +283,9 @@ public class JsonFileWriter implements AutoCloseable {
           break;
         }
         case DECIMAL: {
-          String hexString = Hex.encodeHexString(DecimalUtility.getByteArrayFromArrowBuf(buffer,
-                  index));
-          generator.writeString(hexString);
+          BigDecimal decimalValue = DecimalUtility.getBigDecimalFromArrowBuf(buffer, index, scale);
+          // We write the unscaled value, because the scale is stored in the type metadata.
+          generator.writeString(decimalValue.unscaledValue().toString());
           break;
         }
         default:
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 ede4f99..aec7d0f 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
@@ -20,6 +20,7 @@ package org.apache.arrow.vector;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 import java.util.List;
 
@@ -113,10 +114,23 @@ public class TestUnionVector {
         assertEquals(srcVector.getField(), destVector.getField());
 
         // now check the values are transferred
-        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));
-        }
+        assertEquals(6, destVector.getValueCount());
+
+        assertFalse(destVector.isNull(0));
+        assertEquals(5, destVector.getObject(0));
+
+        assertFalse(destVector.isNull(1));
+        assertEquals(false, destVector.getObject(1));
+
+        assertTrue(destVector.isNull(2));
+
+        assertFalse(destVector.isNull(3));
+        assertEquals(10, destVector.getObject(3));
+
+        assertTrue(destVector.isNull(4));
+
+        assertFalse(destVector.isNull(5));
+        assertEquals(false, destVector.getObject(5));
       }
     }
   }

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