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/28 17:32:46 UTC

[arrow] branch master updated: ARROW-1710: [Java] Remove Non-Nullable Vectors

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


The following commit(s) were added to refs/heads/master by this push:
     new 689bbd7  ARROW-1710: [Java] Remove Non-Nullable Vectors
689bbd7 is described below

commit 689bbd72ae186ad8aaf2df904357c3981f1108d4
Author: Bryan Cutler <cu...@gmail.com>
AuthorDate: Tue Nov 28 12:32:42 2017 -0500

    ARROW-1710: [Java] Remove Non-Nullable Vectors
    
    This removes non-nullable vectors that are no longer part of the vector class hierarchy and renames Nullable*Vector classes to remove the Nullable prefix.
    
    Author: Bryan Cutler <cu...@gmail.com>
    
    Closes #1341 from BryanCutler/java-nullable-vector-rename-ARROW-1710 and squashes the following commits:
    
    7d930dc2 [Bryan Cutler] fixed realloc test
    ff2120df [Bryan Cutler] clean up test
    374dfcc5 [Bryan Cutler] properly rename BitVector file
    6b7a85e7 [Bryan Cutler] remove old BitVector.java before rebase
    089f7fc9 [Bryan Cutler] some minor cleanup
    4e580d93 [Bryan Cutler] removed legacy BitVector
    74f771f5 [Bryan Cutler] fixed remaining tests
    8c5dfef6 [Bryan Cutler] fix naming in support classes
    6e498e55 [Bryan Cutler] removed nullable prefix
    dfed444c [Bryan Cutler] removed non-nullable vectors
---
 .../org/apache/arrow/tools/EchoServerTest.java     |  30 +-
 .../codegen/templates/CaseSensitiveMapWriters.java |   2 +-
 .../src/main/codegen/templates/ComplexReaders.java |  12 +-
 .../src/main/codegen/templates/ComplexWriters.java |  10 +-
 .../main/codegen/templates/FixedValueVectors.java  | 765 ------------------
 .../src/main/codegen/templates/MapWriters.java     |   9 +-
 .../src/main/codegen/templates/UnionVector.java    |  12 +-
 .../codegen/templates/VariableLengthVectors.java   | 677 ----------------
 ...dWidthVector.java => BaseFixedWidthVector.java} |  22 +-
 ...dthVector.java => BaseVariableWidthVector.java} |  16 +-
 ...NullableBigIntVector.java => BigIntVector.java} |  32 +-
 .../java/org/apache/arrow/vector/BitVector.java    | 878 +++++++++------------
 ...llableDateDayVector.java => DateDayVector.java} |  32 +-
 ...leDateMilliVector.java => DateMilliVector.java} |  32 +-
 ...llableDecimalVector.java => DecimalVector.java} |  34 +-
 ...NullableFloat4Vector.java => Float4Vector.java} |  32 +-
 ...NullableFloat8Vector.java => Float8Vector.java} |  32 +-
 .../apache/arrow/vector/GenerateSampleData.java    | 142 ++--
 .../{NullableIntVector.java => IntVector.java}     |  32 +-
 ...tervalDayVector.java => IntervalDayVector.java} |  32 +-
 ...rvalYearVector.java => IntervalYearVector.java} |  32 +-
 .../org/apache/arrow/vector/NullableBitVector.java | 513 ------------
 .../org/apache/arrow/vector/NullableVector.java    |  26 -
 ...ableSmallIntVector.java => SmallIntVector.java} |  32 +-
 ...leTimeMicroVector.java => TimeMicroVector.java} |  32 +-
 ...leTimeMilliVector.java => TimeMilliVector.java} |  32 +-
 ...ableTimeNanoVector.java => TimeNanoVector.java} |  32 +-
 ...llableTimeSecVector.java => TimeSecVector.java} |  32 +-
 ...roTZVector.java => TimeStampMicroTZVector.java} |  18 +-
 ...pMicroVector.java => TimeStampMicroVector.java} |  18 +-
 ...liTZVector.java => TimeStampMilliTZVector.java} |  18 +-
 ...pMilliVector.java => TimeStampMilliVector.java} |  18 +-
 ...anoTZVector.java => TimeStampNanoTZVector.java} |  18 +-
 ...ampNanoVector.java => TimeStampNanoVector.java} |  18 +-
 ...pSecTZVector.java => TimeStampSecTZVector.java} |  18 +-
 ...StampSecVector.java => TimeStampSecVector.java} |  18 +-
 ...leTimeStampVector.java => TimeStampVector.java} |  22 +-
 ...llableTinyIntVector.java => TinyIntVector.java} |  32 +-
 .../{NullableUInt1Vector.java => UInt1Vector.java} |  26 +-
 .../{NullableUInt2Vector.java => UInt2Vector.java} |  26 +-
 .../{NullableUInt4Vector.java => UInt4Vector.java} |  26 +-
 .../{NullableUInt8Vector.java => UInt8Vector.java} |  26 +-
 ...leVarBinaryVector.java => VarBinaryVector.java} |  32 +-
 ...llableVarCharVector.java => VarCharVector.java} |  32 +-
 ...tionSetter.java => VectorDefinitionSetter.java} |   2 +-
 .../vector/complex/AbstractContainerVector.java    |   4 +-
 .../arrow/vector/complex/EmptyValuePopulator.java  |  10 +-
 .../org/apache/arrow/vector/complex/MapVector.java | 562 ++++++++-----
 .../{MapVector.java => NonNullableMapVector.java}  |  30 +-
 .../arrow/vector/complex/NullableMapVector.java    | 512 ------------
 .../vector/complex/impl/ComplexWriterImpl.java     |   6 +-
 .../vector/complex/impl/NullableMapReaderImpl.java |   8 +-
 .../complex/impl/NullableMapWriterFactory.java     |   4 +-
 .../vector/complex/impl/PromotableWriter.java      |   4 +-
 .../vector/complex/impl/SingleMapReaderImpl.java   |   5 +-
 .../arrow/vector/complex/impl/UnionListReader.java |   1 -
 .../apache/arrow/vector/ipc/JsonFileReader.java    |  14 +-
 .../apache/arrow/vector/ipc/JsonFileWriter.java    |  52 +-
 .../java/org/apache/arrow/vector/types/Types.java  | 186 ++---
 .../apache/arrow/vector/util/DecimalUtility.java   |   1 -
 .../org/apache/arrow/vector/TestBitVector.java     | 189 ++---
 .../arrow/vector/TestBufferOwnershipTransfer.java  |   8 +-
 .../java/org/apache/arrow/vector/TestCopyFrom.java |  86 +-
 .../org/apache/arrow/vector/TestDecimalVector.java |   6 +-
 .../apache/arrow/vector/TestDictionaryVector.java  |  22 +-
 .../arrow/vector/TestFixedSizeListVector.java      |   8 +-
 .../org/apache/arrow/vector/TestListVector.java    |   8 +-
 .../org/apache/arrow/vector/TestMapVector.java     |   4 +-
 .../TestOversizedAllocationForValueVector.java     |   4 +-
 .../apache/arrow/vector/TestSplitAndTransfer.java  |   8 +-
 .../java/org/apache/arrow/vector/TestUtils.java    |   8 +-
 .../org/apache/arrow/vector/TestValueVector.java   | 308 ++++----
 .../org/apache/arrow/vector/TestVectorReAlloc.java |  17 +-
 .../org/apache/arrow/vector/TestVectorReset.java   |   5 +-
 .../apache/arrow/vector/TestVectorUnloadLoad.java  |  10 +-
 .../vector/complex/impl/TestPromotableWriter.java  |   4 +-
 .../vector/complex/writer/TestComplexWriter.java   |  22 +-
 .../org/apache/arrow/vector/ipc/BaseFileTest.java  |  60 +-
 .../org/apache/arrow/vector/ipc/TestArrowFile.java |  26 +-
 .../apache/arrow/vector/ipc/TestArrowStream.java   |   6 +-
 .../arrow/vector/ipc/TestArrowStreamPipe.java      |   7 +-
 .../org/apache/arrow/vector/ipc/TestJSONFile.java  |  10 +-
 82 files changed, 1822 insertions(+), 4335 deletions(-)

diff --git a/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java b/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
index d8693c5..47b5541 100644
--- a/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
+++ b/java/tools/src/test/java/org/apache/arrow/tools/EchoServerTest.java
@@ -35,9 +35,9 @@ import java.util.List;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.NullableIntVector;
-import org.apache.arrow.vector.NullableTinyIntVector;
-import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.impl.UnionListWriter;
@@ -92,7 +92,7 @@ public class EchoServerTest {
 
   private void testEchoServer(int serverPort,
                               Field field,
-                              NullableTinyIntVector vector,
+                              TinyIntVector vector,
                               int batches)
       throws UnknownHostException, IOException {
     VectorSchemaRoot root = new VectorSchemaRoot(asList(field), asList((FieldVector) vector), 0);
@@ -115,7 +115,7 @@ public class EchoServerTest {
 
       assertEquals(new Schema(asList(field)), reader.getVectorSchemaRoot().getSchema());
 
-      NullableTinyIntVector readVector = (NullableTinyIntVector) reader.getVectorSchemaRoot()
+      TinyIntVector readVector = (TinyIntVector) reader.getVectorSchemaRoot()
           .getFieldVectors().get(0);
       for (int i = 0; i < batches; i++) {
         Assert.assertTrue(reader.loadNextBatch());
@@ -140,8 +140,8 @@ public class EchoServerTest {
         "testField",
         new FieldType(true, new ArrowType.Int(8, true), null, null),
         Collections.<Field>emptyList());
-    NullableTinyIntVector vector =
-        new NullableTinyIntVector("testField", FieldType.nullable(TINYINT.getType()), alloc);
+    TinyIntVector vector =
+        new TinyIntVector("testField", FieldType.nullable(TINYINT.getType()), alloc);
     Schema schema = new Schema(asList(field));
 
     // Try an empty stream, just the header.
@@ -158,13 +158,13 @@ public class EchoServerTest {
   public void testFlatDictionary() throws IOException {
     DictionaryEncoding writeEncoding = new DictionaryEncoding(1L, false, null);
     try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE);
-         NullableIntVector writeVector =
-             new NullableIntVector(
+         IntVector writeVector =
+             new IntVector(
                  "varchar",
                  new FieldType(true, MinorType.INT.getType(), writeEncoding, null),
                  allocator);
-         NullableVarCharVector writeDictionaryVector =
-             new NullableVarCharVector(
+         VarCharVector writeDictionaryVector =
+             new VarCharVector(
                  "dict",
                  FieldType.nullable(VARCHAR.getType()),
                  allocator)) {
@@ -218,7 +218,7 @@ public class EchoServerTest {
 
         Dictionary dictionary = reader.lookup(1L);
         Assert.assertNotNull(dictionary);
-        NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary.getVector());
+        VarCharVector dictionaryVector = ((VarCharVector) dictionary.getVector());
         Assert.assertEquals(3, dictionaryVector.getValueCount());
         Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
         Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
@@ -231,8 +231,8 @@ public class EchoServerTest {
   public void testNestedDictionary() throws IOException {
     DictionaryEncoding writeEncoding = new DictionaryEncoding(2L, false, null);
     try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE);
-         NullableVarCharVector writeDictionaryVector =
-             new NullableVarCharVector("dictionary", FieldType.nullable(VARCHAR.getType()), allocator);
+         VarCharVector writeDictionaryVector =
+             new VarCharVector("dictionary", FieldType.nullable(VARCHAR.getType()), allocator);
          ListVector writeVector = ListVector.empty("list", allocator)) {
 
       // data being written:
@@ -300,7 +300,7 @@ public class EchoServerTest {
 
         Dictionary readDictionary = reader.lookup(2L);
         Assert.assertNotNull(readDictionary);
-        NullableVarCharVector dictionaryVector = ((NullableVarCharVector) readDictionary.getVector());
+        VarCharVector dictionaryVector = ((VarCharVector) readDictionary.getVector());
         Assert.assertEquals(2, dictionaryVector.getValueCount());
         Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
         Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
diff --git a/java/vector/src/main/codegen/templates/CaseSensitiveMapWriters.java b/java/vector/src/main/codegen/templates/CaseSensitiveMapWriters.java
index 5357f9b..7c65290 100644
--- a/java/vector/src/main/codegen/templates/CaseSensitiveMapWriters.java
+++ b/java/vector/src/main/codegen/templates/CaseSensitiveMapWriters.java
@@ -23,7 +23,7 @@
 <#if mode == "Single">
 <#assign containerClass = "MapVector" />
 <#else>
-<#assign containerClass = "NullableMapVector" />
+<#assign containerClass = "MapVector" />
 </#if>
 
 <#include "/@includes/license.ftl" />
diff --git a/java/vector/src/main/codegen/templates/ComplexReaders.java b/java/vector/src/main/codegen/templates/ComplexReaders.java
index 60347a2..4863ecd 100644
--- a/java/vector/src/main/codegen/templates/ComplexReaders.java
+++ b/java/vector/src/main/codegen/templates/ComplexReaders.java
@@ -53,9 +53,9 @@ package org.apache.arrow.vector.complex.impl;
 @SuppressWarnings("unused")
 public class ${name}ReaderImpl extends AbstractFieldReader {
   
-  private final ${nullMode}${name}Vector vector;
+  private final ${name}Vector vector;
   
-  public ${name}ReaderImpl(${nullMode}${name}Vector vector){
+  public ${name}ReaderImpl(${name}Vector vector){
     super();
     this.vector = vector;
   }
@@ -69,11 +69,7 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
   }
   
   public boolean isSet(){
-    <#if nullMode == "Nullable">
-        return !vector.isNull(idx());
-    <#else>
-    return true;
-    </#if>
+    return !vector.isNull(idx());
   }
 
   public void copyAsValue(${minor.class?cap_first}Writer writer){
@@ -88,7 +84,7 @@ public class ${name}ReaderImpl extends AbstractFieldReader {
 
   <#if nullMode != "Nullable">
   public void read(${minor.class?cap_first}Holder h){
-    vector.getAccessor().get(idx(), h);
+    vector.get(idx(), h);
   }
   </#if>
 
diff --git a/java/vector/src/main/codegen/templates/ComplexWriters.java b/java/vector/src/main/codegen/templates/ComplexWriters.java
index 406bbb3..98672d5 100644
--- a/java/vector/src/main/codegen/templates/ComplexWriters.java
+++ b/java/vector/src/main/codegen/templates/ComplexWriters.java
@@ -39,9 +39,9 @@ package org.apache.arrow.vector.complex.impl;
 @SuppressWarnings("unused")
 public class ${eName}WriterImpl extends AbstractFieldWriter {
 
-  final Nullable${name}Vector vector;
+  final ${name}Vector vector;
 
-  public ${eName}WriterImpl(Nullable${name}Vector vector) {
+  public ${eName}WriterImpl(${name}Vector vector) {
     this.vector = vector;
   }
 
@@ -82,7 +82,7 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
     vector.getMutator().setValueCount(idx()+1);
   }
 
-  public void write(Nullable${minor.class?cap_first}Holder h) {
+  public void write(${minor.class?cap_first}Holder h) {
     mutator.addSafe(idx(), h);
     vector.getMutator().setValueCount(idx()+1);
   }
@@ -111,7 +111,7 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
   }
 
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
-    vector.setSafe(idx()<#if mode == "Nullable">, 1</#if><#list fields as field><#if field.include!true >, ${field.name}</#if></#list>);
+    vector.setSafe(idx(), 1<#list fields as field><#if field.include!true >, ${field.name}</#if></#list>);
     vector.setValueCount(idx()+1);
   }
 
@@ -122,13 +122,11 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
   }
   </#if>
 
-  <#if mode == "Nullable">
   public void writeNull() {
     vector.setNull(idx());
     vector.setValueCount(idx()+1);
   }
   </#if>
-  </#if>
 }
 
 <@pp.changeOutputFile name="/org/apache/arrow/vector/complex/writer/${eName}Writer.java" />
diff --git a/java/vector/src/main/codegen/templates/FixedValueVectors.java b/java/vector/src/main/codegen/templates/FixedValueVectors.java
deleted file mode 100644
index e07416b..0000000
--- a/java/vector/src/main/codegen/templates/FixedValueVectors.java
+++ /dev/null
@@ -1,765 +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.
- */
-
-import org.apache.arrow.vector.util.DecimalUtility;
-
-import java.lang.Override;
-import java.util.concurrent.TimeUnit;
-
-<@pp.dropOutputFile />
-<#list vv.types as type>
-<#list type.minor as minor>
-<#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
-<#assign className = "${minor.class}Vector" />
-
-<#if type.major == "Fixed">
-<@pp.changeOutputFile name="/org/apache/arrow/vector/${className}.java" />
-<#include "/@includes/license.ftl" />
-
-package org.apache.arrow.vector;
-
-<#include "/@includes/vv_imports.ftl" />
-
-/**
- * ${minor.class} implements a vector of fixed width values.  Elements in the vector are accessed
- * by position, starting from the logical start of the vector.  Values should be pushed onto the
- * vector sequentially, but may be randomly accessed.
- *   The width of each element is ${type.width} byte(s)
- *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
- *
- * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
- */
-public final class ${className} extends BaseDataValueVector implements FixedWidthVector{
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class);
-
-  public static final int TYPE_WIDTH = ${type.width};
-
-  private final Accessor accessor = new Accessor();
-  private final Mutator mutator = new Mutator();
-
-  private int allocationSizeInBytes = INITIAL_VALUE_ALLOCATION * ${type.width};
-  private int allocationMonitor = 0;
-  <#if minor.typeParams??>
-
-    <#assign typeParams = minor.typeParams?reverse />
-    <#list typeParams as typeParam>
-  private final ${typeParam.type} ${typeParam.name};
-    </#list>
-
-  public ${className}(String name, BufferAllocator allocator<#list typeParams as typeParam>, ${typeParam.type} ${typeParam.name}</#list>) {
-    super(name, allocator);
-    <#list typeParams as typeParam>
-    this.${typeParam.name} = ${typeParam.name};
-    </#list>
-  }
-  <#else>
-  public ${className}(String name, BufferAllocator allocator) {
-    super(name, allocator);
-  }
-  </#if>
-
-  @Override
-  public MinorType getMinorType() {
-    return MinorType.${minor.class?upper_case};
-  }
-
-  @Override
-  public Field getField() {
-        throw new UnsupportedOperationException("internal vector");
-  }
-
-  @Override
-  public FieldReader getReader(){
-        throw new UnsupportedOperationException("non-nullable vectors cannot be used in readers");
-  }
-
-  @Override
-  public int getBufferSizeFor(final int valueCount) {
-    if (valueCount == 0) {
-      return 0;
-    }
-    return valueCount * ${type.width};
-  }
-
-  @Override
-  public ArrowBuf getValidityBuffer() {
-    /* this operation is not supported for non-nullable vectors */
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrowBuf getDataBuffer() {
-    /* we are not throwing away getBuffer() of BaseDataValueVector so use it wherever applicable */
-    return getBuffer();
-  }
-
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    /* this operation is not supported for fixed-width vectors */
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int getValueCapacity(){
-    return (int) (data.capacity() *1.0 / ${type.width});
-  }
-
-  @Override
-  public Accessor getAccessor(){
-    return accessor;
-  }
-
-  @Override
-  public Mutator getMutator(){
-    return mutator;
-  }
-
-  int getAllocationSize() {
-    return allocationSizeInBytes;
-  }
-
-  @Override
-  public void setInitialCapacity(final int valueCount) {
-    final long size = 1L * valueCount * ${type.width};
-    if (size > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
-    }
-    allocationSizeInBytes = (int)size;
-  }
-
-  @Override
-  public void allocateNew() {
-    if(!allocateNewSafe()){
-      throw new OutOfMemoryException("Failure while allocating buffer.");
-    }
-  }
-
-  @Override
-  public boolean allocateNewSafe() {
-    long curAllocationSize = allocationSizeInBytes;
-    if (allocationMonitor > 10) {
-      curAllocationSize = Math.max(8, curAllocationSize / 2);
-      allocationMonitor = 0;
-    } else if (allocationMonitor < -2) {
-      curAllocationSize = allocationSizeInBytes * 2L;
-      allocationMonitor = 0;
-    }
-
-    try{
-      allocateBytes(curAllocationSize);
-    } catch (RuntimeException ex) {
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Allocate a new buffer that supports setting at least the provided number of values. May actually be sized bigger
-   * depending on underlying buffer rounding size. Must be called prior to using the ValueVector.
-   *
-   * Note that the maximum number of values a vector can allocate is Integer.MAX_VALUE / value width.
-   *
-   * @param valueCount the number of values to allocate for
-   * @throws org.apache.arrow.memory.OutOfMemoryException if it can't allocate the new buffer
-   */
-  @Override
-  public void allocateNew(final int valueCount) {
-    allocateBytes(valueCount * ${type.width});
-  }
-
-  @Override
-  public void reset() {
-    allocationSizeInBytes = INITIAL_VALUE_ALLOCATION * ${type.width};
-    allocationMonitor = 0;
-    zeroVector();
-    super.reset();
-  }
-
-  private void allocateBytes(final long size) {
-    if (size > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
-    }
-
-    final int curSize = (int)size;
-    clear();
-    data = allocator.buffer(curSize);
-    data.readerIndex(0);
-    allocationSizeInBytes = curSize;
-  }
-
-  /**
-   * Allocate new buffer with double capacity, and copy data into the new buffer. Replace vector's buffer with new buffer, and release old one
-   *
-   * @throws org.apache.arrow.memory.OutOfMemoryException if it can't allocate the new buffer
-   */
-  public void reAlloc() {
-    long baseSize  = allocationSizeInBytes;
-    final int currentBufferCapacity = data.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. Max allowed buffer size is reached.");
-    }
-
-    logger.debug("Reallocating vector [{}]. # of bytes: [{}] -> [{}]", name, allocationSizeInBytes, newAllocationSize);
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
-    newBuf.setBytes(0, data, 0, currentBufferCapacity);
-    final int halfNewCapacity = newBuf.capacity() / 2;
-    newBuf.setZero(halfNewCapacity, halfNewCapacity);
-    newBuf.writerIndex(data.writerIndex());
-    data.release(1);
-    data = newBuf;
-    allocationSizeInBytes = (int)newAllocationSize;
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void zeroVector() {
-    data.setZero(0, data.capacity());
-  }
-
-  public TransferPair getTransferPair(BufferAllocator allocator){
-    return new TransferImpl(name, allocator);
-  }
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-    return new TransferImpl(ref, allocator);
-  }
-
-  @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((${className}) to);
-  }
-
-  public void transferTo(${className} target){
-    target.clear();
-    target.data = data.transferOwnership(target.allocator).buffer;
-    target.data.writerIndex(data.writerIndex());
-    clear();
-  }
-
-  public void splitAndTransferTo(int startIndex, int length, ${className} target) {
-    final int startPoint = startIndex * ${type.width};
-    final int sliceLength = length * ${type.width};
-    target.clear();
-    target.data = data.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer;
-    target.data.writerIndex(sliceLength);
-  }
-
-  private class TransferImpl implements TransferPair{
-    private ${className} to;
-
-    public TransferImpl(String name, BufferAllocator allocator){
-      to = new ${className}(name, allocator<#if minor.typeParams??><#list typeParams as typeParam>,  ${className}.this.${typeParam.name}</#list></#if>);
-    }
-
-    public TransferImpl(${className} to) {
-      this.to = to;
-    }
-
-    @Override
-    public ${className} getTo(){
-      return to;
-    }
-
-    @Override
-    public void transfer(){
-      transferTo(to);
-    }
-
-    @Override
-    public void splitAndTransfer(int startIndex, int length) {
-      splitAndTransferTo(startIndex, length, to);
-    }
-
-    @Override
-    public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, ${className}.this);
-    }
-  }
-
-  public void copyFrom(int fromIndex, int thisIndex, ${className} from){
-    <#if (type.width > 8 || minor.class == "IntervalDay")>
-    from.data.getBytes(fromIndex * ${type.width}, data, thisIndex * ${type.width}, ${type.width});
-    <#else> <#-- type.width <= 8 -->
-    data.set${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width},
-        from.data.get${(minor.javaType!type.javaType)?cap_first}(fromIndex * ${type.width})
-    );
-    </#if> <#-- type.width -->
-  }
-
-  public void copyFromSafe(int fromIndex, int thisIndex, ${className} from){
-    while(thisIndex >= getValueCapacity()) {
-        reAlloc();
-    }
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-  public void decrementAllocationMonitor() {
-    if (allocationMonitor > 0) {
-      allocationMonitor = 0;
-    }
-    --allocationMonitor;
-  }
-
-  private void incrementAllocationMonitor() {
-    ++allocationMonitor;
-  }
-
-  public final class Accessor extends BaseDataValueVector.BaseAccessor {
-    @Override
-    public int getValueCount() {
-      return data.writerIndex() / ${type.width};
-    }
-
-    @Override
-    public boolean isNull(int index){
-      return false;
-    }
-
-    <#if (type.width > 8 || minor.class == "IntervalDay")>
-    public ${minor.javaType!type.javaType} get(int index) {
-      return data.slice(index * ${type.width}, ${type.width});
-    }
-
-      <#if (minor.class == "IntervalDay")>
-    public void get(int index, ${minor.class}Holder holder){
-      final int offsetIndex = index * ${type.width};
-      holder.days = data.getInt(offsetIndex);
-      holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-    }
-
-    public void get(int index, Nullable${minor.class}Holder holder){
-      final int offsetIndex = index * ${type.width};
-      holder.isSet = 1;
-      holder.days = data.getInt(offsetIndex);
-      holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-    }
-
-    @Override
-    public ${friendlyType} getObject(int index) {
-      final int offsetIndex = index * ${type.width};
-      final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-      final int  days   = data.getInt(offsetIndex);
-      final Period p = new Period();
-      return p.plusDays(days).plusMillis(millis);
-    }
-
-    public StringBuilder getAsStringBuilder(int index) {
-      final int offsetIndex = index * ${type.width};
-
-      int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-      final int  days   = data.getInt(offsetIndex);
-
-      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));
-    }
-
-      <#elseif minor.class == "Decimal">
-    public void get(int index, ${minor.class}Holder holder) {
-        holder.start = index * ${type.width};
-        holder.buffer = data;
-        holder.scale = scale;
-        holder.precision = precision;
-    }
-
-    public void get(int index, Nullable${minor.class}Holder holder) {
-        holder.isSet = 1;
-        holder.start = index * ${type.width};
-        holder.buffer = data;
-        holder.scale = scale;
-        holder.precision = precision;
-    }
-
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return DecimalUtility.getBigDecimalFromArrowBuf(data, index, scale);
-    }
-
-      <#else>
-    public void get(int index, ${minor.class}Holder holder){
-      holder.buffer = data;
-      holder.start = index * ${type.width};
-    }
-
-    public void get(int index, Nullable${minor.class}Holder holder){
-      holder.isSet = 1;
-      holder.buffer = data;
-      holder.start = index * ${type.width};
-    }
-
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return data.slice(index * ${type.width}, ${type.width})
-    }
-
-      </#if>
-    <#else> <#-- type.width <= 8 -->
-
-    public ${minor.javaType!type.javaType} get(int index) {
-      return data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-    }
-      <#if type.width == 4>
-    public long getTwoAsLong(int index) {
-      return data.getLong(index * ${type.width});
-    }
-
-      </#if>
-      <#if minor.class == "DateDay" ||
-           minor.class == "TimeSec" ||
-           minor.class == "TimeMicro" ||
-           minor.class == "TimeNano">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return get(index);
-    }
-
-      <#elseif minor.class == "DateMilli" || minor.class == "TimeMilli" || minor.class == "TimeStampMilli">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      org.joda.time.LocalDateTime ldt = new org.joda.time.LocalDateTime(get(index), org.joda.time.DateTimeZone.UTC);
-      return ldt;
-    }
-
-      <#elseif minor.class == "TimeStampSec">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      long secs = java.util.concurrent.TimeUnit.SECONDS.toMillis(get(index));
-      org.joda.time.LocalDateTime date = new org.joda.time.LocalDateTime(secs, org.joda.time.DateTimeZone.UTC);
-      return date;
-    }
-
-      <#elseif minor.class == "TimeStampMicro">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      // value is truncated when converting microseconds to milliseconds in order to use DateTime type
-      long micros = java.util.concurrent.TimeUnit.MICROSECONDS.toMillis(get(index));
-      org.joda.time.LocalDateTime date = new org.joda.time.LocalDateTime(micros, org.joda.time.DateTimeZone.UTC);
-      return date;
-    }
-
-      <#elseif minor.class == "TimeStampNano">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      // value is truncated when converting nanoseconds to milliseconds in order to use DateTime type
-      long millis = java.util.concurrent.TimeUnit.NANOSECONDS.toMillis(get(index));
-      org.joda.time.LocalDateTime date = new org.joda.time.LocalDateTime(millis, org.joda.time.DateTimeZone.UTC);
-      return date;
-    }
-
-      <#elseif minor.class == "IntervalYear">
-    @Override
-    public ${friendlyType} getObject(int index) {
-
-      final int value = get(index);
-
-      final int years  = (value / org.apache.arrow.vector.util.DateUtility.yearsToMonths);
-      final int months = (value % org.apache.arrow.vector.util.DateUtility.yearsToMonths);
-      final Period p = new Period();
-      return p.plusYears(years).plusMonths(months);
-    }
-
-    public StringBuilder getAsStringBuilder(int index) {
-      int months  = data.getInt(index);
-
-      final int years  = (months / org.apache.arrow.vector.util.DateUtility.yearsToMonths);
-      months = (months % 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));
-    }
-
-      <#else>
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return get(index);
-    }
-    public ${minor.javaType!type.javaType} getPrimitiveObject(int index) {
-      return get(index);
-    }
-
-      </#if>
-    public void get(int index, ${minor.class}Holder holder){
-      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-    }
-
-    public void get(int index, Nullable${minor.class}Holder holder){
-      holder.isSet = 1;
-      holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-    }
-
-    </#if> <#-- type.width -->
-  }
-
-  /**
-   * ${minor.class}.Mutator implements a mutable vector of fixed width values.  Elements in the
-   * vector are accessed by position from the logical start of the vector.  Values should be pushed
-   * onto the vector sequentially, but may be randomly accessed.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from FixedValueVectorTypes.tdd using FreeMarker.
-   */
-  public final class Mutator extends BaseDataValueVector.BaseMutator {
-
-    private Mutator(){};
-
-   /**
-    * Set the element at the given index to the given value.  Note that widths smaller than
-    * 32 bits are handled by the ArrowBuf interface.
-    *
-    * @param index   position of the bit to set
-    * @param value   value to set
-    */
-    <#if (type.width > 8) || minor.class == "IntervalDay">
-   public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-     data.setBytes(index * ${type.width}, value, 0, ${type.width});
-   }
-
-   public void setSafe(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     data.setBytes(index * ${type.width}, value, 0, ${type.width});
-   }
-
-      <#if (minor.class == "IntervalDay")>
-   public void set(int index, int days, int milliseconds){
-     final int offsetIndex = index * ${type.width};
-     data.setInt(offsetIndex, days);
-     data.setInt((offsetIndex + ${minor.millisecondsOffset}), milliseconds);
-   }
-
-   protected void set(int index, ${minor.class}Holder holder){
-     set(index, holder.days, holder.milliseconds);
-   }
-
-   protected void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.days, holder.milliseconds);
-   }
-
-   public void setSafe(int index, int days, int milliseconds){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, days, milliseconds);
-   }
-
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.days, holder.milliseconds);
-   }
-
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.days, holder.milliseconds);
-   }
-
-       <#elseif minor.class == "Decimal">
-   public void set(int index, ${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
-
-   void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
-
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-
-   public void setSafe(int index, int start, ArrowBuf buffer){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, start, buffer);
-   }
-
-   public void set(int index, int start, ArrowBuf buffer){
-     data.setBytes(index * ${type.width}, buffer, start, ${type.width});
-   }
-
-   public void set(int index, ${friendlyType} value){
-     DecimalUtility.checkPrecisionAndScale(value, precision, scale);
-     DecimalUtility.writeBigDecimalToArrowBuf(value, data, index);
-   }
-
-   public void setSafe(int index, ${friendlyType} value){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, value);
-   }
-
-       <#else>
-   protected void set(int index, ${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
-
-   public void set(int index, Nullable${minor.class}Holder holder){
-     set(index, holder.start, holder.buffer);
-   }
-
-   public void set(int index, int start, ArrowBuf buffer){
-     data.setBytes(index * ${type.width}, buffer, start, ${type.width});
-   }
-
-   public void setSafe(int index, ${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     setSafe(index, holder.start, holder.buffer);
-   }
-
-   public void setSafe(int index, int start, ArrowBuf buffer){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, holder);
-   }
-
-   public void set(int index, Nullable${minor.class}Holder holder){
-     data.setBytes(index * ${type.width}, holder.buffer, holder.start, ${type.width});
-   }
-       </#if>
-
-   @Override
-   public void generateTestData(int count) {
-     setValueCount(count);
-     boolean even = true;
-     final int valueCount = getAccessor().getValueCount();
-     for(int i = 0; i < valueCount; i++, even = !even) {
-       final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
-       for(int w = 0; w < ${type.width}; w++){
-         data.setByte(i + w, b);
-       }
-     }
-   }
-
-     <#else> <#-- type.width <= 8 -->
-   public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, value);
-   }
-
-   public void setSafe(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, value);
-   }
-
-   protected void set(int index, ${minor.class}Holder holder){
-     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
-   }
-
-   public void setSafe(int index, ${minor.class}Holder holder){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, holder);
-   }
-
-   protected void set(int index, Nullable${minor.class}Holder holder){
-     data.set${(minor.javaType!type.javaType)?cap_first}(index * ${type.width}, holder.value);
-   }
-
-   public void setSafe(int index, Nullable${minor.class}Holder holder){
-     while(index >= getValueCapacity()) {
-       reAlloc();
-     }
-     set(index, holder);
-   }
-
-   @Override
-   public void generateTestData(int size) {
-     setValueCount(size);
-     boolean even = true;
-     final int valueCount = getAccessor().getValueCount();
-     for(int i = 0; i < valueCount; i++, even = !even) {
-       if(even){
-         set(i, ${minor.boxedType!type.boxedType}.MIN_VALUE);
-       }else{
-         set(i, ${minor.boxedType!type.boxedType}.MAX_VALUE);
-       }
-     }
-   }
-
-   public void generateTestDataAlt(int size) {
-     setValueCount(size);
-     boolean even = true;
-     final int valueCount = getAccessor().getValueCount();
-     for(int i = 0; i < valueCount; i++, even = !even) {
-       if(even){
-         set(i, (${(minor.javaType!type.javaType)}) 1);
-       }else{
-         set(i, (${(minor.javaType!type.javaType)}) 0);
-       }
-     }
-   }
-
-    </#if> <#-- type.width -->
-
-   @Override
-   public void setValueCount(int valueCount) {
-     final int currentValueCapacity = getValueCapacity();
-     final int idx = (${type.width} * valueCount);
-     while(valueCount > getValueCapacity()) {
-       reAlloc();
-     }
-     if (valueCount > 0 && currentValueCapacity > valueCount * 2) {
-       incrementAllocationMonitor();
-     } else if (allocationMonitor > 0) {
-       allocationMonitor = 0;
-     }
-     VectorTrimmer.trim(data, idx);
-     data.writerIndex(valueCount * ${type.width});
-   }
- }
-}
-
-  </#if> <#-- type.major -->
-</#list>
-</#list>
diff --git a/java/vector/src/main/codegen/templates/MapWriters.java b/java/vector/src/main/codegen/templates/MapWriters.java
index 6afd508..a5ac1b7 100644
--- a/java/vector/src/main/codegen/templates/MapWriters.java
+++ b/java/vector/src/main/codegen/templates/MapWriters.java
@@ -21,9 +21,9 @@
 <@pp.changeOutputFile name="/org/apache/arrow/vector/complex/impl/${mode}MapWriter.java" />
 <#assign index = "idx()">
 <#if mode == "Single">
-<#assign containerClass = "MapVector" />
+<#assign containerClass = "NonNullableMapVector" />
 <#else>
-<#assign containerClass = "NullableMapVector" />
+<#assign containerClass = "MapVector" />
 </#if>
 
 <#include "/@includes/license.ftl" />
@@ -51,7 +51,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
   private final Map<String, FieldWriter> fields = Maps.newHashMap();
   public ${mode}MapWriter(${containerClass} container) {
     <#if mode == "Single">
-    if (container instanceof NullableMapVector) {
+    if (container instanceof MapVector) {
       throw new IllegalArgumentException("Invalid container: " + container);
     }
     </#if>
@@ -124,7 +124,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
     FieldWriter writer = fields.get(finalName);
     if(writer == null){
       int vectorCount=container.size();
-      NullableMapVector vector = container.addOrGet(name, FieldType.nullable(MinorType.MAP.getType()), NullableMapVector.class);
+      MapVector vector = container.addOrGet(name, FieldType.nullable(MinorType.MAP.getType()), MapVector.class);
       writer = new PromotableWriter(vector, container, getNullableMapWriterFactory());
       if(vectorCount != container.size()) {
         writer.allocate();
@@ -214,7 +214,6 @@ public class ${mode}MapWriter extends AbstractFieldWriter {
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
   <#assign vectName = capName />
-  <#assign vectName = "Nullable${capName}" />
 
   <#if minor.typeParams?? >
   @Override
diff --git a/java/vector/src/main/codegen/templates/UnionVector.java b/java/vector/src/main/codegen/templates/UnionVector.java
index 7316531..501933f 100644
--- a/java/vector/src/main/codegen/templates/UnionVector.java
+++ b/java/vector/src/main/codegen/templates/UnionVector.java
@@ -65,7 +65,7 @@ public class UnionVector implements FieldVector {
   MapVector internalMap;
   protected ArrowBuf typeBuffer;
 
-  private NullableMapVector mapVector;
+  private MapVector mapVector;
   private ListVector listVector;
 
   private FieldReader reader;
@@ -174,10 +174,10 @@ public class UnionVector implements FieldVector {
   @Override
   public ArrowBuf getOffsetBuffer() { throw new UnsupportedOperationException(); }
 
-  public NullableMapVector getMap() {
+  public MapVector getMap() {
     if (mapVector == null) {
       int vectorCount = internalMap.size();
-      mapVector = addOrGet(MinorType.MAP, NullableMapVector.class);
+      mapVector = addOrGet(MinorType.MAP, MapVector.class);
       if (internalMap.size() > vectorCount) {
         mapVector.allocateNew();
         if (callBack != null) {
@@ -195,12 +195,12 @@ public class UnionVector implements FieldVector {
       <#assign lowerCaseName = name?lower_case/>
       <#if !minor.typeParams?? >
 
-  private Nullable${name}Vector ${uncappedName}Vector;
+  private ${name}Vector ${uncappedName}Vector;
 
-  public Nullable${name}Vector get${name}Vector() {
+  public ${name}Vector get${name}Vector() {
     if (${uncappedName}Vector == null) {
       int vectorCount = internalMap.size();
-      ${uncappedName}Vector = addOrGet(MinorType.${name?upper_case}, Nullable${name}Vector.class);
+      ${uncappedName}Vector = addOrGet(MinorType.${name?upper_case}, ${name}Vector.class);
       if (internalMap.size() > vectorCount) {
         ${uncappedName}Vector.allocateNew();
         if (callBack != null) {
diff --git a/java/vector/src/main/codegen/templates/VariableLengthVectors.java b/java/vector/src/main/codegen/templates/VariableLengthVectors.java
deleted file mode 100644
index 3934e74..0000000
--- a/java/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ /dev/null
@@ -1,677 +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.
- */
-
-import java.lang.Override;
-
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.vector.BaseDataValueVector;
-import org.apache.drill.exec.vector.BaseValueVector;
-import org.apache.drill.exec.vector.VariableWidthVector;
-
-<@pp.dropOutputFile />
-<#list vv.types as type>
-<#list type.minor as minor>
-
-<#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
-<#assign className = "${minor.class}Vector" />
-
-<#if type.major == "VarLen">
-<@pp.changeOutputFile name="/org/apache/arrow/vector/${minor.class}Vector.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.arrow.vector;
-
-<#include "/@includes/vv_imports.ftl" />
-
-/**
- * ${minor.class}Vector implements a vector of variable width values.  Elements in the vector
- * are accessed by position from the logical start of the vector.  A fixed width offsetVector
- * is used to convert an element's position to it's offset from the start of the (0-based)
- * ArrowBuf.  Size is inferred by adjacent elements.
- *   The width of each element is ${type.width} byte(s)
- *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
- *
- * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
- */
-public final class ${className} extends BaseDataValueVector implements VariableWidthVector{
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class);
-
-  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
-  private static final int INITIAL_BYTE_COUNT = 4096 * DEFAULT_RECORD_BYTE_COUNT;
-  private static final int MIN_BYTE_COUNT = 4096;
-
-  public final static String OFFSETS_VECTOR_NAME = "$offsets$";
-  final UInt${type.width}Vector offsetVector = new UInt${type.width}Vector(OFFSETS_VECTOR_NAME, allocator);
-
-  private final Accessor accessor;
-  private final Mutator mutator;
-
-  private final UInt${type.width}Vector.Accessor oAccessor;
-
-  private int allocationSizeInBytes = INITIAL_BYTE_COUNT;
-  private int allocationMonitor = 0;
-
-  <#if minor.typeParams??>
-     <#list minor.typeParams as typeParam>
-  private final ${typeParam.type} ${typeParam.name};
-    </#list>
-
-  public ${className}(String name, BufferAllocator allocator<#list minor.typeParams as typeParam>, ${typeParam.type} ${typeParam.name}</#list>) {
-    super(name, allocator);
-    this.oAccessor = offsetVector.getAccessor();
-    this.accessor = new Accessor();
-    this.mutator = new Mutator();
-    <#list minor.typeParams as typeParam>
-    this.${typeParam.name} = ${typeParam.name};
-    </#list>
-  }
-  <#else>
-  public ${className}(String name, BufferAllocator allocator) {
-    super(name, allocator);
-    this.oAccessor = offsetVector.getAccessor();
-    this.accessor = new Accessor();
-    this.mutator = new Mutator();
-  }
-  </#if>
-
-  @Override
-  public Field getField() {
-    throw new UnsupportedOperationException("internal vector");
-  }
-
-  @Override
-  public MinorType getMinorType() {
-    return MinorType.${minor.class?upper_case};
-  }
-
-  @Override
-  public FieldReader getReader(){
-    throw new UnsupportedOperationException("internal vector");
-  }
-
-  @Override
-  public int getBufferSize(){
-    if (getAccessor().getValueCount() == 0) {
-      return 0;
-    }
-    return offsetVector.getBufferSize() + data.writerIndex();
-  }
-
-  @Override
-  public int getBufferSizeFor(final int valueCount) {
-    if (valueCount == 0) {
-      return 0;
-    }
-
-    final int idx = offsetVector.getAccessor().get(valueCount);
-    return offsetVector.getBufferSizeFor(valueCount + 1) + idx;
-  }
-
-  @Override
-  public ArrowBuf getValidityBuffer() {
-    /* this operation is not supported for non-nullable vectors */
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrowBuf getDataBuffer() {
-    /* we are not throwing away getBuffer() of BaseDataValueVector so use it wherever applicable */
-    return getBuffer();
-  }
-
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    /* dataBuffer associated with the underlying offsetVector */
-    return offsetVector.getDataBuffer();
-  }
-
-  @Override
-  public int getValueCapacity(){
-    return Math.max(offsetVector.getValueCapacity() - 1, 0);
-  }
-
-  @Override
-  public int getByteCapacity(){
-    return data.capacity();
-  }
-
-  @Override
-  public int getCurrentSizeInBytes() {
-    return offsetVector.getAccessor().get(getAccessor().getValueCount());
-  }
-
-  /**
-   * Return the number of bytes contained in the current var len byte vector.
-   * @return the number of bytes contained in the current var len byte vector
-   */
-  public int getVarByteLength(){
-    final int valueCount = getAccessor().getValueCount();
-    if(valueCount == 0) {
-      return 0;
-    }
-    return offsetVector.getAccessor().get(valueCount);
-  }
-
-  @Override
-  public void clear() {
-    super.clear();
-    offsetVector.clear();
-  }
-
-  @Override
-  public ArrowBuf[] getBuffers(boolean clear) {
-    final ArrowBuf[] buffers = ObjectArrays.concat(offsetVector.getBuffers(false), super.getBuffers(false), ArrowBuf.class);
-    if (clear) {
-      // does not make much sense but we have to retain buffers even when clear is set. refactor this interface.
-      for (final ArrowBuf buffer:buffers) {
-        buffer.retain(1);
-      }
-      clear();
-    }
-    return buffers;
-  }
-
-  public long getOffsetAddr(){
-    return offsetVector.getDataBuffer().memoryAddress();
-  }
-
-  public UInt${type.width}Vector getOffsetVector(){
-    return offsetVector;
-  }
-
-  @Override
-  public TransferPair getTransferPair(BufferAllocator allocator){
-        return new TransferImpl(name, allocator);
-  }
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
-    return new TransferImpl(ref, allocator);
-  }
-
-  @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((${className}) to);
-  }
-
-  public void transferTo(${className} target){
-    target.clear();
-    this.offsetVector.transferTo(target.offsetVector);
-    target.data = data.transferOwnership(target.allocator).buffer;
-    target.data.writerIndex(data.writerIndex());
-    clear();
-  }
-
-  public void splitAndTransferTo(int startIndex, int length, ${className} target) {
-    UInt${type.width}Vector.Accessor offsetVectorAccessor = this.offsetVector.getAccessor();
-    final int startPoint = offsetVectorAccessor.get(startIndex);
-    final int sliceLength = offsetVectorAccessor.get(startIndex + length) - startPoint;
-    target.clear();
-    target.offsetVector.allocateNew(length + 1);
-    offsetVectorAccessor = this.offsetVector.getAccessor();
-    final UInt4Vector.Mutator targetOffsetVectorMutator = target.offsetVector.getMutator();
-    for (int i = 0; i < length + 1; i++) {
-      targetOffsetVectorMutator.set(i, offsetVectorAccessor.get(startIndex + i) - startPoint);
-    }
-    target.data = data.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer;
-    target.getMutator().setValueCount(length);
-}
-
-  protected void copyFrom(int fromIndex, int thisIndex, ${className} from){
-    final UInt4Vector.Accessor fromOffsetVectorAccessor = from.offsetVector.getAccessor();
-    final int start = fromOffsetVectorAccessor.get(fromIndex);
-    final int end = fromOffsetVectorAccessor.get(fromIndex + 1);
-    final int len = end - start;
-
-    final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width});
-    from.data.getBytes(start, data, outputStart, len);
-    offsetVector.data.set${(minor.javaType!type.javaType)?cap_first}( (thisIndex+1) * ${type.width}, outputStart + len);
-  }
-
-  public boolean copyFromSafe(int fromIndex, int thisIndex, ${className} from){
-    final UInt${type.width}Vector.Accessor fromOffsetVectorAccessor = from.offsetVector.getAccessor();
-    final int start = fromOffsetVectorAccessor.get(fromIndex);
-    final int end =   fromOffsetVectorAccessor.get(fromIndex + 1);
-    final int len = end - start;
-    final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(thisIndex * ${type.width});
-
-    while(data.capacity() < outputStart + len) {
-        reAlloc();
-    }
-
-    offsetVector.getMutator().setSafe(thisIndex + 1, outputStart + len);
-    from.data.getBytes(start, data, outputStart, len);
-    return true;
-  }
-
-  private class TransferImpl implements TransferPair{
-    ${className} to;
-
-    public TransferImpl(String name, BufferAllocator allocator){
-      to = new ${className}(name, allocator<#if minor.typeParams??><#list minor.typeParams as typeParam>,  ${className}.this.${typeParam.name}</#list></#if>);
-    }
-
-    public TransferImpl(${className} to){
-      this.to = to;
-    }
-
-    @Override
-    public ${className} getTo(){
-      return to;
-    }
-
-    @Override
-    public void transfer(){
-      transferTo(to);
-    }
-
-    @Override
-    public void splitAndTransfer(int startIndex, int length) {
-      splitAndTransferTo(startIndex, length, to);
-    }
-
-    @Override
-    public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, ${className}.this);
-    }
-  }
-
-  @Override
-  public void setInitialCapacity(final int valueCount) {
-    final long size = 1L * valueCount * ${type.width};
-    if (size > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
-    }
-    allocationSizeInBytes = (int)size;
-    offsetVector.setInitialCapacity(valueCount + 1);
-  }
-
-  @Override
-  public void allocateNew() {
-    if(!allocateNewSafe()){
-      throw new OutOfMemoryException("Failure while allocating buffer.");
-    }
-  }
-
-  @Override
-  public boolean allocateNewSafe() {
-    long curAllocationSize = allocationSizeInBytes;
-    if (allocationMonitor > 10) {
-      curAllocationSize = Math.max(MIN_BYTE_COUNT, curAllocationSize / 2);
-      allocationMonitor = 0;
-    } else if (allocationMonitor < -2) {
-      curAllocationSize = curAllocationSize * 2L;
-      allocationMonitor = 0;
-    }
-
-    if (curAllocationSize > MAX_ALLOCATION_SIZE) {
-      return false;
-    }
-
-    clear();
-    /* Boolean to keep track if all the memory allocations were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    try {
-      final int requestedSize = (int)curAllocationSize;
-      data = allocator.buffer(requestedSize);
-      allocationSizeInBytes = requestedSize;
-      offsetVector.allocateNew();
-    } catch (OutOfMemoryException e) {
-      clear();
-      return false;
-    }
-    data.readerIndex(0);
-    offsetVector.zeroVector();
-    return true;
-  }
-
-  @Override
-  public void allocateNew(int totalBytes, int valueCount) {
-    clear();
-    assert totalBytes >= 0;
-    try {
-      data = allocator.buffer(totalBytes);
-      offsetVector.allocateNew(valueCount + 1);
-    } catch (RuntimeException e) {
-      clear();
-      throw e;
-    }
-    data.readerIndex(0);
-    allocationSizeInBytes = totalBytes;
-    offsetVector.zeroVector();
-  }
-
-  @Override
-  public void reset() {
-    allocationSizeInBytes = INITIAL_BYTE_COUNT;
-    allocationMonitor = 0;
-    data.readerIndex(0);
-    offsetVector.zeroVector();
-    super.reset();
-  }
-
-  public void reAlloc() {
-    long baseSize = allocationSizeInBytes;
-    final int currentBufferCapacity = data.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. Max allowed buffer size is reached.");
-    }
-
-    final ArrowBuf newBuf = allocator.buffer((int)newAllocationSize);
-    newBuf.setBytes(0, data, 0, currentBufferCapacity);
-    data.release();
-    data = newBuf;
-    allocationSizeInBytes = (int)newAllocationSize;
-  }
-
-  public void decrementAllocationMonitor() {
-    if (allocationMonitor > 0) {
-      allocationMonitor = 0;
-    }
-    --allocationMonitor;
-  }
-
-  private void incrementAllocationMonitor() {
-    ++allocationMonitor;
-  }
-
-  @Override
-  public Accessor getAccessor(){
-    return accessor;
-  }
-
-  @Override
-  public Mutator getMutator() {
-    return mutator;
-  }
-
-  public final class Accessor extends BaseValueVector.BaseAccessor implements VariableWidthAccessor {
-    final UInt${type.width}Vector.Accessor oAccessor = offsetVector.getAccessor();
-    public long getStartEnd(int index){
-      return oAccessor.getTwoAsLong(index);
-    }
-
-    public byte[] get(int index) {
-      assert index >= 0;
-      final int startIdx = oAccessor.get(index);
-      final int length = oAccessor.get(index + 1) - startIdx;
-      assert length >= 0;
-      final byte[] dst = new byte[length];
-      data.getBytes(startIdx, dst, 0, length);
-      return dst;
-    }
-
-    @Override
-    public int getValueLength(int index) {
-      final UInt${type.width}Vector.Accessor offsetVectorAccessor = offsetVector.getAccessor();
-      return offsetVectorAccessor.get(index + 1) - offsetVectorAccessor.get(index);
-    }
-
-    public void get(int index, ${minor.class}Holder holder){
-      holder.start = oAccessor.get(index);
-      holder.end = oAccessor.get(index + 1);
-      holder.buffer = data;
-    }
-
-    public void get(int index, Nullable${minor.class}Holder holder){
-      holder.isSet = 1;
-      holder.start = oAccessor.get(index);
-      holder.end = oAccessor.get(index + 1);
-      holder.buffer = data;
-    }
-
-    <#switch minor.class>
-      <#case "VarChar">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      Text text = new Text();
-      text.set(get(index));
-      return text;
-    }
-      <#break>
-      <#case "Decimal">
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return new BigDecimal(new BigInteger(get(index)), scale);
-    }
-      <#break>
-      <#default>
-    @Override
-    public ${friendlyType} getObject(int index) {
-      return get(index);
-    }
-    </#switch>
-
-    @Override
-    public int getValueCount() {
-      return Math.max(offsetVector.getAccessor().getValueCount()-1, 0);
-    }
-
-    @Override
-    public boolean isNull(int index){
-      return false;
-    }
-
-    public UInt${type.width}Vector getOffsetVector(){
-      return offsetVector;
-    }
-  }
-
-  /**
-   * Mutable${minor.class} implements a vector of variable width values.  Elements in the vector
-   * are accessed by position from the logical start of the vector.  A fixed width offsetVector
-   * is used to convert an element's position to it's offset from the start of the (0-based)
-   * ArrowBuf.  Size is inferred by adjacent elements.
-   *   The width of each element is ${type.width} byte(s)
-   *   The equivalent Java primitive is '${minor.javaType!type.javaType}'
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public final class Mutator extends BaseValueVector.BaseMutator implements VariableWidthVector.VariableWidthMutator {
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     */
-    protected void set(int index, byte[] bytes) {
-      assert index >= 0;
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      offsetVector.getMutator().set(index + 1, currentOffset + bytes.length);
-      data.setBytes(currentOffset, bytes, 0, bytes.length);
-    }
-
-    public void setSafe(int index, byte[] bytes) {
-      assert index >= 0;
-
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      while (data.capacity() < currentOffset + bytes.length) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, currentOffset + bytes.length);
-      data.setBytes(currentOffset, bytes, 0, bytes.length);
-    }
-
-    /**
-     * Set the variable length element at the specified index to the supplied byte array.
-     *
-     * @param index   position of the bit to set
-     * @param bytes   array of bytes to write
-     * @param start   start index of bytes to write
-     * @param length  length of bytes to write
-     */
-    protected void set(int index, byte[] bytes, int start, int length) {
-      assert index >= 0;
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      offsetVector.getMutator().set(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, bytes, start, length);
-    }
-
-    public void setSafe(int index, ByteBuffer bytes, int start, int length) {
-      assert index >= 0;
-
-      int currentOffset = offsetVector.getAccessor().get(index);
-
-      while (data.capacity() < currentOffset + length) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, bytes, start, length);
-    }
-
-    public void setSafe(int index, byte[] bytes, int start, int length) {
-      assert index >= 0;
-
-      final int currentOffset = offsetVector.getAccessor().get(index);
-
-      while (data.capacity() < currentOffset + length) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, bytes, start, length);
-    }
-
-    @Override
-    public void setValueLengthSafe(int index, int length) {
-      final int offset = offsetVector.getAccessor().get(index);
-      while(data.capacity() < offset + length ) {
-        reAlloc();
-      }
-      offsetVector.getMutator().setSafe(index + 1, offsetVector.getAccessor().get(index) + length);
-    }
-
-
-    public void setSafe(int index, int start, int end, ArrowBuf buffer){
-      final int len = end - start;
-      final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
-      }
-
-      offsetVector.getMutator().setSafe( index+1,  outputStart + len);
-      buffer.getBytes(start, data, outputStart, len);
-    }
-
-    public void setSafe(int index, Nullable${minor.class}Holder holder){
-      assert holder.isSet == 1;
-
-      final int start = holder.start;
-      final int end =   holder.end;
-      final int len = end - start;
-
-      int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
-      }
-
-      holder.buffer.getBytes(start, data, outputStart, len);
-      offsetVector.getMutator().setSafe( index+1,  outputStart + len);
-    }
-
-    public void setSafe(int index, ${minor.class}Holder holder){
-      final int start = holder.start;
-      final int end =   holder.end;
-      final int len = end - start;
-      final int outputStart = offsetVector.data.get${(minor.javaType!type.javaType)?cap_first}(index * ${type.width});
-
-      while(data.capacity() < outputStart + len) {
-        reAlloc();
-      }
-
-      holder.buffer.getBytes(start, data, outputStart, len);
-      offsetVector.getMutator().setSafe( index+1,  outputStart + len);
-    }
-
-    protected void set(int index, int start, int length, ArrowBuf buffer){
-      assert index >= 0;
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      offsetVector.getMutator().set(index + 1, currentOffset + length);
-      final ArrowBuf bb = buffer.slice(start, length);
-      data.setBytes(currentOffset, bb);
-    }
-
-    protected void set(int index, Nullable${minor.class}Holder holder){
-      final int length = holder.end - holder.start;
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      offsetVector.getMutator().set(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, holder.buffer, holder.start, length);
-    }
-
-    protected void set(int index, ${minor.class}Holder holder){
-      final int length = holder.end - holder.start;
-      final int currentOffset = offsetVector.getAccessor().get(index);
-      offsetVector.getMutator().set(index + 1, currentOffset + length);
-      data.setBytes(currentOffset, holder.buffer, holder.start, length);
-    }
-
-    @Override
-    public void setValueCount(int valueCount) {
-      if (valueCount == 0) {
-        // if no values in vector, don't try to retrieve the current value count.
-        offsetVector.getMutator().setValueCount(0);
-      } else {
-        final int currentByteCapacity = getByteCapacity();
-        final int idx = offsetVector.getAccessor().get(valueCount);
-        data.writerIndex(idx);
-        if (currentByteCapacity > idx * 2) {
-          incrementAllocationMonitor();
-        } else if (allocationMonitor > 0) {
-          allocationMonitor = 0;
-        }
-        VectorTrimmer.trim(data, idx);
-        offsetVector.getMutator().setValueCount(valueCount+1);
-      }
-    }
-
-    @Override
-    public void generateTestData(int size){
-      boolean even = true;
-      <#switch minor.class>
-      <#case "Var16Char">
-      final java.nio.charset.Charset charset = Charsets.UTF_16;
-      <#break>
-      <#case "VarChar">
-      <#default>
-      final java.nio.charset.Charset charset = Charsets.UTF_8;
-      </#switch>
-      final byte[] evenValue = new String("aaaaa").getBytes(charset);
-      final byte[] oddValue = new String("bbbbbbbbbb").getBytes(charset);
-      for(int i =0; i < size; i++, even = !even){
-        set(i, even ? evenValue : oddValue);
-        }
-      setValueCount(size);
-    }
-  }
-}
-
-</#if> <#-- type.major -->
-</#list>
-</#list>
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
similarity index 97%
rename from java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
index f82077f..cc05690 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableFixedWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseFixedWidthVector.java
@@ -36,12 +36,12 @@ import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * BaseNullableFixedWidthVector provides an abstract interface for
+ * BaseFixedWidthVector provides an abstract interface for
  * implementing vectors of fixed width values. The vectors are nullable
  * implying that zero or more elements in the vector could be NULL.
  */
-public abstract class BaseNullableFixedWidthVector extends BaseValueVector
-        implements FixedWidthVector, FieldVector, NullableVectorDefinitionSetter {
+public abstract class BaseFixedWidthVector extends BaseValueVector
+        implements FixedWidthVector, FieldVector, VectorDefinitionSetter {
   private final byte typeWidth;
 
   protected int valueAllocationSizeInBytes;
@@ -53,7 +53,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
   protected ArrowBuf valueBuffer;
   protected int valueCount;
 
-  public BaseNullableFixedWidthVector(final String name, final BufferAllocator allocator,
+  public BaseFixedWidthVector(final String name, final BufferAllocator allocator,
                                       FieldType fieldType, final byte typeWidth) {
     super(name, allocator);
     this.typeWidth = typeWidth;
@@ -66,7 +66,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
       valueAllocationSizeInBytes = INITIAL_VALUE_ALLOCATION * typeWidth;
       validityAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
     } else {
-      /* specialized handling for NullableBitVector */
+      /* specialized handling for BitVector */
       valueAllocationSizeInBytes = getValidityBufferSizeFromCount(INITIAL_VALUE_ALLOCATION);
       validityAllocationSizeInBytes = valueAllocationSizeInBytes;
     }
@@ -311,7 +311,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
     long valueBufferSize = valueCount * typeWidth;
     long validityBufferSize = getValidityBufferSizeFromCount(valueCount);
     if (typeWidth == 0) {
-      /* specialized handling for NullableBitVector */
+      /* specialized handling for BitVector */
       valueBufferSize = validityBufferSize;
     }
 
@@ -556,7 +556,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
     } else {
       validityBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
       if (typeWidth == 0) {
-        /* specialized handling for NullableBitVector */
+        /* specialized handling for BitVector */
         valueBuffer.writerIndex(getValidityBufferSizeFromCount(valueCount));
       } else {
         valueBuffer.writerIndex(valueCount * typeWidth);
@@ -600,7 +600,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
    * for accounting and management purposes.
    * @param target destination vector for transfer
    */
-  public void transferTo(BaseNullableFixedWidthVector target) {
+  public void transferTo(BaseFixedWidthVector target) {
     compareTypes(target, "transferTo");
     target.clear();
     target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
@@ -617,7 +617,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
    * @param target destination vector
    */
   public void splitAndTransferTo(int startIndex, int length,
-                                 BaseNullableFixedWidthVector target) {
+                                 BaseFixedWidthVector target) {
     compareTypes(target, "splitAndTransferTo");
     target.clear();
     splitAndTransferValidityBuffer(startIndex, length, target);
@@ -629,7 +629,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
    * Data buffer can always be split and transferred using slicing.
    */
   private void splitAndTransferValueBuffer(int startIndex, int length,
-                                           BaseNullableFixedWidthVector target) {
+                                           BaseFixedWidthVector target) {
     final int startPoint = startIndex * typeWidth;
     final int sliceLength = length * typeWidth;
     target.valueBuffer = valueBuffer.slice(startPoint, sliceLength).transferOwnership(target.allocator).buffer;
@@ -640,7 +640,7 @@ public abstract class BaseNullableFixedWidthVector extends BaseValueVector
    * the starting position of the source index.
    */
   private void splitAndTransferValidityBuffer(int startIndex, int length,
-                                              BaseNullableFixedWidthVector target) {
+                                              BaseFixedWidthVector target) {
     assert startIndex + length <= valueCount;
     int firstByteSource = BitVectorHelper.byteIndex(startIndex);
     int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
similarity index 98%
rename from java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
index b9e5442..b515586 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseNullableVariableWidthVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseVariableWidthVector.java
@@ -24,7 +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.complex.MapVector;
 import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
@@ -37,8 +37,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-public abstract class BaseNullableVariableWidthVector extends BaseValueVector
-        implements VariableWidthVector, FieldVector, NullableVectorDefinitionSetter {
+public abstract class BaseVariableWidthVector extends BaseValueVector
+        implements VariableWidthVector, FieldVector, VectorDefinitionSetter {
   private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
   private static final int INITIAL_BYTE_COUNT = INITIAL_VALUE_ALLOCATION * DEFAULT_RECORD_BYTE_COUNT;
 
@@ -57,7 +57,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
   protected final Field field;
   private boolean cleared;
 
-  public BaseNullableVariableWidthVector(final String name, final BufferAllocator allocator,
+  public BaseVariableWidthVector(final String name, final BufferAllocator allocator,
                                          FieldType fieldType) {
     super(name, allocator);
     valueAllocationSizeInBytes = INITIAL_BYTE_COUNT;
@@ -657,7 +657,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    * for accounting and management purposes.
    * @param target destination vector for transfer
    */
-  public void transferTo(BaseNullableVariableWidthVector target) {
+  public void transferTo(BaseVariableWidthVector target) {
     compareTypes(target, "transferTo");
     target.clear();
     target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
@@ -678,7 +678,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    * @param target destination vector
    */
   public void splitAndTransferTo(int startIndex, int length,
-                                 BaseNullableVariableWidthVector target) {
+                                 BaseVariableWidthVector target) {
     compareTypes(target, "splitAndTransferTo");
     target.clear();
     splitAndTransferValidityBuffer(startIndex, length, target);
@@ -697,7 +697,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    * 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) {
+  private void splitAndTransferOffsetBuffer(int startIndex, int length, BaseVariableWidthVector target) {
     final int start = offsetBuffer.getInt(startIndex * OFFSET_WIDTH);
     final int end = offsetBuffer.getInt((startIndex + length) * OFFSET_WIDTH);
     final int dataLength = end - start;
@@ -713,7 +713,7 @@ public abstract class BaseNullableVariableWidthVector extends BaseValueVector
    * Transfer the validity.
    */
   private void splitAndTransferValidityBuffer(int startIndex, int length,
-                                              BaseNullableVariableWidthVector target) {
+                                              BaseVariableWidthVector target) {
     assert startIndex + length <= valueCount;
     int firstByteSource = BitVectorHelper.byteIndex(startIndex);
     int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
index 8d44c92..ccf0c30 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableBigIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BigIntVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableBigIntVector implements a fixed width vector (8 bytes) of
+ * BigIntVector implements a fixed width vector (8 bytes) of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableBigIntVector extends BaseNullableFixedWidthVector {
+public class BigIntVector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableBigIntVector. This doesn't allocate any memory for
+   * Instantiate a BigIntVector. 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) {
+  public BigIntVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.BIGINT.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableBigIntVector. This doesn't allocate any memory for
+   * Instantiate a BigIntVector. 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) {
+  public BigIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new BigIntReaderImpl(NullableBigIntVector.this);
+    reader = new BigIntReaderImpl(BigIntVector.this);
   }
 
   /**
@@ -136,21 +136,21 @@ public class NullableBigIntVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableBigIntVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, BigIntVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableBigIntVector)} except that
+   * Same as {@link #copyFrom(int, int, BigIntVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, BigIntVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -329,22 +329,22 @@ public class NullableBigIntVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableBigIntVector) to);
+    return new TransferImpl((BigIntVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableBigIntVector to;
+    BigIntVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableBigIntVector(ref, field.getFieldType(), allocator);
+      to = new BigIntVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableBigIntVector to) {
+    public TransferImpl(BigIntVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableBigIntVector getTo() {
+    public BigIntVector getTo() {
       return to;
     }
 
@@ -360,7 +360,7 @@ public class NullableBigIntVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableBigIntVector.this);
+      to.copyFromSafe(fromIndex, toIndex, BigIntVector.this);
     }
   }
 }
\ No newline at end of file
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 26c8170..b6f7323 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
@@ -18,342 +18,469 @@
 
 package org.apache.arrow.vector;
 
+import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.memory.BaseAllocator;
-import org.apache.arrow.memory.OutOfMemoryException;
+import org.apache.arrow.vector.complex.impl.BitReaderImpl;
 import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.holders.BitHolder;
 import org.apache.arrow.vector.holders.NullableBitHolder;
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
-import org.apache.arrow.vector.types.Types.MinorType;
-import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
-import io.netty.buffer.ArrowBuf;
-
 /**
- * Bit implements a vector of bit-width values. Elements in the vector are accessed by position from the logical start
- * of the vector. The width of each element is 1 bit. The equivalent Java primitive is an int containing the value '0'
- * or '1'.
+ * BitVector implements a fixed width (1 bit) vector of
+ * boolean values which could be null. Each value in the vector corresponds
+ * to a single bit in the underlying data stream backing the vector.
  */
-public final class BitVector extends BaseDataValueVector implements FixedWidthVector {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
-
-  private final Accessor accessor = new Accessor();
-  private final Mutator mutator = new Mutator();
-
-  int valueCount;
-  private int allocationSizeInBytes = getSizeFromCount(INITIAL_VALUE_ALLOCATION);
-  private int allocationMonitor = 0;
+public class BitVector extends BaseFixedWidthVector {
+  private final FieldReader reader;
 
+  /**
+   * Instantiate a BitVector. This doesn't allocate any memory for
+   * the data in vector.
+   *
+   * @param name      name of the vector
+   * @param allocator allocator for memory management.
+   */
   public BitVector(String name, BufferAllocator allocator) {
-    super(name, allocator);
+    this(name, FieldType.nullable(Types.MinorType.BIT.getType()),
+            allocator);
   }
 
-  @Override
-  public void load(ArrowFieldNode fieldNode, ArrowBuf data) {
-    // When the vector is all nulls or all defined, the content of the buffer can be omitted
-    if (data.readableBytes() == 0 && fieldNode.getLength() != 0) {
-      int count = fieldNode.getLength();
-      allocateNew(count);
-      int n = getSizeFromCount(count);
-      if (fieldNode.getNullCount() == 0) {
-        // all defined
-        // create an all 1s buffer
-        // set full bytes
-        int fullBytesCount = count / 8;
-        for (int i = 0; i < fullBytesCount; ++i) {
-          this.data.setByte(i, 0xFF);
-        }
-        int remainder = count % 8;
-        // set remaining bits
-        if (remainder > 0) {
-          byte bitMask = (byte) (0xFFL >>> ((8 - remainder) & 7));
-          this.data.setByte(fullBytesCount, bitMask);
-        }
-      } else if (fieldNode.getNullCount() == fieldNode.getLength()) {
-        // all null
-        // create an all 0s buffer
-        zeroVector();
-      } else {
-        throw new IllegalArgumentException("The buffer can be empty only if there's no data or it's all null or all defined");
-      }
-      this.data.writerIndex(n);
-    } else {
-      super.load(fieldNode, data);
-    }
-    this.valueCount = fieldNode.getLength();
+  /**
+   * Instantiate a BitVector. 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 BitVector(String name, FieldType fieldType, BufferAllocator allocator) {
+    super(name, allocator, fieldType, (byte) 0);
+    reader = new BitReaderImpl(BitVector.this);
   }
 
+  /**
+   * Get a reader that supports reading values from this vector
+   *
+   * @return Field Reader for this vector
+   */
   @Override
-  public Field getField() {
-    throw new UnsupportedOperationException("internal vector");
+  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 MinorType getMinorType() {
-    return MinorType.BIT;
+  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 FieldReader getReader() {
-    throw new UnsupportedOperationException("internal vector");
+  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 getBufferSize() {
-    return getSizeFromCount(valueCount);
+  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 valueCount) {
-    return getSizeFromCount(valueCount);
+  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 ArrowBuf getValidityBuffer() {
-    /* this operation is not supported for non-nullable vectors */
-    throw new  UnsupportedOperationException();
+  public int getBufferSize() {
+    return getBufferSizeFor(valueCount);
   }
 
-  @Override
-  public ArrowBuf getDataBuffer() {
-    /* we are not throwing away getBuffer() of BaseDataValueVector so use it wherever applicable */
-    return getBuffer();
-  }
+  /**
+   * 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,
+                                 BaseFixedWidthVector target) {
+    compareTypes(target, "splitAndTransferTo");
+    target.clear();
+    target.validityBuffer = splitAndTransferBuffer(startIndex, length, target,
+            validityBuffer, target.validityBuffer);
+    target.valueBuffer = splitAndTransferBuffer(startIndex, length, target,
+            valueBuffer, target.valueBuffer);
 
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    /* this operation is not supported for fixed-width vectors */
-    throw new UnsupportedOperationException();
+    target.setValueCount(length);
   }
 
-  int getSizeFromCount(int valueCount) {
-    return (int) Math.ceil(valueCount / 8.0);
-  }
+  private ArrowBuf splitAndTransferBuffer(int startIndex, int length,
+                                          BaseFixedWidthVector 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;
 
-  @Override
-  public int getValueCapacity() {
-    return (int) Math.min((long) Integer.MAX_VALUE, data.capacity() * 8L);
-  }
+    if (length > 0) {
+      if (offset == 0) {
+            /* slice */
+        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());
+
+        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));
+        }
 
-  private int getByteIndex(int index) {
-    return (int) Math.floor(index / 8.0);
+            /* Copying the last piece is done in the following manner:
+             * if the source vector has 1 or more bytes remaining, we copy
+             * the last piece as a byte formed by shifting data
+             * from the current byte and the next byte.
+             *
+             * if the source vector has no more bytes remaining
+             * (we are at the last byte), we copy the last piece as a byte
+             * by shifting data from the current byte.
+             */
+        if ((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(sourceBuffer,
+                  firstByteSource + byteSizeTarget, offset);
+
+          destBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+        } else {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          destBuffer.setByte(byteSizeTarget - 1, b1);
+        }
+      }
+    }
+
+    return destBuffer;
   }
 
-  @Override
-  public void setInitialCapacity(final int valueCount) {
-    allocationSizeInBytes = getSizeFromCount(valueCount);
+
+  /******************************************************************
+   *                                                                *
+   *          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));
   }
 
-  @Override
-  public void allocateNew() {
-    if (!allocateNewSafe()) {
-      throw new OutOfMemoryException();
+  /**
+   * 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);
   }
 
-  @Override
-  public boolean allocateNewSafe() {
-    long curAllocationSize = allocationSizeInBytes;
-    if (allocationMonitor > 10) {
-      curAllocationSize = Math.max(8, allocationSizeInBytes / 2);
-      allocationMonitor = 0;
-    } else if (allocationMonitor < -2) {
-      curAllocationSize = allocationSizeInBytes * 2L;
-      allocationMonitor = 0;
+  /**
+   * 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);
+  }
 
-    try {
-      allocateBytes(curAllocationSize);
-    } catch (OutOfMemoryException ex) {
-      return false;
+  /**
+   * 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);
     }
-    return true;
   }
 
-  @Override
-  public void reset() {
-    valueCount = 0;
-    allocationSizeInBytes = getSizeFromCount(INITIAL_VALUE_ALLOCATION);
-    allocationMonitor = 0;
-    zeroVector();
-    super.reset();
+  /**
+   * 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, BitVector from) {
+    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
+    BitVectorHelper.setValidityBit(valueBuffer, thisIndex, from.getBit(fromIndex));
   }
 
   /**
-   * Allocate a new memory space for this vector. Must be called prior to using the ValueVector.
+   * Same as {@link #copyFrom(int, int, BitVector)} except that
+   * it handles the case when the capacity of the vector needs to be expanded
+   * before copy.
    *
-   * @param valueCount The number of values which can be contained within this vector.
+   * @param fromIndex position to copy from in source vector
+   * @param thisIndex position to copy to in this vector
+   * @param from      source vector
    */
-  @Override
-  public void allocateNew(int valueCount) {
-    final int size = getSizeFromCount(valueCount);
-    allocateBytes(size);
+  public void copyFromSafe(int fromIndex, int thisIndex, BitVector from) {
+    handleSafe(thisIndex);
+    copyFrom(fromIndex, thisIndex, from);
   }
 
-  private void allocateBytes(final long size) {
-    if (size > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
-    }
 
-    final int curSize = (int) size;
-    clear();
-    data = allocator.buffer(curSize);
-    zeroVector();
-    allocationSizeInBytes = curSize;
-  }
+  /******************************************************************
+   *                                                                *
+   *          vector value setter methods                           *
+   *                                                                *
+   ******************************************************************/
+
 
   /**
-   * Allocate new buffer with double capacity, and copy data into the new buffer. Replace vector's buffer with new buffer, and release old one
+   * Set the element at the given index to the given value.
+   *
+   * @param index position of element
+   * @param value value of element
    */
-  public void reAlloc() {
-    long baseSize  = allocationSizeInBytes;
-    final int currentBufferCapacity = data.capacity();
-    if (baseSize < (long)currentBufferCapacity) {
-      baseSize = (long)currentBufferCapacity;
+  public void set(int index, int value) {
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+    if (value != 0) {
+      BitVectorHelper.setValidityBitToOne(valueBuffer, index);
+    } else {
+      BitVectorHelper.setValidityBit(valueBuffer, index, 0);
     }
-    long newAllocationSize = baseSize * 2L;
-    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+  }
 
-    if (newAllocationSize > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed allocation size");
+  /**
+   * 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);
     }
+  }
 
-    final int curSize = (int) newAllocationSize;
-    final ArrowBuf newBuf = allocator.buffer(curSize);
-    newBuf.setZero(0, newBuf.capacity());
-    newBuf.setBytes(0, data, 0, currentBufferCapacity);
-    data.release();
-    data = newBuf;
-    allocationSizeInBytes = curSize;
+  /**
+   * 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);
+    }
   }
 
   /**
-   * {@inheritDoc}
+   * 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
    */
-  @Override
-  public void zeroVector() {
-    data.setZero(0, data.capacity());
+  public void setSafe(int index, int value) {
+    handleSafe(index);
+    set(index, value);
   }
 
-  public void copyFrom(int inIndex, int outIndex, BitVector from) {
-    this.mutator.set(outIndex, from.accessor.get(inIndex));
+  /**
+   * 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);
   }
 
-  public void copyFromSafe(int inIndex, int outIndex, BitVector from) {
-    if (outIndex >= this.getValueCapacity()) {
-      reAlloc();
-    }
-    copyFrom(inIndex, outIndex, from);
+  /**
+   * 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);
   }
 
-  @Override
-  public Mutator getMutator() {
-    return mutator;
+  /**
+   * 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);
   }
 
-  @Override
-  public Accessor getAccessor() {
-    return accessor;
+  /**
+   * 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);
+    }
   }
 
-  @Override
-  public TransferPair getTransferPair(BufferAllocator allocator) {
-    return new TransferImpl(name, allocator);
+  /**
+   * 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((BitVector) to);
   }
 
-
-  public void transferTo(BitVector target) {
-    target.clear();
-    target.data = data.transferOwnership(target.allocator).buffer;
-    target.valueCount = valueCount;
-    clear();
-  }
-
-  public void splitAndTransferTo(int startIndex, int length, BitVector target) {
-    assert startIndex + length <= valueCount;
-    int firstByteSource = getByteIndex(startIndex);
-    int lastByteSource = getByteIndex(valueCount - 1);
-    int byteSizeTarget = getSizeFromCount(length);
-    int offset = startIndex % 8;
-
-    if (length > 0) {
-      if (offset == 0) {
-        target.clear();
-        // slice
-        if (target.data != null) {
-          target.data.release();
-        }
-        target.data = data.slice(firstByteSource, byteSizeTarget);
-        target.data.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.clear();
-        target.allocateNew(byteSizeTarget * 8);
-
-        // TODO maybe do this one word at a time, rather than byte?
-
-        for (int i = 0; i < byteSizeTarget - 1; i++) {
-          byte b1 = getBitsFromCurrentByte(this.data, firstByteSource + i, offset);
-          byte b2 = getBitsFromNextByte(this.data, firstByteSource + i + 1, offset);
-
-          target.data.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 = getBitsFromCurrentByte(this.data, firstByteSource + byteSizeTarget - 1, offset);
-          byte b2 = getBitsFromNextByte(this.data, firstByteSource + byteSizeTarget, offset);
-
-          target.data.setByte(byteSizeTarget - 1, b1 + b2);
-        }
-        else {
-          byte b1 = getBitsFromCurrentByte(this.data, firstByteSource + byteSizeTarget - 1, offset);
-
-          target.data.setByte(byteSizeTarget - 1, b1);
-        }
-      }
-    }
-    target.getMutator().setValueCount(length);
-  }
-
-  private static byte getBitsFromCurrentByte(ArrowBuf data, int index, int offset) {
-    return (byte)((data.getByte(index) & 0xFF) >>> offset);
-  }
-
-  private static byte getBitsFromNextByte(ArrowBuf data, int index, int offset) {
-    return (byte)((data.getByte(index) << (8 - offset)));
-  }
-
   private class TransferImpl implements TransferPair {
     BitVector to;
 
-    public TransferImpl(String name, BufferAllocator allocator) {
-      this.to = new BitVector(name, allocator);
+    public TransferImpl(String ref, BufferAllocator allocator) {
+      to = new BitVector(ref, field.getFieldType(), allocator);
     }
 
     public TransferImpl(BitVector to) {
@@ -380,269 +507,4 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       to.copyFromSafe(fromIndex, toIndex, BitVector.this);
     }
   }
-
-  private void decrementAllocationMonitor() {
-    if (allocationMonitor > 0) {
-      allocationMonitor = 0;
-    }
-    --allocationMonitor;
-  }
-
-  private void incrementAllocationMonitor() {
-    ++allocationMonitor;
-  }
-
-  public class Accessor extends BaseAccessor {
-
-    /**
-     * Get the byte holding the desired bit, then mask all other bits. Iff the result is 0, the bit was not set.
-     *
-     * @param index position of the bit in the vector
-     * @return 1 if set, otherwise 0
-     */
-    public final int get(int index) {
-      int byteIndex = index >> 3;
-      byte b = data.getByte(byteIndex);
-      int bitIndex = index & 7;
-      return Long.bitCount(b & (1L << bitIndex));
-    }
-
-    @Override
-    public boolean isNull(int index) {
-      return false;
-    }
-
-    @Override
-    public final Boolean getObject(int index) {
-      return new Boolean(get(index) != 0);
-    }
-
-    @Override
-    public final int getValueCount() {
-      return valueCount;
-    }
-
-    public final void get(int index, BitHolder holder) {
-      holder.value = get(index);
-    }
-
-    public final void get(int index, NullableBitHolder holder) {
-      holder.isSet = 1;
-      holder.value = get(index);
-    }
-
-    /**
-     * Get the number nulls, this correspond to the number of bits set to 0 in the vector
-     *
-     * @return the number of bits set to 0
-     */
-    @Override
-    public final int getNullCount() {
-      int count = 0;
-      int sizeInBytes = getSizeFromCount(valueCount);
-
-      for (int i = 0; i < sizeInBytes; ++i) {
-        byte byteValue = data.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;
-    }
-  }
-
-  /**
-   * MutableBit implements a vector of bit-width values. Elements in the vector are accessed by position from the
-   * logical start of the vector. Values should be pushed onto the vector sequentially, but may be randomly accessed.
-   *
-   * NB: this class is automatically generated from ValueVectorTypes.tdd using FreeMarker.
-   */
-  public class Mutator extends BaseMutator {
-
-    private Mutator() {
-    }
-
-    /**
-     * Set the bit at the given index to the specified value.
-     *
-     * @param index position of the bit to set
-     * @param value value to set (either 1 or 0)
-     */
-    public final void set(int index, int value) {
-      int byteIndex = byteIndex(index);
-      int bitIndex = bitIndex(index);
-      byte currentByte = data.getByte(byteIndex);
-      byte bitMask = (byte) (1L << bitIndex);
-      if (value != 0) {
-        currentByte |= bitMask;
-      } else {
-        currentByte -= (bitMask & currentByte);
-      }
-      data.setByte(byteIndex, currentByte);
-    }
-
-    /**
-     * Set the bit at the given index to 1.
-     *
-     * @param index position of the bit to set
-     */
-    public final void setToOne(int index) {
-      int byteIndex = byteIndex(index);
-      int bitIndex = bitIndex(index);
-      byte currentByte = data.getByte(byteIndex);
-      byte bitMask = (byte) (1L << bitIndex);
-      currentByte |= bitMask;
-      data.setByte(byteIndex, currentByte);
-    }
-
-    /**
-     * set count bits to 1 in data starting at firstBitIndex
-     *
-     * @param firstBitIndex the index of the first bit to set
-     * @param count         the number of bits to set
-     */
-    public void setRangeToOne(int firstBitIndex, int count) {
-      int starByteIndex = byteIndex(firstBitIndex);
-      final int lastBitIndex = firstBitIndex + count;
-      final int endByteIndex = byteIndex(lastBitIndex);
-      final int startByteBitIndex = bitIndex(firstBitIndex);
-      final int endBytebitIndex = bitIndex(lastBitIndex);
-      if (count < 8 && starByteIndex == endByteIndex) {
-        // handles the case where we don't have a first and a last byte
-        byte bitMask = 0;
-        for (int i = startByteBitIndex; i < endBytebitIndex; ++i) {
-          bitMask |= (byte) (1L << i);
-        }
-        byte currentByte = data.getByte(starByteIndex);
-        currentByte |= bitMask;
-        data.setByte(starByteIndex, currentByte);
-      } else {
-        // fill in first byte (if it's not full)
-        if (startByteBitIndex != 0) {
-          byte currentByte = data.getByte(starByteIndex);
-          final byte bitMask = (byte) (0xFFL << startByteBitIndex);
-          currentByte |= bitMask;
-          data.setByte(starByteIndex, currentByte);
-          ++starByteIndex;
-        }
-
-        // fill in one full byte at a time
-        for (int i = starByteIndex; i < endByteIndex; i++) {
-          data.setByte(i, 0xFF);
-        }
-
-        // fill in the last byte (if it's not full)
-        if (endBytebitIndex != 0) {
-          final int byteIndex = byteIndex(lastBitIndex - endBytebitIndex);
-          byte currentByte = data.getByte(byteIndex);
-          final byte bitMask = (byte) (0xFFL >>> ((8 - endBytebitIndex) & 7));
-          currentByte |= bitMask;
-          data.setByte(byteIndex, currentByte);
-        }
-
-      }
-    }
-
-    /**
-     * @param absoluteBitIndex the index of the bit in the buffer
-     * @return the index of the byte containing that bit
-     */
-    private int byteIndex(int absoluteBitIndex) {
-      return absoluteBitIndex >> 3;
-    }
-
-    /**
-     * @param absoluteBitIndex the index of the bit in the buffer
-     * @return the index of the bit inside the byte
-     */
-    private int bitIndex(int absoluteBitIndex) {
-      return absoluteBitIndex & 7;
-    }
-
-    public final void set(int index, BitHolder holder) {
-      set(index, holder.value);
-    }
-
-    final void set(int index, NullableBitHolder holder) {
-      set(index, holder.value);
-    }
-
-    public void setSafe(int index, int value) {
-      while (index >= getValueCapacity()) {
-        reAlloc();
-      }
-      set(index, value);
-    }
-
-    public void setSafeToOne(int index) {
-      while (index >= getValueCapacity()) {
-        reAlloc();
-      }
-      setToOne(index);
-    }
-
-    public void setSafe(int index, BitHolder holder) {
-      while (index >= getValueCapacity()) {
-        reAlloc();
-      }
-      set(index, holder.value);
-    }
-
-    public void setSafe(int index, NullableBitHolder holder) {
-      while (index >= getValueCapacity()) {
-        reAlloc();
-      }
-      set(index, holder.value);
-    }
-
-    @Override
-    public final void setValueCount(int valueCount) {
-      int currentValueCapacity = getValueCapacity();
-      BitVector.this.valueCount = valueCount;
-      int idx = getSizeFromCount(valueCount);
-      while (valueCount > getValueCapacity()) {
-        reAlloc();
-      }
-      if (valueCount > 0 && currentValueCapacity > valueCount * 2) {
-        incrementAllocationMonitor();
-      } else if (allocationMonitor > 0) {
-        allocationMonitor = 0;
-      }
-      VectorTrimmer.trim(data, idx);
-    }
-
-    @Override
-    public final void generateTestData(int values) {
-      boolean even = true;
-      for (int i = 0; i < values; i++, even = !even) {
-        if (even) {
-          set(i, 1);
-        }
-      }
-      setValueCount(values);
-    }
-
-    public void generateTestDataAlt(int size) {
-      setValueCount(size);
-      boolean even = true;
-      final int valueCount = getAccessor().getValueCount();
-      for (int i = 0; i < valueCount; i++, even = !even) {
-        if (even) {
-          set(i, (byte) 1);
-        } else {
-          set(i, (byte) 0);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void clear() {
-    this.valueCount = 0;
-    super.clear();
-  }
-}
+}
\ 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/DateDayVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java
index bb14d1a..ed8956c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDateDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DateDayVector.java
@@ -30,35 +30,35 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableDateDayVector implements a fixed width (4 bytes) vector of
+ * DateDayVector implements a fixed width (4 bytes) vector of
  * date values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableDateDayVector extends BaseNullableFixedWidthVector {
+public class DateDayVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableDateDayVector. This doesn't allocate any memory for
+   * Instantiate a DateDayVector. 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) {
+  public DateDayVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.DATEDAY.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableDateDayVector. This doesn't allocate any memory for
+   * Instantiate a DateDayVector. 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) {
+  public DateDayVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new DateDayReaderImpl(NullableDateDayVector.this);
+    reader = new DateDayReaderImpl(DateDayVector.this);
   }
 
   /**
@@ -138,21 +138,21 @@ public class NullableDateDayVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableDateDayVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, DateDayVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableDateDayVector)} except that
+   * Same as {@link #copyFrom(int, int, DateDayVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, DateDayVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -331,22 +331,22 @@ public class NullableDateDayVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableDateDayVector) to);
+    return new TransferImpl((DateDayVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableDateDayVector to;
+    DateDayVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableDateDayVector(ref, field.getFieldType(), allocator);
+      to = new DateDayVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableDateDayVector to) {
+    public TransferImpl(DateDayVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableDateDayVector getTo() {
+    public DateDayVector getTo() {
       return to;
     }
 
@@ -362,7 +362,7 @@ public class NullableDateDayVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableDateDayVector.this);
+      to.copyFromSafe(fromIndex, toIndex, DateDayVector.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/DateMilliVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java
index 97205c3..f21b58f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDateMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DateMilliVector.java
@@ -32,35 +32,35 @@ import org.joda.time.LocalDateTimes;
 import org.slf4j.Logger;
 
 /**
- * NullableDateMilliVector implements a fixed width vector (8 bytes) of
+ * DateMilliVector implements a fixed width vector (8 bytes) of
  * date values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableDateMilliVector extends BaseNullableFixedWidthVector {
+public class DateMilliVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableDateMilliVector. This doesn't allocate any memory for
+   * Instantiate a DateMilliVector. 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) {
+  public DateMilliVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.DATEMILLI.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableDateMilliVector. This doesn't allocate any memory for
+   * Instantiate a DateMilliVector. 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) {
+  public DateMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new DateMilliReaderImpl(NullableDateMilliVector.this);
+    reader = new DateMilliReaderImpl(DateMilliVector.this);
   }
 
   /**
@@ -143,21 +143,21 @@ public class NullableDateMilliVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableDateMilliVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, DateMilliVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableDateMilliVector)} except that
+   * Same as {@link #copyFrom(int, int, DateMilliVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, DateMilliVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -336,22 +336,22 @@ public class NullableDateMilliVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableDateMilliVector) to);
+    return new TransferImpl((DateMilliVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableDateMilliVector to;
+    DateMilliVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableDateMilliVector(ref, field.getFieldType(), allocator);
+      to = new DateMilliVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableDateMilliVector to) {
+    public TransferImpl(DateMilliVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableDateMilliVector getTo() {
+    public DateMilliVector getTo() {
       return to;
     }
 
@@ -367,7 +367,7 @@ public class NullableDateMilliVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableDateMilliVector.this);
+      to.copyFromSafe(fromIndex, toIndex, DateMilliVector.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/DecimalVector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java
index 5d2782a..71a74cb 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableDecimalVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/DecimalVector.java
@@ -32,11 +32,11 @@ import org.apache.arrow.vector.util.TransferPair;
 import java.math.BigDecimal;
 
 /**
- * NullableDecimalVector implements a fixed width vector (16 bytes) of
+ * DecimalVector implements a fixed width vector (16 bytes) of
  * decimal values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableDecimalVector extends BaseNullableFixedWidthVector {
+public class DecimalVector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 16;
   private final FieldReader reader;
 
@@ -44,28 +44,28 @@ public class NullableDecimalVector extends BaseNullableFixedWidthVector {
   private final int scale;
 
   /**
-   * Instantiate a NullableDecimalVector. This doesn't allocate any memory for
+   * Instantiate a DecimalVector. 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,
+  public DecimalVector(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
+   * Instantiate a DecimalVector. 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) {
+  public DecimalVector(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);
+    reader = new DecimalReaderImpl(DecimalVector.this);
     this.precision = arrowType.getPrecision();
     this.scale = arrowType.getScale();
   }
@@ -150,21 +150,21 @@ public class NullableDecimalVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableDecimalVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, DecimalVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, valueBuffer,
             thisIndex * TYPE_WIDTH, TYPE_WIDTH);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableDecimalVector)} except that
+   * Same as {@link #copyFrom(int, int, DecimalVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, DecimalVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -385,23 +385,23 @@ public class NullableDecimalVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableDecimalVector) to);
+    return new TransferImpl((DecimalVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableDecimalVector to;
+    DecimalVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableDecimalVector(ref, allocator, NullableDecimalVector.this.precision,
-              NullableDecimalVector.this.scale);
+      to = new DecimalVector(ref, allocator, DecimalVector.this.precision,
+              DecimalVector.this.scale);
     }
 
-    public TransferImpl(NullableDecimalVector to) {
+    public TransferImpl(DecimalVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableDecimalVector getTo() {
+    public DecimalVector getTo() {
       return to;
     }
 
@@ -417,7 +417,7 @@ public class NullableDecimalVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableDecimalVector.this);
+      to.copyFromSafe(fromIndex, toIndex, DecimalVector.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/Float4Vector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java
index 5d7128d..dc78bfd 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/Float4Vector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableFloat4Vector implements a fixed width vector (4 bytes) of
+ * Float4Vector implements a fixed width vector (4 bytes) of
  * float values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
+public class Float4Vector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableFloat4Vector. This doesn't allocate any memory for
+   * Instantiate a Float4Vector. 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) {
+  public Float4Vector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.FLOAT4.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableFloat4Vector. This doesn't allocate any memory for
+   * Instantiate a Float4Vector. 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) {
+  public Float4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new Float4ReaderImpl(NullableFloat4Vector.this);
+    reader = new Float4ReaderImpl(Float4Vector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableFloat4Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, Float4Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final float value = from.valueBuffer.getFloat(fromIndex * TYPE_WIDTH);
     valueBuffer.setFloat(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableFloat4Vector)} except that
+   * Same as {@link #copyFrom(int, int, Float4Vector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, Float4Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -330,22 +330,22 @@ public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableFloat4Vector) to);
+    return new TransferImpl((Float4Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableFloat4Vector to;
+    Float4Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableFloat4Vector(ref, field.getFieldType(), allocator);
+      to = new Float4Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableFloat4Vector to) {
+    public TransferImpl(Float4Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableFloat4Vector getTo() {
+    public Float4Vector getTo() {
       return to;
     }
 
@@ -361,7 +361,7 @@ public class NullableFloat4Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableFloat4Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, Float4Vector.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/Float8Vector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java
index 7aa5461..1b410b8 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableFloat8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/Float8Vector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableFloat8Vector implements a fixed width vector (8 bytes) of
+ * Float8Vector implements a fixed width vector (8 bytes) of
  * double values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
+public class Float8Vector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableFloat8Vector. This doesn't allocate any memory for
+   * Instantiate a Float8Vector. 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) {
+  public Float8Vector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.FLOAT8.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableFloat8Vector. This doesn't allocate any memory for
+   * Instantiate a Float8Vector. 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) {
+  public Float8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new Float8ReaderImpl(NullableFloat8Vector.this);
+    reader = new Float8ReaderImpl(Float8Vector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableFloat8Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, Float8Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final double value = from.valueBuffer.getDouble(fromIndex * TYPE_WIDTH);
     valueBuffer.setDouble(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableFloat8Vector)} except that
+   * Same as {@link #copyFrom(int, int, Float8Vector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, Float8Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -330,22 +330,22 @@ public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableFloat8Vector) to);
+    return new TransferImpl((Float8Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableFloat8Vector to;
+    Float8Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableFloat8Vector(ref, field.getFieldType(), allocator);
+      to = new Float8Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableFloat8Vector to) {
+    public TransferImpl(Float8Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableFloat8Vector getTo() {
+    public Float8Vector getTo() {
       return to;
     }
 
@@ -361,7 +361,7 @@ public class NullableFloat8Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableFloat8Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, Float8Vector.this);
     }
   }
 }
\ No newline at end of file
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
index 806beb5..89e2a02 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/GenerateSampleData.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/GenerateSampleData.java
@@ -30,62 +30,62 @@ import java.nio.charset.Charset;
 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);
+    if (vector instanceof IntVector) {
+      writeIntData((IntVector) vector, valueCount);
+    } else if (vector instanceof DecimalVector) {
+      writeDecimalData((DecimalVector) vector, valueCount);
+    } else if (vector instanceof BitVector) {
+      writeBooleanData((BitVector) vector, valueCount);
+    } else if (vector instanceof VarCharVector) {
+      writeVarCharData((VarCharVector) vector, valueCount);
+    } else if (vector instanceof VarBinaryVector) {
+      writeVarBinaryData((VarBinaryVector) vector, valueCount);
+    } else if (vector instanceof BigIntVector) {
+      writeBigIntData((BigIntVector) vector, valueCount);
+    } else if (vector instanceof Float4Vector) {
+      writeFloatData((Float4Vector) vector, valueCount);
+    } else if (vector instanceof Float8Vector) {
+      writeDoubleData((Float8Vector) vector, valueCount);
+    } else if (vector instanceof DateDayVector) {
+      writeDateDayData((DateDayVector) vector, valueCount);
+    } else if (vector instanceof DateMilliVector) {
+      writeDateMilliData((DateMilliVector) vector, valueCount);
+    } else if (vector instanceof IntervalDayVector) {
+      writeIntervalDayData((IntervalDayVector) vector, valueCount);
+    } else if (vector instanceof IntervalYearVector) {
+      writeIntervalYearData((IntervalYearVector) vector, valueCount);
+    } else if (vector instanceof SmallIntVector) {
+      writeSmallIntData((SmallIntVector) vector, valueCount);
+    } else if (vector instanceof TinyIntVector) {
+      writeTinyIntData((TinyIntVector) vector, valueCount);
+    } else if (vector instanceof TimeMicroVector) {
+      writeTimeMicroData((TimeMicroVector) vector, valueCount);
+    } else if (vector instanceof TimeMilliVector) {
+      writeTimeMilliData((TimeMilliVector) vector, valueCount);
+    } else if (vector instanceof TimeNanoVector) {
+      writeTimeNanoData((TimeNanoVector) vector, valueCount);
+    } else if (vector instanceof TimeSecVector) {
+      writeTimeSecData((TimeSecVector) vector, valueCount);
+    } else if (vector instanceof TimeStampSecVector) {
+      writeTimeStampData((TimeStampSecVector) vector, valueCount);
+    } else if (vector instanceof TimeStampMicroVector) {
+      writeTimeStampData((TimeStampMicroVector) vector, valueCount);
+    } else if (vector instanceof TimeStampMilliVector) {
+      writeTimeStampData((TimeStampMilliVector) vector, valueCount);
+    } else if (vector instanceof TimeStampNanoVector) {
+      writeTimeStampData((TimeStampNanoVector) vector, valueCount);
+    } else if (vector instanceof TimeStampSecTZVector) {
+      writeTimeStampData((TimeStampSecTZVector) vector, valueCount);
+    } else if (vector instanceof TimeStampMicroTZVector) {
+      writeTimeStampData((TimeStampMicroTZVector) vector, valueCount);
+    } else if (vector instanceof TimeStampMilliTZVector) {
+      writeTimeStampData((TimeStampMilliTZVector) vector, valueCount);
+    } else if (vector instanceof TimeStampNanoTZVector) {
+      writeTimeStampData((TimeStampNanoTZVector) vector, valueCount);
     }
   }
 
-  private static void writeTimeStampData(NullableTimeStampVector vector, int valueCount) {
+  private static void writeTimeStampData(TimeStampVector vector, int valueCount) {
     final long even = 100000;
     final long odd = 200000;
     for (int i = 0; i < valueCount; i++) {
@@ -98,7 +98,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeDecimalData(NullableDecimalVector vector, int valueCount) {
+  private static void writeDecimalData(DecimalVector vector, int valueCount) {
     final BigDecimal even = new BigDecimal(0.0543278923);
     final BigDecimal odd = new BigDecimal(2.0543278923);
     for (int i = 0; i < valueCount; i++) {
@@ -111,7 +111,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeIntData(NullableIntVector vector, int valueCount) {
+  private static void writeIntData(IntVector vector, int valueCount) {
     final int even = 1000;
     final int odd = 2000;
     for (int i = 0; i < valueCount; i++) {
@@ -124,7 +124,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeBooleanData(NullableBitVector vector, int valueCount) {
+  private static void writeBooleanData(BitVector vector, int valueCount) {
     final int even = 0;
     final int odd = 1;
     for (int i = 0; i < valueCount; i++) {
@@ -137,7 +137,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeIntervalYearData(NullableIntervalYearVector vector, int valueCount) {
+  private static void writeIntervalYearData(IntervalYearVector vector, int valueCount) {
     final int even = 1;
     final int odd = 2;
     for (int i = 0; i < valueCount; i++) {
@@ -150,7 +150,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeIntervalDayData(NullableIntervalDayVector vector, int valueCount) {
+  private static void writeIntervalDayData(IntervalDayVector vector, int valueCount) {
     for (int i = 0; i < valueCount; i++) {
       if (i % 2 == 0) {
         vector.setSafe(i, 1, 50);
@@ -161,7 +161,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeTimeSecData(NullableTimeSecVector vector, int valueCount) {
+  private static void writeTimeSecData(TimeSecVector vector, int valueCount) {
     final int even = 500;
     final int odd = 900;
     for (int i = 0; i < valueCount; i++) {
@@ -174,7 +174,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeTimeMilliData(NullableTimeMilliVector vector, int valueCount) {
+  private static void writeTimeMilliData(TimeMilliVector vector, int valueCount) {
     final int even = 1000;
     final int odd = 2000;
     for (int i = 0; i < valueCount; i++) {
@@ -187,7 +187,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeTimeMicroData(NullableTimeMicroVector vector, int valueCount) {
+  private static void writeTimeMicroData(TimeMicroVector vector, int valueCount) {
     final long even = 1000000000;
     final long odd = 2000000000;
     for (int i = 0; i < valueCount; i++) {
@@ -201,7 +201,7 @@ public class GenerateSampleData {
 
   }
 
-  private static void writeTimeNanoData(NullableTimeNanoVector vector, int valueCount) {
+  private static void writeTimeNanoData(TimeNanoVector vector, int valueCount) {
     final long even = 1000000000;
     final long odd = 2000000000;
     for (int i = 0; i < valueCount; i++) {
@@ -214,7 +214,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeDateDayData(NullableDateDayVector vector, int valueCount) {
+  private static void writeDateDayData(DateDayVector vector, int valueCount) {
     final int even = 1000;
     final int odd = 2000;
     for (int i = 0; i < valueCount; i++) {
@@ -227,7 +227,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeDateMilliData(NullableDateMilliVector vector, int valueCount) {
+  private static void writeDateMilliData(DateMilliVector vector, int valueCount) {
     final long even = 1000000000;
     final long odd = 2000000000;
     for (int i = 0; i < valueCount; i++) {
@@ -240,7 +240,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeSmallIntData(NullableSmallIntVector vector, int valueCount) {
+  private static void writeSmallIntData(SmallIntVector vector, int valueCount) {
     final short even = 10;
     final short odd = 20;
     for (int i = 0; i < valueCount; i++) {
@@ -253,7 +253,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeTinyIntData(NullableTinyIntVector vector, int valueCount) {
+  private static void writeTinyIntData(TinyIntVector vector, int valueCount) {
     final byte even = 1;
     final byte odd = 2;
     for (int i = 0; i < valueCount; i++) {
@@ -266,7 +266,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeBigIntData(NullableBigIntVector vector, int valueCount) {
+  private static void writeBigIntData(BigIntVector vector, int valueCount) {
     final long even = 1000000000;
     final long odd = 2000000000;
     for (int i = 0; i < valueCount; i++) {
@@ -279,7 +279,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeFloatData(NullableFloat4Vector vector, int valueCount) {
+  private static void writeFloatData(Float4Vector vector, int valueCount) {
     final float even = 20.3f;
     final float odd = 40.2f;
     for (int i = 0; i < valueCount; i++) {
@@ -292,7 +292,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeDoubleData(NullableFloat8Vector vector, int valueCount) {
+  private static void writeDoubleData(Float8Vector vector, int valueCount) {
     final double even = 20.2373;
     final double odd = 40.2378;
     for (int i = 0; i < valueCount; i++) {
@@ -305,7 +305,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeVarBinaryData(NullableVarBinaryVector vector, int valueCount) {
+  private static void writeVarBinaryData(VarBinaryVector vector, int valueCount) {
     Charset utf8Charset = Charset.forName("UTF-8");
     final byte[] even = "AAAAA1".getBytes(utf8Charset);
     final byte[] odd = "BBBBBBBBB2".getBytes(utf8Charset);
@@ -319,7 +319,7 @@ public class GenerateSampleData {
     vector.setValueCount(valueCount);
   }
 
-  private static void writeVarCharData(NullableVarCharVector vector, int valueCount) {
+  private static void writeVarCharData(VarCharVector vector, int valueCount) {
     Charset utf8Charset = Charset.forName("UTF-8");
     final byte[] even = "AAAAA1".getBytes(utf8Charset);
     final byte[] odd = "BBBBBBBBB2".getBytes(utf8Charset);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
index 7c6f29a..2364310 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntVector.java
@@ -29,37 +29,37 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableIntVector implements a fixed width (4 bytes) vector of
+ * IntVector implements a fixed width (4 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableIntVector extends BaseNullableFixedWidthVector {
+public class IntVector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableIntVector. This doesn't allocate any memory for
+   * Instantiate a IntVector. 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) {
+  public IntVector(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
+   * Instantiate a IntVector. 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) {
+  public IntVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new IntReaderImpl(NullableIntVector.this);
+    reader = new IntReaderImpl(IntVector.this);
   }
 
   /**
@@ -142,14 +142,14 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from      source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableIntVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, IntVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableIntVector)} except that
+   * Same as {@link #copyFrom(int, int, IntVector)} except that
    * it handles the case when the capacity of the vector needs to be expanded
    * before copy.
    *
@@ -157,7 +157,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from      source vector
    */
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableIntVector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, IntVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -340,22 +340,22 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableIntVector) to);
+    return new TransferImpl((IntVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableIntVector to;
+    IntVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableIntVector(ref, field.getFieldType(), allocator);
+      to = new IntVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableIntVector to) {
+    public TransferImpl(IntVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableIntVector getTo() {
+    public IntVector getTo() {
       return to;
     }
 
@@ -371,7 +371,7 @@ public class NullableIntVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableIntVector.this);
+      to.copyFromSafe(fromIndex, toIndex, IntVector.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/IntervalDayVector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java
index 862f642..481a66f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalDayVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntervalDayVector.java
@@ -30,37 +30,37 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.Period;
 
 /**
- * NullableIntervalDayVector implements a fixed width vector (8 bytes) of
+ * IntervalDayVector implements a fixed width vector (8 bytes) of
  * interval (days and milliseconds) values which could be null.
  * A validity buffer (bit vector) is maintained to track which elements in the
  * vector are null.
  */
-public class NullableIntervalDayVector extends BaseNullableFixedWidthVector {
+public class IntervalDayVector extends BaseFixedWidthVector {
   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
+   * Instantiate a IntervalDayVector. 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) {
+  public IntervalDayVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.INTERVALDAY.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableIntervalDayVector. This doesn't allocate any memory for
+   * Instantiate a IntervalDayVector. 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) {
+  public IntervalDayVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new IntervalDayReaderImpl(NullableIntervalDayVector.this);
+    reader = new IntervalDayReaderImpl(IntervalDayVector.this);
   }
 
   /**
@@ -185,21 +185,21 @@ public class NullableIntervalDayVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableIntervalDayVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, IntervalDayVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     from.valueBuffer.getBytes(fromIndex * TYPE_WIDTH, this.valueBuffer,
               thisIndex * TYPE_WIDTH, TYPE_WIDTH);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableIntervalDayVector)} except that
+   * Same as {@link #copyFrom(int, int, IntervalDayVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, IntervalDayVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -388,22 +388,22 @@ public class NullableIntervalDayVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableIntervalDayVector) to);
+    return new TransferImpl((IntervalDayVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableIntervalDayVector to;
+    IntervalDayVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableIntervalDayVector(ref, field.getFieldType(), allocator);
+      to = new IntervalDayVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableIntervalDayVector to) {
+    public TransferImpl(IntervalDayVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableIntervalDayVector getTo() {
+    public IntervalDayVector getTo() {
       return to;
     }
 
@@ -419,7 +419,7 @@ public class NullableIntervalDayVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableIntervalDayVector.this);
+      to.copyFromSafe(fromIndex, toIndex, IntervalDayVector.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/IntervalYearVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java
index 3fba5bf..2aa728f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableIntervalYearVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/IntervalYearVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.Period;
 
 /**
- * NullableIntervalYearVector implements a fixed width (4 bytes) vector of
+ * IntervalYearVector implements a fixed width (4 bytes) vector of
  * interval (years and months) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
+public class IntervalYearVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableIntervalYearVector. This doesn't allocate any memory for
+   * Instantiate a IntervalYearVector. 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) {
+  public IntervalYearVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.INTERVALYEAR.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableIntervalYearVector. This doesn't allocate any memory for
+   * Instantiate a IntervalYearVector. 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) {
+  public IntervalYearVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new IntervalYearReaderImpl(NullableIntervalYearVector.this);
+    reader = new IntervalYearReaderImpl(IntervalYearVector.this);
   }
 
   /**
@@ -169,21 +169,21 @@ public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableIntervalYearVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, IntervalYearVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableIntervalYearVector)} except that
+   * Same as {@link #copyFrom(int, int, IntervalYearVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, IntervalYearVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -348,22 +348,22 @@ public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableIntervalYearVector) to);
+    return new TransferImpl((IntervalYearVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableIntervalYearVector to;
+    IntervalYearVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableIntervalYearVector(ref, field.getFieldType(), allocator);
+      to = new IntervalYearVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableIntervalYearVector to) {
+    public TransferImpl(IntervalYearVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableIntervalYearVector getTo() {
+    public IntervalYearVector getTo() {
       return to;
     }
 
@@ -379,7 +379,7 @@ public class NullableIntervalYearVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableIntervalYearVector.this);
+      to.copyFromSafe(fromIndex, toIndex, IntervalYearVector.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
deleted file mode 100644
index 086331e..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableBitVector.java
+++ /dev/null
@@ -1,513 +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
- * <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 io.netty.buffer.ArrowBuf;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.impl.BitReaderImpl;
-import org.apache.arrow.vector.complex.reader.FieldReader;
-import org.apache.arrow.vector.holders.BitHolder;
-import org.apache.arrow.vector.holders.NullableBitHolder;
-import org.apache.arrow.vector.types.Types;
-import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.util.OversizedAllocationException;
-import org.apache.arrow.vector.util.TransferPair;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * NullableBitVector implements a fixed width (1 bit) vector of
- * boolean values which could be null. Each value in the vector corresponds
- * to a single bit in the underlying data stream backing the vector.
- */
-public class NullableBitVector extends BaseNullableFixedWidthVector {
-  private final FieldReader reader;
-
-  /**
-   * Instantiate a NullableBitVector. This doesn't allocate any memory for
-   * the data in vector.
-   *
-   * @param name      name of the vector
-   * @param allocator allocator for memory management.
-   */
-  public NullableBitVector(String name, BufferAllocator allocator) {
-    this(name, FieldType.nullable(Types.MinorType.BIT.getType()),
-            allocator);
-  }
-
-  /**
-   * Instantiate a NullableBitVector. This doesn't allocate any memory for
-   * the data in vector.
-   *
-   * @param name      name of the vector
-   * @param fieldType type of Field materialized by this vector
-   * @param allocator allocator for memory management.
-   */
-  public NullableBitVector(String name, FieldType fieldType, BufferAllocator allocator) {
-    super(name, allocator, fieldType, (byte) 0);
-    reader = new BitReaderImpl(NullableBitVector.this);
-  }
-
-  /**
-   * Get a reader that supports reading values from this vector
-   *
-   * @return Field Reader for this vector
-   */
-  @Override
-  public FieldReader getReader() {
-    return reader;
-  }
-
-  /**
-   * Get minor type for this vector. The vector holds values belonging
-   * to a particular type.
-   *
-   * @return {@link org.apache.arrow.vector.types.Types.MinorType}
-   */
-  @Override
-  public Types.MinorType getMinorType() {
-    return Types.MinorType.BIT;
-  }
-
-  /**
-   * Sets the desired value capacity for the vector. This function doesn't
-   * allocate any memory for the vector.
-   *
-   * @param valueCount desired number of elements in the vector
-   */
-  @Override
-  public void setInitialCapacity(int valueCount) {
-    final int size = getValidityBufferSizeFromCount(valueCount);
-    if (size > MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Requested amount of memory is more than max allowed");
-    }
-    valueAllocationSizeInBytes = size;
-    validityAllocationSizeInBytes = size;
-  }
-
-  /**
-   * Get the current value capacity for the vector
-   *
-   * @return number of elements that vector can hold.
-   */
-  @Override
-  public int getValueCapacity() {
-    return (int) (validityBuffer.capacity() * 8L);
-  }
-
-  /**
-   * Get the potential buffer size for a particular number of records.
-   *
-   * @param count desired number of elements in the vector
-   * @return estimated size of underlying buffers if the vector holds
-   * a given number of elements
-   */
-  @Override
-  public int getBufferSizeFor(final int count) {
-    if (count == 0) {
-      return 0;
-    }
-    return 2 * getValidityBufferSizeFromCount(count);
-  }
-
-  /**
-   * Get the size (number of bytes) of underlying buffers used by this
-   * vector
-   *
-   * @return size of underlying buffers.
-   */
-  @Override
-  public int getBufferSize() {
-    return getBufferSizeFor(valueCount);
-  }
-
-  /**
-   * Slice this vector at desired index and length and transfer the
-   * corresponding data to the target vector.
-   *
-   * @param startIndex start position of the split in source vector.
-   * @param length     length of the split.
-   * @param target     destination vector
-   */
-  public void splitAndTransferTo(int startIndex, int length,
-                                 BaseNullableFixedWidthVector target) {
-    compareTypes(target, "splitAndTransferTo");
-    target.clear();
-    target.validityBuffer = splitAndTransferBuffer(startIndex, length, target,
-            validityBuffer, target.validityBuffer);
-    target.valueBuffer = splitAndTransferBuffer(startIndex, length, target,
-            valueBuffer, target.valueBuffer);
-
-    target.setValueCount(length);
-  }
-
-  private ArrowBuf splitAndTransferBuffer(int startIndex, int length,
-                                          BaseNullableFixedWidthVector target,
-                                          ArrowBuf sourceBuffer, ArrowBuf destBuffer) {
-    assert startIndex + length <= valueCount;
-    int firstByteSource = BitVectorHelper.byteIndex(startIndex);
-    int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
-    int byteSizeTarget = getValidityBufferSizeFromCount(length);
-    int offset = startIndex % 8;
-
-    if (length > 0) {
-      if (offset == 0) {
-            /* slice */
-        if (destBuffer != null) {
-          destBuffer.release();
-        }
-        destBuffer = 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());
-
-        for (int i = 0; i < byteSizeTarget - 1; i++) {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer, firstByteSource + i, offset);
-          byte b2 = BitVectorHelper.getBitsFromNextByte(sourceBuffer, firstByteSource + i + 1, offset);
-
-          destBuffer.setByte(i, (b1 + b2));
-        }
-
-            /* Copying the last piece is done in the following manner:
-             * if the source vector has 1 or more bytes remaining, we copy
-             * the last piece as a byte formed by shifting data
-             * from the current byte and the next byte.
-             *
-             * if the source vector has no more bytes remaining
-             * (we are at the last byte), we copy the last piece as a byte
-             * by shifting data from the current byte.
-             */
-        if ((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
-                  firstByteSource + byteSizeTarget - 1, offset);
-          byte b2 = BitVectorHelper.getBitsFromNextByte(sourceBuffer,
-                  firstByteSource + byteSizeTarget, offset);
-
-          destBuffer.setByte(byteSizeTarget - 1, b1 + b2);
-        } else {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(sourceBuffer,
-                  firstByteSource + byteSizeTarget - 1, offset);
-          destBuffer.setByte(byteSizeTarget - 1, b1);
-        }
-      }
-    }
-
-    return destBuffer;
-  }
-
-
-  /******************************************************************
-   *                                                                *
-   *          vector value retrieval methods                        *
-   *                                                                *
-   ******************************************************************/
-
-  private int getBit(int index) {
-    final int byteIndex = index >> 3;
-    final byte b = valueBuffer.getByte(byteIndex);
-    final int bitIndex = index & 7;
-    return Long.bitCount(b & (1L << bitIndex));
-  }
-
-  /**
-   * Get the element at the given index from the vector.
-   *
-   * @param index position of element
-   * @return element at given index
-   */
-  public int get(int index) throws IllegalStateException {
-    if (isSet(index) == 0) {
-      throw new IllegalStateException("Value at index is null");
-    }
-    return getBit(index);
-  }
-
-  /**
-   * Get the element at the given index from the vector and
-   * sets the state in holder. If element at given index
-   * is null, holder.isSet will be zero.
-   *
-   * @param index position of element
-   */
-  public void get(int index, NullableBitHolder holder) {
-    if (isSet(index) == 0) {
-      holder.isSet = 0;
-      return;
-    }
-    holder.isSet = 1;
-    holder.value = getBit(index);
-  }
-
-  /**
-   * Same as {@link #get(int)}.
-   *
-   * @param index position of element
-   * @return element at given index
-   */
-  public Boolean getObject(int index) {
-    if (isSet(index) == 0) {
-      return null;
-    } else {
-      return new Boolean(getBit(index) != 0);
-    }
-  }
-
-  /**
-   * Copy a cell value from a particular index in source vector to a particular
-   * position in this vector
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFrom(int fromIndex, int thisIndex, NullableBitVector from) {
-    BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
-    BitVectorHelper.setValidityBit(valueBuffer, thisIndex, from.getBit(fromIndex));
-  }
-
-  /**
-   * Same as {@link #copyFrom(int, int, NullableBitVector)} except that
-   * it handles the case when the capacity of the vector needs to be expanded
-   * before copy.
-   *
-   * @param fromIndex position to copy from in source vector
-   * @param thisIndex position to copy to in this vector
-   * @param from      source vector
-   */
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableBitVector from) {
-    handleSafe(thisIndex);
-    copyFrom(fromIndex, thisIndex, from);
-  }
-
-
-  /******************************************************************
-   *                                                                *
-   *          vector value setter methods                           *
-   *                                                                *
-   ******************************************************************/
-
-
-  /**
-   * Set the element at the given index to the given value.
-   *
-   * @param index position of element
-   * @param value value of element
-   */
-  public void set(int index, int value) {
-    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-    if (value != 0) {
-      BitVectorHelper.setValidityBitToOne(valueBuffer, index);
-    } else {
-      BitVectorHelper.setValidityBit(valueBuffer, index, 0);
-    }
-  }
-
-  /**
-   * Set the element at the given index to the value set in data holder.
-   * If the value in holder is not indicated as set, element in the
-   * at the given index will be null.
-   *
-   * @param index  position of element
-   * @param holder nullable data holder for value of element
-   */
-  public void set(int index, NullableBitHolder holder) throws IllegalArgumentException {
-    if (holder.isSet < 0) {
-      throw new IllegalArgumentException();
-    } else if (holder.isSet > 0) {
-      BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-      if (holder.value != 0) {
-        BitVectorHelper.setValidityBitToOne(valueBuffer, index);
-      } else {
-        BitVectorHelper.setValidityBit(valueBuffer, index, 0);
-      }
-    } else {
-      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-    }
-  }
-
-  /**
-   * Set the element at the given index to the value set in data holder.
-   *
-   * @param index  position of element
-   * @param holder data holder for value of element
-   */
-  public void set(int index, BitHolder holder) {
-    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-    if (holder.value != 0) {
-      BitVectorHelper.setValidityBitToOne(valueBuffer, index);
-    } else {
-      BitVectorHelper.setValidityBit(valueBuffer, index, 0);
-    }
-  }
-
-  /**
-   * Same as {@link #set(int, int)} except that it handles the
-   * case when index is greater than or equal to existing
-   * value capacity {@link #getValueCapacity()}.
-   *
-   * @param index position of element
-   * @param value value of element
-   */
-  public void setSafe(int index, int value) {
-    handleSafe(index);
-    set(index, value);
-  }
-
-  /**
-   * Same as {@link #set(int, NullableBitHolder)} except that it handles the
-   * case when index is greater than or equal to existing
-   * value capacity {@link #getValueCapacity()}.
-   *
-   * @param index  position of element
-   * @param holder nullable data holder for value of element
-   */
-  public void setSafe(int index, NullableBitHolder holder) throws IllegalArgumentException {
-    handleSafe(index);
-    set(index, holder);
-  }
-
-  /**
-   * Same as {@link #set(int, BitHolder)} except that it handles the
-   * case when index is greater than or equal to existing
-   * value capacity {@link #getValueCapacity()}.
-   *
-   * @param index  position of element
-   * @param holder data holder for value of element
-   */
-  public void setSafe(int index, BitHolder holder) {
-    handleSafe(index);
-    set(index, holder);
-  }
-
-  /**
-   * Set the element at the given index to null.
-   *
-   * @param index position of element
-   */
-  public void setNull(int index) {
-    handleSafe(index);
-      /* not really needed to set the bit to 0 as long as
-       * the buffer always starts from 0.
-       */
-    BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-  }
-
-  /**
-   * Store the given value at a particular position in the vector. isSet indicates
-   * whether the value is NULL or not.
-   *
-   * @param index position of the new value
-   * @param isSet 0 for NULL value, 1 otherwise
-   * @param value element value
-   */
-  public void set(int index, int isSet, int value) {
-    if (isSet > 0) {
-      set(index, value);
-    } else {
-      BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-    }
-  }
-
-  /**
-   * Same as {@link #set(int, int, int)} except that it handles the case
-   * when index is greater than or equal to current value capacity of the
-   * vector.
-   *
-   * @param index position of the new value
-   * @param isSet 0 for NULL value, 1 otherwise
-   * @param value element value
-   */
-  public void setSafe(int index, int isSet, int value) {
-    handleSafe(index);
-    set(index, isSet, value);
-  }
-
-
-  /******************************************************************
-   *                                                                *
-   *                      vector transfer                           *
-   *                                                                *
-   ******************************************************************/
-
-
-  /**
-   * Construct a TransferPair comprising of this and and a target vector of
-   * the same type.
-   *
-   * @param ref       name of the target vector
-   * @param allocator allocator for the target vector
-   * @return {@link TransferPair}
-   */
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    return new TransferImpl(ref, allocator);
-  }
-
-  /**
-   * Construct a TransferPair with a desired target vector of the same type.
-   *
-   * @param to target vector
-   * @return {@link TransferPair}
-   */
-  @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableBitVector) to);
-  }
-
-  private class TransferImpl implements TransferPair {
-    NullableBitVector to;
-
-    public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableBitVector(ref, field.getFieldType(), allocator);
-    }
-
-    public TransferImpl(NullableBitVector to) {
-      this.to = to;
-    }
-
-    @Override
-    public NullableBitVector getTo() {
-      return to;
-    }
-
-    @Override
-    public void transfer() {
-      transferTo(to);
-    }
-
-    @Override
-    public void splitAndTransfer(int startIndex, int length) {
-      splitAndTransferTo(startIndex, length, to);
-    }
-
-    @Override
-    public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableBitVector.this);
-    }
-  }
-}
\ No newline at end of file
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableVector.java b/java/vector/src/main/java/org/apache/arrow/vector/NullableVector.java
deleted file mode 100644
index b2455e9..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVector.java
+++ /dev/null
@@ -1,26 +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;
-
-public interface NullableVector extends ValueVector {
-
-  BitVector getValidityVector();
-
-  ValueVector getValuesVector();
-}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java b/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
index d5c66b1..859e62d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableSmallIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/SmallIntVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableSmallIntVector implements a fixed width (2 bytes) vector of
+ * SmallIntVector implements a fixed width (2 bytes) vector of
  * short values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
+public class SmallIntVector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 2;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableSmallIntVector. This doesn't allocate any memory for
+   * Instantiate a SmallIntVector. 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) {
+  public SmallIntVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.SMALLINT.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableSmallIntVector. This doesn't allocate any memory for
+   * Instantiate a SmallIntVector. 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) {
+  public SmallIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new SmallIntReaderImpl(NullableSmallIntVector.this);
+    reader = new SmallIntReaderImpl(SmallIntVector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableSmallIntVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, SmallIntVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final short value = from.valueBuffer.getShort(fromIndex * TYPE_WIDTH);
     valueBuffer.setShort(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableSmallIntVector)} except that
+   * Same as {@link #copyFrom(int, int, SmallIntVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, SmallIntVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -357,22 +357,22 @@ public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableSmallIntVector) to);
+    return new TransferImpl((SmallIntVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableSmallIntVector to;
+    SmallIntVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableSmallIntVector(ref, field.getFieldType(), allocator);
+      to = new SmallIntVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableSmallIntVector to) {
+    public TransferImpl(SmallIntVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableSmallIntVector getTo() {
+    public SmallIntVector getTo() {
       return to;
     }
 
@@ -388,7 +388,7 @@ public class NullableSmallIntVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableSmallIntVector.this);
+      to.copyFromSafe(fromIndex, toIndex, SmallIntVector.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/TimeMicroVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java
index ca8e1aa..604cedf 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMicroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeMicroVector.java
@@ -30,36 +30,36 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableTimeMicroVector implements a fixed width vector (8 bytes) of
+ * TimeMicroVector implements a fixed width vector (8 bytes) of
  * time (microsecond resolution) values which could be null.
  * A validity buffer (bit vector) is maintained to track which elements in the
  * vector are null.
  */
-public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
+public class TimeMicroVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeMicroVector. This doesn't allocate any memory for
+   * Instantiate a TimeMicroVector. 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) {
+  public TimeMicroVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMEMICRO.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeMicroVector. This doesn't allocate any memory for
+   * Instantiate a TimeMicroVector. 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) {
+  public TimeMicroVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new TimeMicroReaderImpl(NullableTimeMicroVector.this);
+    reader = new TimeMicroReaderImpl(TimeMicroVector.this);
   }
 
   /**
@@ -138,21 +138,21 @@ public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTimeMicroVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TimeMicroVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableTimeMicroVector)} except that
+   * Same as {@link #copyFrom(int, int, TimeMicroVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TimeMicroVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -331,22 +331,22 @@ public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeMicroVector) to);
+    return new TransferImpl((TimeMicroVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableTimeMicroVector to;
+    TimeMicroVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableTimeMicroVector(ref, field.getFieldType(), allocator);
+      to = new TimeMicroVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableTimeMicroVector to) {
+    public TransferImpl(TimeMicroVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTimeMicroVector getTo() {
+    public TimeMicroVector getTo() {
       return to;
     }
 
@@ -362,7 +362,7 @@ public class NullableTimeMicroVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTimeMicroVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TimeMicroVector.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/TimeMilliVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java
index a28ccfe..c3d100c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeMilliVector.java
@@ -31,35 +31,35 @@ import org.joda.time.LocalDateTime;
 import org.slf4j.Logger;
 
 /**
- * NullableTimeMilliVector implements a fixed width (4 bytes) vector of
+ * TimeMilliVector implements a fixed width (4 bytes) vector of
  * time (millisecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
+public class TimeMilliVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeMilliVector. This doesn't allocate any memory for
+   * Instantiate a TimeMilliVector. 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) {
+  public TimeMilliVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMEMILLI.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeMilliVector. This doesn't allocate any memory for
+   * Instantiate a TimeMilliVector. 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) {
+  public TimeMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new TimeMilliReaderImpl(NullableTimeMilliVector.this);
+    reader = new TimeMilliReaderImpl(TimeMilliVector.this);
   }
 
   /**
@@ -139,21 +139,21 @@ public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTimeMilliVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TimeMilliVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableTimeMilliVector)} except that
+   * Same as {@link #copyFrom(int, int, TimeMilliVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TimeMilliVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -332,22 +332,22 @@ public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeMilliVector) to);
+    return new TransferImpl((TimeMilliVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableTimeMilliVector to;
+    TimeMilliVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableTimeMilliVector(ref, field.getFieldType(), allocator);
+      to = new TimeMilliVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableTimeMilliVector to) {
+    public TransferImpl(TimeMilliVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTimeMilliVector getTo() {
+    public TimeMilliVector getTo() {
       return to;
     }
 
@@ -363,7 +363,7 @@ public class NullableTimeMilliVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTimeMilliVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TimeMilliVector.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/TimeNanoVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java
index e6903c0..97401ec 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeNanoVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeNanoVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeNanoVector implements a fixed width vector (8 bytes) of
+ * TimeNanoVector implements a fixed width vector (8 bytes) of
  * time (nanosecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
+public class TimeNanoVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeNanoVector. This doesn't allocate any memory for
+   * Instantiate a TimeNanoVector. 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) {
+  public TimeNanoVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMENANO.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeNanoVector. This doesn't allocate any memory for
+   * Instantiate a TimeNanoVector. 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) {
+  public TimeNanoVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new TimeNanoReaderImpl(NullableTimeNanoVector.this);
+    reader = new TimeNanoReaderImpl(TimeNanoVector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTimeNanoVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TimeNanoVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableTimeNanoVector)} except that
+   * Same as {@link #copyFrom(int, int, TimeNanoVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TimeNanoVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -329,22 +329,22 @@ public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeNanoVector) to);
+    return new TransferImpl((TimeNanoVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableTimeNanoVector to;
+    TimeNanoVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableTimeNanoVector(ref, field.getFieldType(), allocator);
+      to = new TimeNanoVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableTimeNanoVector to) {
+    public TransferImpl(TimeNanoVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTimeNanoVector getTo() {
+    public TimeNanoVector getTo() {
       return to;
     }
 
@@ -360,7 +360,7 @@ public class NullableTimeNanoVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTimeNanoVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TimeNanoVector.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/TimeSecVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java
index 85426a3..a7823a9 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeSecVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeSecVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeSecVector implements a fixed width (4 bytes) vector of
+ * TimeSecVector implements a fixed width (4 bytes) vector of
  * time (seconds resolution) values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
+public class TimeSecVector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeSecVector. This doesn't allocate any memory for
+   * Instantiate a TimeSecVector. 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) {
+  public TimeSecVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMESEC.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeSecVector. This doesn't allocate any memory for
+   * Instantiate a TimeSecVector. 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) {
+  public TimeSecVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new TimeSecReaderImpl(NullableTimeSecVector.this);
+    reader = new TimeSecReaderImpl(TimeSecVector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTimeSecVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TimeSecVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableTimeSecVector)} except that
+   * Same as {@link #copyFrom(int, int, TimeSecVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TimeSecVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -330,22 +330,22 @@ public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeSecVector) to);
+    return new TransferImpl((TimeSecVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableTimeSecVector to;
+    TimeSecVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableTimeSecVector(ref, field.getFieldType(), allocator);
+      to = new TimeSecVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableTimeSecVector to) {
+    public TransferImpl(TimeSecVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTimeSecVector getTo() {
+    public TimeSecVector getTo() {
       return to;
     }
 
@@ -361,7 +361,7 @@ public class NullableTimeSecVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTimeSecVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TimeSecVector.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/TimeStampMicroTZVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampMicroTZVector.java
index c7ad141..bfe330a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampMicroTZVector.java
@@ -29,37 +29,37 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeStampMicroTZVector implements a fixed width vector (8 bytes) of
+ * TimeStampMicroTZVector implements a fixed width vector (8 bytes) of
  * timestamp (microsecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampMicroTZVector extends NullableTimeStampVector {
+public class TimeStampMicroTZVector extends TimeStampVector {
   private final FieldReader reader;
   private final String timeZone;
 
   /**
-   * Instantiate a NullableTimeStampMicroTZVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMicroTZVector. 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) {
+  public TimeStampMicroTZVector(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
+   * Instantiate a TimeStampMicroTZVector. 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) {
+  public TimeStampMicroTZVector(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);
+    reader = new TimeStampMicroTZReaderImpl(TimeStampMicroTZVector.this);
   }
 
   /**
@@ -200,7 +200,7 @@ public class NullableTimeStampMicroTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampMicroTZVector to = new NullableTimeStampMicroTZVector(ref,
+    TimeStampMicroTZVector to = new TimeStampMicroTZVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -212,6 +212,6 @@ public class NullableTimeStampMicroTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampMicroTZVector) to);
+    return new TransferImpl((TimeStampMicroTZVector) 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/TimeStampMicroVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampMicroVector.java
index 5a34b19..85b615d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMicroVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampMicroVector.java
@@ -29,34 +29,34 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.LocalDateTime;
 
 /**
- * NullableTimeStampMicroVector implements a fixed width vector (8 bytes) of
+ * TimeStampMicroVector implements a fixed width vector (8 bytes) of
  * timestamp (microsecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampMicroVector extends NullableTimeStampVector {
+public class TimeStampMicroVector extends TimeStampVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeStampMicroVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMicroVector. 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) {
+  public TimeStampMicroVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMESTAMPMICRO.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeStampMicroVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMicroVector. 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) {
+  public TimeStampMicroVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, fieldType, allocator);
-    reader = new TimeStampMicroReaderImpl(NullableTimeStampMicroVector.this);
+    reader = new TimeStampMicroReaderImpl(TimeStampMicroVector.this);
   }
 
   /**
@@ -202,7 +202,7 @@ public class NullableTimeStampMicroVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampMicroVector to = new NullableTimeStampMicroVector(ref,
+    TimeStampMicroVector to = new TimeStampMicroVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -214,6 +214,6 @@ public class NullableTimeStampMicroVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampMicroVector) to);
+    return new TransferImpl((TimeStampMicroVector) 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/TimeStampMilliTZVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampMilliTZVector.java
index f9736ae..9d68b56 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampMilliTZVector.java
@@ -29,37 +29,37 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeStampMilliTZVector implements a fixed width vector (8 bytes) of
+ * TimeStampMilliTZVector implements a fixed width vector (8 bytes) of
  * timestamp (millisecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampMilliTZVector extends NullableTimeStampVector {
+public class TimeStampMilliTZVector extends TimeStampVector {
   private final FieldReader reader;
   private final String timeZone;
 
   /**
-   * Instantiate a NullableTimeStampMilliTZVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMilliTZVector. 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) {
+  public TimeStampMilliTZVector(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
+   * Instantiate a TimeStampMilliTZVector. 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) {
+  public TimeStampMilliTZVector(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);
+    reader = new TimeStampMilliTZReaderImpl(TimeStampMilliTZVector.this);
   }
 
   /**
@@ -199,7 +199,7 @@ public class NullableTimeStampMilliTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampMilliTZVector to = new NullableTimeStampMilliTZVector(ref,
+    TimeStampMilliTZVector to = new TimeStampMilliTZVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -211,6 +211,6 @@ public class NullableTimeStampMilliTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampMilliTZVector) to);
+    return new TransferImpl((TimeStampMilliTZVector) 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/TimeStampMilliVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampMilliVector.java
index fc38a6b..7e8a1d0 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampMilliVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampMilliVector.java
@@ -29,34 +29,34 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.LocalDateTime;
 
 /**
- * NullableTimeStampMilliVector implements a fixed width vector (8 bytes) of
+ * TimeStampMilliVector implements a fixed width vector (8 bytes) of
  * timestamp (millisecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampMilliVector extends NullableTimeStampVector {
+public class TimeStampMilliVector extends TimeStampVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeStampMilliVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMilliVector. 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) {
+  public TimeStampMilliVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMESTAMPMILLI.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeStampMilliVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampMilliVector. 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) {
+  public TimeStampMilliVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, fieldType, allocator);
-    reader = new TimeStampMilliReaderImpl(NullableTimeStampMilliVector.this);
+    reader = new TimeStampMilliReaderImpl(TimeStampMilliVector.this);
   }
 
   /**
@@ -200,7 +200,7 @@ public class NullableTimeStampMilliVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampMilliVector to = new NullableTimeStampMilliVector(ref,
+    TimeStampMilliVector to = new TimeStampMilliVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -212,6 +212,6 @@ public class NullableTimeStampMilliVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampMilliVector) to);
+    return new TransferImpl((TimeStampMilliVector) 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/TimeStampNanoTZVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampNanoTZVector.java
index 8eb4846..e036182 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampNanoTZVector.java
@@ -29,37 +29,37 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeStampNanoTZVector implements a fixed width vector (8 bytes) of
+ * TimeStampNanoTZVector implements a fixed width vector (8 bytes) of
  * timestamp (nanosecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampNanoTZVector extends NullableTimeStampVector {
+public class TimeStampNanoTZVector extends TimeStampVector {
   private final FieldReader reader;
   private final String timeZone;
 
   /**
-   * Instantiate a NullableTimeStampNanoTZVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampNanoTZVector. 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) {
+  public TimeStampNanoTZVector(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
+   * Instantiate a TimeStampNanoTZVector. 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) {
+  public TimeStampNanoTZVector(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);
+    reader = new TimeStampNanoTZReaderImpl(TimeStampNanoTZVector.this);
   }
 
   /**
@@ -200,7 +200,7 @@ public class NullableTimeStampNanoTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampNanoTZVector to = new NullableTimeStampNanoTZVector(ref,
+    TimeStampNanoTZVector to = new TimeStampNanoTZVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -212,6 +212,6 @@ public class NullableTimeStampNanoTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampNanoTZVector) to);
+    return new TransferImpl((TimeStampNanoTZVector) 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/TimeStampNanoVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampNanoVector.java
index 6cc5c90..fdf5d26 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampNanoVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampNanoVector.java
@@ -29,34 +29,34 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.LocalDateTime;
 
 /**
- * NullableTimeStampNanoVector implements a fixed width vector (8 bytes) of
+ * TimeStampNanoVector implements a fixed width vector (8 bytes) of
  * timestamp (nanosecond resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampNanoVector extends NullableTimeStampVector {
+public class TimeStampNanoVector extends TimeStampVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeStampNanoVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampNanoVector. 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) {
+  public TimeStampNanoVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMESTAMPNANO.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeStampNanoVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampNanoVector. 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) {
+  public TimeStampNanoVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, fieldType, allocator);
-    reader = new TimeStampNanoReaderImpl(NullableTimeStampNanoVector.this);
+    reader = new TimeStampNanoReaderImpl(TimeStampNanoVector.this);
   }
 
   /**
@@ -201,7 +201,7 @@ public class NullableTimeStampNanoVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampNanoVector to = new NullableTimeStampNanoVector(ref,
+    TimeStampNanoVector to = new TimeStampNanoVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -213,6 +213,6 @@ public class NullableTimeStampNanoVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampNanoVector) to);
+    return new TransferImpl((TimeStampNanoVector) 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/TimeStampSecTZVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampSecTZVector.java
index 0593ac7..201f1c3 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecTZVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampSecTZVector.java
@@ -28,37 +28,37 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeStampSecTZVector implements a fixed width vector (8 bytes) of
+ * TimeStampSecTZVector implements a fixed width vector (8 bytes) of
  * timestamp (seconds resolution) values which could be null. A validity buffer
  * (bit vector) is maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampSecTZVector extends NullableTimeStampVector {
+public class TimeStampSecTZVector extends TimeStampVector {
   private final FieldReader reader;
   private final String timeZone;
 
   /**
-   * Instantiate a NullableTimeStampSecTZVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampSecTZVector. 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) {
+  public TimeStampSecTZVector(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
+   * Instantiate a TimeStampSecTZVector. 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) {
+  public TimeStampSecTZVector(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);
+    reader = new TimeStampSecTZReaderImpl(TimeStampSecTZVector.this);
   }
 
   /**
@@ -198,7 +198,7 @@ public class NullableTimeStampSecTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampSecTZVector to = new NullableTimeStampSecTZVector(ref,
+    TimeStampSecTZVector to = new TimeStampSecTZVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -210,6 +210,6 @@ public class NullableTimeStampSecTZVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampSecTZVector) to);
+    return new TransferImpl((TimeStampSecTZVector) 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/TimeStampSecVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampSecVector.java
index 1afdb30..4bcd4f7 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampSecVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampSecVector.java
@@ -29,34 +29,34 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.joda.time.LocalDateTime;
 
 /**
- * NullableTimeStampSecVector implements a fixed width vector (8 bytes) of
+ * TimeStampSecVector implements a fixed width vector (8 bytes) of
  * timestamp (seconds resolution) values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableTimeStampSecVector extends NullableTimeStampVector {
+public class TimeStampSecVector extends TimeStampVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTimeStampSecVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampSecVector. 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) {
+  public TimeStampSecVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TIMESTAMPSEC.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTimeStampSecVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampSecVector. 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) {
+  public TimeStampSecVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, fieldType, allocator);
-    reader = new TimeStampSecReaderImpl(NullableTimeStampSecVector.this);
+    reader = new TimeStampSecReaderImpl(TimeStampSecVector.this);
   }
 
   /**
@@ -201,7 +201,7 @@ public class NullableTimeStampSecVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    NullableTimeStampSecVector to = new NullableTimeStampSecVector(ref,
+    TimeStampSecVector to = new TimeStampSecVector(ref,
             field.getFieldType(), allocator);
     return new TransferImpl(to);
   }
@@ -213,6 +213,6 @@ public class NullableTimeStampSecVector extends NullableTimeStampVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTimeStampSecVector) to);
+    return new TransferImpl((TimeStampSecVector) 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/TimeStampVector.java
similarity index 88%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java
index e1b719f..4c70b81 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTimeStampVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TimeStampVector.java
@@ -24,21 +24,21 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTimeStampVector is an abstract interface for fixed width vector (8 bytes)
+ * TimeStampVector is an abstract interface for fixed width vector (8 bytes)
  * of timestamp values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public abstract class NullableTimeStampVector extends BaseNullableFixedWidthVector {
+public abstract class TimeStampVector extends BaseFixedWidthVector {
   protected static final byte TYPE_WIDTH = 8;
 
   /**
-   * Instantiate a NullableTimeStampVector. This doesn't allocate any memory for
+   * Instantiate a TimeStampVector. 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) {
+  public TimeStampVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
   }
 
@@ -69,21 +69,21 @@ public abstract class NullableTimeStampVector extends BaseNullableFixedWidthVect
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTimeStampVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TimeStampVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFromSafe(int, int, NullableTimeStampVector)} except that
+   * Same as {@link #copyFromSafe(int, int, TimeStampVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TimeStampVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -188,14 +188,14 @@ public abstract class NullableTimeStampVector extends BaseNullableFixedWidthVect
 
 
   public class TransferImpl implements TransferPair {
-    NullableTimeStampVector to;
+    TimeStampVector to;
 
-    public TransferImpl(NullableTimeStampVector to) {
+    public TransferImpl(TimeStampVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTimeStampVector getTo() {
+    public TimeStampVector getTo() {
       return to;
     }
 
@@ -211,7 +211,7 @@ public abstract class NullableTimeStampVector extends BaseNullableFixedWidthVect
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTimeStampVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TimeStampVector.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/TinyIntVector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java
index cadfe60..306437f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableTinyIntVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TinyIntVector.java
@@ -29,35 +29,35 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableTinyIntVector implements a fixed width (1 bytes) vector of
+ * TinyIntVector implements a fixed width (1 bytes) vector of
  * byte values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableTinyIntVector extends BaseNullableFixedWidthVector {
+public class TinyIntVector extends BaseFixedWidthVector {
   public static final byte TYPE_WIDTH = 1;
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableTinyIntVector. This doesn't allocate any memory for
+   * Instantiate a TinyIntVector. 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) {
+  public TinyIntVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.TINYINT.getType()),
             allocator);
   }
 
   /**
-   * Instantiate a NullableTinyIntVector. This doesn't allocate any memory for
+   * Instantiate a TinyIntVector. 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) {
+  public TinyIntVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new TinyIntReaderImpl(NullableTinyIntVector.this);
+    reader = new TinyIntReaderImpl(TinyIntVector.this);
   }
 
   /**
@@ -137,21 +137,21 @@ public class NullableTinyIntVector extends BaseNullableFixedWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableTinyIntVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, TinyIntVector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final byte value = from.valueBuffer.getByte(fromIndex * TYPE_WIDTH);
     valueBuffer.setByte(thisIndex * TYPE_WIDTH, value);
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableTinyIntVector)} except that
+   * Same as {@link #copyFrom(int, int, TinyIntVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, TinyIntVector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -357,22 +357,22 @@ public class NullableTinyIntVector extends BaseNullableFixedWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableTinyIntVector) to);
+    return new TransferImpl((TinyIntVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableTinyIntVector to;
+    TinyIntVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableTinyIntVector(ref, field.getFieldType(), allocator);
+      to = new TinyIntVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableTinyIntVector to) {
+    public TransferImpl(TinyIntVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableTinyIntVector getTo() {
+    public TinyIntVector getTo() {
       return to;
     }
 
@@ -388,7 +388,7 @@ public class NullableTinyIntVector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableTinyIntVector.this);
+      to.copyFromSafe(fromIndex, toIndex, TinyIntVector.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/UInt1Vector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
index ebd9b56..6901a88 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt1Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt1Vector.java
@@ -28,22 +28,22 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 import org.apache.arrow.vector.util.TransferPair;
 
 /**
- * NullableUInt1Vector implements a fixed width (1 bytes) vector of
+ * UInt1Vector implements a fixed width (1 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
+public class UInt1Vector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 1;
   private final FieldReader reader;
 
-  public NullableUInt1Vector(String name, BufferAllocator allocator) {
+  public UInt1Vector(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) {
+  public UInt1Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new UInt1ReaderImpl(NullableUInt1Vector.this);
+    reader = new UInt1ReaderImpl(UInt1Vector.this);
   }
 
   @Override
@@ -107,13 +107,13 @@ public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, NullableUInt1Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, UInt1Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final byte value = from.valueBuffer.getByte(fromIndex * TYPE_WIDTH);
     valueBuffer.setByte(thisIndex * TYPE_WIDTH, value);
   }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt1Vector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, UInt1Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -279,22 +279,22 @@ public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableUInt1Vector) to);
+    return new TransferImpl((UInt1Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableUInt1Vector to;
+    UInt1Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableUInt1Vector(ref, field.getFieldType(), allocator);
+      to = new UInt1Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableUInt1Vector to) {
+    public TransferImpl(UInt1Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableUInt1Vector getTo() {
+    public UInt1Vector getTo() {
       return to;
     }
 
@@ -310,7 +310,7 @@ public class NullableUInt1Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableUInt1Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, UInt1Vector.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/UInt2Vector.java
similarity index 91%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
index bedb3f3..968ce91 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt2Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt2Vector.java
@@ -29,22 +29,22 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableUInt2Vector implements a fixed width (2 bytes) vector of
+ * UInt2Vector implements a fixed width (2 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
+public class UInt2Vector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 2;
   private final FieldReader reader;
 
-  public NullableUInt2Vector(String name, BufferAllocator allocator) {
+  public UInt2Vector(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) {
+  public UInt2Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new UInt2ReaderImpl(NullableUInt2Vector.this);
+    reader = new UInt2ReaderImpl(UInt2Vector.this);
   }
 
   @Override
@@ -108,13 +108,13 @@ public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, NullableUInt2Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, UInt2Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final char value = from.valueBuffer.getChar(fromIndex * TYPE_WIDTH);
     valueBuffer.setChar(thisIndex * TYPE_WIDTH, value);
   }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt2Vector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, UInt2Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -280,22 +280,22 @@ public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableUInt2Vector) to);
+    return new TransferImpl((UInt2Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableUInt2Vector to;
+    UInt2Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableUInt2Vector(ref, field.getFieldType(), allocator);
+      to = new UInt2Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableUInt2Vector to) {
+    public TransferImpl(UInt2Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableUInt2Vector getTo() {
+    public UInt2Vector getTo() {
       return to;
     }
 
@@ -311,7 +311,7 @@ public class NullableUInt2Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableUInt2Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, UInt2Vector.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/UInt4Vector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
index 99b547c..af219cb 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt4Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt4Vector.java
@@ -29,22 +29,22 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableUInt4Vector implements a fixed width (4 bytes) vector of
+ * UInt4Vector implements a fixed width (4 bytes) vector of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
+public class UInt4Vector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 4;
   private final FieldReader reader;
 
-  public NullableUInt4Vector(String name, BufferAllocator allocator) {
+  public UInt4Vector(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) {
+  public UInt4Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new UInt4ReaderImpl(NullableUInt4Vector.this);
+    reader = new UInt4ReaderImpl(UInt4Vector.this);
   }
 
   @Override
@@ -108,13 +108,13 @@ public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, NullableUInt4Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, UInt4Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final int value = from.valueBuffer.getInt(fromIndex * TYPE_WIDTH);
     valueBuffer.setInt(thisIndex * TYPE_WIDTH, value);
   }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt4Vector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, UInt4Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -252,22 +252,22 @@ public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableUInt4Vector) to);
+    return new TransferImpl((UInt4Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableUInt4Vector to;
+    UInt4Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableUInt4Vector(ref, field.getFieldType(), allocator);
+      to = new UInt4Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableUInt4Vector to) {
+    public TransferImpl(UInt4Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableUInt4Vector getTo() {
+    public UInt4Vector getTo() {
       return to;
     }
 
@@ -283,7 +283,7 @@ public class NullableUInt4Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableUInt4Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, UInt4Vector.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/UInt8Vector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
index e6664a0..eae4a31 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableUInt8Vector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/UInt8Vector.java
@@ -29,22 +29,22 @@ import org.apache.arrow.vector.util.TransferPair;
 import org.slf4j.Logger;
 
 /**
- * NullableUInt8Vector implements a fixed width vector (8 bytes) of
+ * UInt8Vector implements a fixed width vector (8 bytes) of
  * integer values which could be null. A validity buffer (bit vector) is
  * maintained to track which elements in the vector are null.
  */
-public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
+public class UInt8Vector extends BaseFixedWidthVector {
   private static final byte TYPE_WIDTH = 8;
   private final FieldReader reader;
 
-  public NullableUInt8Vector(String name, BufferAllocator allocator) {
+  public UInt8Vector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.UINT8.getType()),
             allocator);
   }
 
-  public NullableUInt8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
+  public UInt8Vector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType, TYPE_WIDTH);
-    reader = new UInt8ReaderImpl(NullableUInt8Vector.this);
+    reader = new UInt8ReaderImpl(UInt8Vector.this);
   }
 
   @Override
@@ -108,13 +108,13 @@ public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, NullableUInt8Vector from) {
+  public void copyFrom(int fromIndex, int thisIndex, UInt8Vector from) {
     BitVectorHelper.setValidityBit(validityBuffer, thisIndex, from.isSet(fromIndex));
     final long value = from.valueBuffer.getLong(fromIndex * TYPE_WIDTH);
     valueBuffer.setLong(thisIndex * TYPE_WIDTH, value);
   }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, NullableUInt8Vector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, UInt8Vector from) {
     handleSafe(thisIndex);
     copyFrom(fromIndex, thisIndex, from);
   }
@@ -252,22 +252,22 @@ public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableUInt8Vector) to);
+    return new TransferImpl((UInt8Vector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableUInt8Vector to;
+    UInt8Vector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableUInt8Vector(ref, field.getFieldType(), allocator);
+      to = new UInt8Vector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableUInt8Vector to) {
+    public TransferImpl(UInt8Vector to) {
       this.to = to;
     }
 
     @Override
-    public NullableUInt8Vector getTo() {
+    public UInt8Vector getTo() {
       return to;
     }
 
@@ -283,7 +283,7 @@ public class NullableUInt8Vector extends BaseNullableFixedWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableUInt8Vector.this);
+      to.copyFromSafe(fromIndex, toIndex, UInt8Vector.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/VarBinaryVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java
index 1dfe891..893ad7c 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarBinaryVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VarBinaryVector.java
@@ -31,33 +31,33 @@ import org.apache.arrow.vector.util.TransferPair;
 import java.nio.ByteBuffer;
 
 /**
- * NullableVarBinaryVector implements a variable width vector of binary
+ * VarBinaryVector implements a variable width vector of binary
  * values which could be NULL. A validity buffer (bit vector) is maintained
  * to track which elements in the vector are null.
  */
-public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
+public class VarBinaryVector extends BaseVariableWidthVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableVarBinaryVector. This doesn't allocate any memory for
+   * Instantiate a VarBinaryVector. 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) {
+  public VarBinaryVector(String name, BufferAllocator allocator) {
     this(name, FieldType.nullable(Types.MinorType.VARBINARY.getType()), allocator);
   }
 
   /**
-   * Instantiate a NullableVarBinaryVector. This doesn't allocate any memory for
+   * Instantiate a VarBinaryVector. 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) {
+  public VarBinaryVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType);
-    reader = new VarBinaryReaderImpl(NullableVarBinaryVector.this);
+    reader = new VarBinaryReaderImpl(VarBinaryVector.this);
   }
 
   /**
@@ -156,7 +156,7 @@ public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableVarBinaryVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, VarBinaryVector 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;
@@ -169,14 +169,14 @@ public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableVarBinaryVector)} except that
+   * Same as {@link #copyFrom(int, int, VarBinaryVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, VarBinaryVector 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;
@@ -291,22 +291,22 @@ public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableVarBinaryVector) to);
+    return new TransferImpl((VarBinaryVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableVarBinaryVector to;
+    VarBinaryVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableVarBinaryVector(ref, field.getFieldType(), allocator);
+      to = new VarBinaryVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableVarBinaryVector to) {
+    public TransferImpl(VarBinaryVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableVarBinaryVector getTo() {
+    public VarBinaryVector getTo() {
       return to;
     }
 
@@ -322,7 +322,7 @@ public class NullableVarBinaryVector extends BaseNullableVariableWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableVarBinaryVector.this);
+      to.copyFromSafe(fromIndex, toIndex, VarBinaryVector.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/VarCharVector.java
similarity index 90%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
rename to java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java
index 31e409c..8a38b1d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVarCharVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VarCharVector.java
@@ -32,33 +32,33 @@ import org.apache.arrow.vector.util.TransferPair;
 import java.nio.ByteBuffer;
 
 /**
- * NullableVarCharVector implements a variable width vector of VARCHAR
+ * VarCharVector implements a variable width vector of VARCHAR
  * values which could be NULL. A validity buffer (bit vector) is maintained
  * to track which elements in the vector are null.
  */
-public class NullableVarCharVector extends BaseNullableVariableWidthVector {
+public class VarCharVector extends BaseVariableWidthVector {
   private final FieldReader reader;
 
   /**
-   * Instantiate a NullableVarCharVector. This doesn't allocate any memory for
+   * Instantiate a VarCharVector. 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) {
+  public VarCharVector(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
+   * Instantiate a VarCharVector. 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) {
+  public VarCharVector(String name, FieldType fieldType, BufferAllocator allocator) {
     super(name, allocator, fieldType);
-    reader = new VarCharReaderImpl(NullableVarCharVector.this);
+    reader = new VarCharReaderImpl(VarCharVector.this);
   }
 
   /**
@@ -159,7 +159,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
    * @param thisIndex position to copy to in this vector
    * @param from source vector
    */
-  public void copyFrom(int fromIndex, int thisIndex, NullableVarCharVector from) {
+  public void copyFrom(int fromIndex, int thisIndex, VarCharVector 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;
@@ -172,14 +172,14 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
   }
 
   /**
-   * Same as {@link #copyFrom(int, int, NullableVarCharVector)} except that
+   * Same as {@link #copyFrom(int, int, VarCharVector)} 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) {
+  public void copyFromSafe(int fromIndex, int thisIndex, VarCharVector 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;
@@ -294,22 +294,22 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
    */
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new TransferImpl((NullableVarCharVector) to);
+    return new TransferImpl((VarCharVector) to);
   }
 
   private class TransferImpl implements TransferPair {
-    NullableVarCharVector to;
+    VarCharVector to;
 
     public TransferImpl(String ref, BufferAllocator allocator) {
-      to = new NullableVarCharVector(ref, field.getFieldType(), allocator);
+      to = new VarCharVector(ref, field.getFieldType(), allocator);
     }
 
-    public TransferImpl(NullableVarCharVector to) {
+    public TransferImpl(VarCharVector to) {
       this.to = to;
     }
 
     @Override
-    public NullableVarCharVector getTo() {
+    public VarCharVector getTo() {
       return to;
     }
 
@@ -325,7 +325,7 @@ public class NullableVarCharVector extends BaseNullableVariableWidthVector {
 
     @Override
     public void copyValueSafe(int fromIndex, int toIndex) {
-      to.copyFromSafe(fromIndex, toIndex, NullableVarCharVector.this);
+      to.copyFromSafe(fromIndex, toIndex, VarCharVector.this);
     }
   }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/NullableVectorDefinitionSetter.java b/java/vector/src/main/java/org/apache/arrow/vector/VectorDefinitionSetter.java
similarity index 94%
rename from java/vector/src/main/java/org/apache/arrow/vector/NullableVectorDefinitionSetter.java
rename to java/vector/src/main/java/org/apache/arrow/vector/VectorDefinitionSetter.java
index 1e0746a..2f45d3a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/NullableVectorDefinitionSetter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/VectorDefinitionSetter.java
@@ -18,7 +18,7 @@
 
 package org.apache.arrow.vector;
 
-public interface NullableVectorDefinitionSetter {
+public interface VectorDefinitionSetter {
 
   public void setIndexDefined(int index);
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractContainerVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractContainerVector.java
index db0ff86..5ded1cc 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractContainerVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/AbstractContainerVector.java
@@ -100,8 +100,8 @@ public abstract class AbstractContainerVector implements ValueVector {
   // return the child vector's ordinal in the composite container
   public abstract VectorWithOrdinal getChildVectorWithOrdinal(String name);
 
-  public NullableMapVector addOrGetMap(String name) {
-    return addOrGet(name, FieldType.nullable(new Struct()), NullableMapVector.class);
+  public MapVector addOrGetMap(String name) {
+    return addOrGet(name, FieldType.nullable(new Struct()), MapVector.class);
   }
 
   public ListVector addOrGetList(String name) {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/EmptyValuePopulator.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/EmptyValuePopulator.java
index a76fbbe..cd3a2ae 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/EmptyValuePopulator.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/EmptyValuePopulator.java
@@ -42,14 +42,12 @@ public class EmptyValuePopulator {
     if (lastIndex < 0) {
       throw new IndexOutOfBoundsException("index cannot be negative");
     }
-    final UInt4Vector.Accessor accessor = offsets.getAccessor();
-    final UInt4Vector.Mutator mutator = offsets.getMutator();
-    final int lastSet = Math.max(accessor.getValueCount() - 1, 0);
-    final int previousEnd = accessor.get(lastSet);//0 ? 0 : accessor.get(lastSet);
+    final int lastSet = Math.max(offsets.getValueCount() - 1, 0);
+    final int previousEnd = offsets.get(lastSet);//0 ? 0 : accessor.get(lastSet);
     for (int i = lastSet; i < lastIndex; i++) {
-      mutator.setSafe(i + 1, previousEnd);
+      offsets.setSafe(i + 1, previousEnd);
     }
-    mutator.setValueCount(lastIndex + 1);
+    offsets.setValueCount(lastIndex + 1);
   }
 
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
index 6089a67..075ae83 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
@@ -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.
@@ -21,332 +21,492 @@ 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.Arrays;
+import java.util.Collections;
 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 com.google.common.collect.ObjectArrays;
 
 import io.netty.buffer.ArrowBuf;
-
+import org.apache.arrow.memory.BaseAllocator;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.*;
-import org.apache.arrow.vector.complex.impl.SingleMapReaderImpl;
-import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.complex.impl.NullableMapReaderImpl;
+import org.apache.arrow.vector.complex.impl.NullableMapWriter;
 import org.apache.arrow.vector.holders.ComplexHolder;
-import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
 import org.apache.arrow.vector.types.pojo.ArrowType;
-import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.ArrowType.Struct;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
 import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.util.CallBack;
-import org.apache.arrow.vector.util.JsonStringHashMap;
+import org.apache.arrow.vector.util.OversizedAllocationException;
 import org.apache.arrow.vector.util.TransferPair;
 
-public class MapVector extends AbstractMapVector {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class);
+public class MapVector extends NonNullableMapVector implements FieldVector {
 
   public static MapVector empty(String name, BufferAllocator allocator) {
-    FieldType fieldType = new FieldType(false, ArrowType.Struct.INSTANCE, null, null);
+    FieldType fieldType = FieldType.nullable(Struct.INSTANCE);
     return new MapVector(name, allocator, fieldType, null);
   }
 
-  private final SingleMapReaderImpl reader = new SingleMapReaderImpl(this);
-  protected final FieldType fieldType;
-  public int valueCount;
+  private final NullableMapReaderImpl reader = new NullableMapReaderImpl(this);
+  private final NullableMapWriter writer = new NullableMapWriter(this);
+
+  protected ArrowBuf validityBuffer;
+  private int validityAllocationSizeInBytes;
 
   // deprecated, use FieldType or static constructor instead
   @Deprecated
   public MapVector(String name, BufferAllocator allocator, CallBack callBack) {
-    this(name, allocator, new FieldType(false, ArrowType.Struct.INSTANCE, null, null), callBack);
+    this(name, allocator, FieldType.nullable(ArrowType.Struct.INSTANCE), callBack);
   }
 
-  public MapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
-    super(name, allocator, callBack);
-    this.fieldType = checkNotNull(fieldType);
-    this.valueCount = 0;
-  }
-
-  @Override
-  public FieldReader getReader() {
-    return reader;
+  // deprecated, use FieldType or static constructor instead
+  @Deprecated
+  public MapVector(String name, BufferAllocator allocator, DictionaryEncoding dictionary, CallBack callBack) {
+    this(name, allocator, new FieldType(true, ArrowType.Struct.INSTANCE, dictionary, null), callBack);
   }
 
-  transient private MapTransferPair ephPair;
-
-  public void copyFromSafe(int fromIndex, int thisIndex, MapVector from) {
-    if (ephPair == null || ephPair.from != from) {
-      ephPair = (MapTransferPair) from.makeTransferPair(this);
-    }
-    ephPair.copyValueSafe(fromIndex, thisIndex);
+  public MapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
+    super(name, checkNotNull(allocator), fieldType, callBack);
+    this.validityBuffer = allocator.getEmpty();
+    this.validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(BaseValueVector.INITIAL_VALUE_ALLOCATION);
   }
 
   @Override
-  protected boolean supportsDirectRead() {
-    return true;
-  }
-
-  public Iterator<String> fieldNameIterator() {
-    return getChildFieldNames().iterator();
+  public Field getField() {
+    Field f = super.getField();
+    FieldType type = new FieldType(true, f.getType(), f.getFieldType().getDictionary(), f.getFieldType().getMetadata());
+    return new Field(f.getName(), type, f.getChildren());
   }
 
   @Override
-  public void setInitialCapacity(int numRecords) {
-    for (final ValueVector v : (Iterable<ValueVector>) this) {
-      v.setInitialCapacity(numRecords);
+  public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
+    if (ownBuffers.size() != 1) {
+      throw new IllegalArgumentException("Illegal buffer count, expected " + 1 + ", got: " + ownBuffers.size());
     }
-  }
 
-  @Override
-  public int getBufferSize() {
-    if (valueCount == 0 || size() == 0) {
-      return 0;
-    }
-    long buffer = 0;
-    for (final ValueVector v : (Iterable<ValueVector>) this) {
-      buffer += v.getBufferSize();
-    }
+    ArrowBuf bitBuffer = ownBuffers.get(0);
 
-    return (int) buffer;
+    validityBuffer.release();
+    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
+    valueCount = fieldNode.getLength();
+    validityAllocationSizeInBytes = validityBuffer.capacity();
   }
 
   @Override
-  public int getBufferSizeFor(final int valueCount) {
-    if (valueCount == 0) {
-      return 0;
-    }
+  public List<ArrowBuf> getFieldBuffers() {
+    List<ArrowBuf> result = new ArrayList<>(1);
+    setReaderAndWriterIndex();
+    result.add(validityBuffer);
 
-    long bufferSize = 0;
-    for (final ValueVector v : (Iterable<ValueVector>) this) {
-      bufferSize += v.getBufferSizeFor(valueCount);
-    }
+    return result;
+  }
 
-    return (int) bufferSize;
+  private void setReaderAndWriterIndex() {
+    validityBuffer.readerIndex(0);
+    validityBuffer.writerIndex(BitVectorHelper.getValidityBufferSize(valueCount));
   }
 
   @Override
-  public ArrowBuf getValidityBuffer() {
-    throw new UnsupportedOperationException();
+  @Deprecated
+  public List<BufferBacked> getFieldInnerVectors() {
+    throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
   }
 
   @Override
-  public ArrowBuf getDataBuffer() {
-    throw new UnsupportedOperationException();
+  public NullableMapReaderImpl getReader() {
+    return reader;
   }
 
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    throw new UnsupportedOperationException();
+  public NullableMapWriter getWriter() {
+    return writer;
   }
 
   @Override
   public TransferPair getTransferPair(BufferAllocator allocator) {
-    return getTransferPair(name, allocator, null);
+    return new NullableMapTransferPair(this, new MapVector(name, allocator, fieldType, null), false);
   }
 
   @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
-    return new MapTransferPair(this, new MapVector(name, allocator, fieldType, callBack), false);
+  public TransferPair makeTransferPair(ValueVector to) {
+    return new NullableMapTransferPair(this, (MapVector) to, true);
   }
 
   @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    return new MapTransferPair(this, (MapVector) to);
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
+    return new NullableMapTransferPair(this, new MapVector(ref, allocator, fieldType, null), false);
   }
 
   @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    return new MapTransferPair(this, new MapVector(ref, allocator, fieldType, callBack), false);
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
+    return new NullableMapTransferPair(this, new MapVector(ref, allocator, fieldType, callBack), false);
   }
 
-  protected static class MapTransferPair implements TransferPair {
-    private final TransferPair[] pairs;
-    private final MapVector from;
-    private final MapVector to;
+  protected class NullableMapTransferPair extends MapTransferPair {
 
-    public MapTransferPair(MapVector from, MapVector to) {
-      this(from, to, true);
-    }
+    private MapVector target;
 
-    protected MapTransferPair(MapVector from, MapVector to, boolean allocate) {
-      this.from = from;
-      this.to = to;
-      this.pairs = new TransferPair[from.size()];
-      this.to.ephPair = null;
-
-      int i = 0;
-      FieldVector vector;
-      for (String child : from.getChildFieldNames()) {
-        int preSize = to.size();
-        vector = from.getChild(child);
-        if (vector == null) {
-          continue;
-        }
-        //DRILL-1872: we add the child fields for the vector, looking up the field by name. For a map vector,
-        // the child fields may be nested fields of the top level child. For example if the structure
-        // of a child field is oa.oab.oabc then we add oa, then add oab to oa then oabc to oab.
-        // But the children member of a Materialized field is a HashSet. If the fields are added in the
-        // children HashSet, and the hashCode of the Materialized field includes the hash code of the
-        // children, the hashCode value of oa changes *after* the field has been added to the HashSet.
-        // (This is similar to what happens in ScanBatch where the children cannot be added till they are
-        // read). To take care of this, we ensure that the hashCode of the MaterializedField does not
-        // include the hashCode of the children but is based only on MaterializedField$key.
-        final FieldVector newVector = to.addOrGet(child, vector.getField().getFieldType(), vector.getClass());
-        if (allocate && to.size() != preSize) {
-          newVector.allocateNew();
-        }
-        pairs[i++] = vector.makeTransferPair(newVector);
-      }
+    protected NullableMapTransferPair(MapVector from, MapVector to, boolean allocate) {
+      super(from, to, allocate);
+      this.target = to;
     }
 
     @Override
     public void transfer() {
-      for (final TransferPair p : pairs) {
-        p.transfer();
-      }
-      to.valueCount = from.valueCount;
-      from.clear();
-    }
-
-    @Override
-    public ValueVector getTo() {
-      return to;
+      target.clear();
+      target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
+      super.transfer();
+      clear();
     }
 
     @Override
-    public void copyValueSafe(int from, int to) {
-      for (TransferPair p : pairs) {
-        p.copyValueSafe(from, to);
+    public void copyValueSafe(int fromIndex, int toIndex) {
+      while (toIndex >= target.getValidityBufferValueCapacity()) {
+        target.reallocValidityBuffer();
       }
+      BitVectorHelper.setValidityBit(target.validityBuffer, toIndex, isSet(fromIndex));
+      super.copyValueSafe(fromIndex, toIndex);
     }
 
     @Override
     public void splitAndTransfer(int startIndex, int length) {
-      for (TransferPair p : pairs) {
-        p.splitAndTransfer(startIndex, length);
+      target.clear();
+      splitAndTransferValidityBuffer(startIndex, length, target);
+      super.splitAndTransfer(startIndex, length);
+    }
+  }
+
+  /*
+   * transfer the validity.
+   */
+  private void splitAndTransferValidityBuffer(int startIndex, int length, MapVector target) {
+    assert startIndex + length <= valueCount;
+    int firstByteSource = BitVectorHelper.byteIndex(startIndex);
+    int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
+    int byteSizeTarget = BitVectorHelper.getValidityBufferSize(length);
+    int offset = startIndex % 8;
+
+    if (length > 0) {
+      if (offset == 0) {
+        // slice
+        if (target.validityBuffer != null) {
+          target.validityBuffer.release();
+        }
+        target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
+        target.validityBuffer.retain(1);
+      } else {
+        /* Copy data
+         * When the first bit starts from the middle of a byte (offset != 0),
+         * copy data from src BitVector.
+         * Each byte in the target is composed by a part in i-th byte,
+         * another part in (i+1)-th byte.
+         */
+        target.allocateValidityBuffer(byteSizeTarget);
+
+        for (int i = 0; i < byteSizeTarget - 1; i++) {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer, firstByteSource + i, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer, firstByteSource + i + 1, offset);
+
+          target.validityBuffer.setByte(i, (b1 + b2));
+        }
+
+        /* Copying the last piece is done in the following manner:
+         * if the source vector has 1 or more bytes remaining, we copy
+         * the last piece as a byte formed by shifting data
+         * from the current byte and the next byte.
+         *
+         * if the source vector has no more bytes remaining
+         * (we are at the last byte), we copy the last piece as a byte
+         * by shifting data from the current byte.
+         */
+        if ((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer,
+                  firstByteSource + byteSizeTarget, offset);
+
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
+        } else {
+          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
+                  firstByteSource + byteSizeTarget - 1, offset);
+          target.validityBuffer.setByte(byteSizeTarget - 1, b1);
+        }
       }
-      to.setValueCount(length);
     }
   }
 
+  /**
+   * 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);
+  }
+
+  /**
+   * Get the current value capacity for the vector
+   * @return number of elements that vector can hold.
+   */
   @Override
   public int getValueCapacity() {
-    if (size() == 0) {
-      return 0;
-    }
+    return Math.min(getValidityBufferValueCapacity(),
+            super.getValueCapacity());
+  }
 
-    final Ordering<ValueVector> natural = new Ordering<ValueVector>() {
-      @Override
-      public int compare(@Nullable ValueVector left, @Nullable ValueVector right) {
-        return Ints.compare(
-            checkNotNull(left).getValueCapacity(),
-            checkNotNull(right).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) {
+      for (ArrowBuf buffer : buffers) {
+        buffer.retain();
       }
-    };
+      clear();
+    }
 
-    return natural.min(getChildren()).getValueCapacity();
+    return buffers;
   }
 
+  /**
+   * Close the vector and release the associated buffers.
+   */
   @Override
-  public Accessor getAccessor() {
-    throw new UnsupportedOperationException("accessor is not needed for MAP");
+  public void close() {
+    clearValidityBuffer();
+    super.close();
   }
 
+  /**
+   * Same as {@link #close()}
+   */
   @Override
-  public Mutator getMutator() {
-    throw new UnsupportedOperationException("mutator is not needed for MAP");
+  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 Object getObject(int index) {
-    Map<String, Object> vv = new JsonStringHashMap<>();
-    for (String child : getChildFieldNames()) {
-      ValueVector v = getChild(child);
-      if (v != null && index < v.getValueCount()) {
-        Object value = v.getObject(index);
-        if (value != null) {
-          vv.put(child, value);
-        }
+  public int getBufferSize() {
+    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);
+  }
+
+  @Override
+  public void setInitialCapacity(int numRecords) {
+    validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(numRecords);
+    super.setInitialCapacity(numRecords);
+  }
+
+  @Override
+  public boolean allocateNewSafe() {
+    /* Boolean to keep track if all the memory allocations were successful
+     * Used in the case of composite vectors when we need to allocate multiple
+     * buffers for multiple vectors. If one of the allocations failed we need to
+     * clear all the memory that we allocated
+     */
+    boolean success = false;
+    try {
+      clear();
+      allocateValidityBuffer(validityAllocationSizeInBytes);
+      success = super.allocateNewSafe();
+    } finally {
+      if (!success) {
+        clear();
+        return false;
       }
     }
-    return vv;
+    return true;
+  }
+
+  private void allocateValidityBuffer(final long size) {
+    final int curSize = (int) size;
+    validityBuffer = allocator.buffer(curSize);
+    validityBuffer.readerIndex(0);
+    validityAllocationSizeInBytes = curSize;
+    validityBuffer.setZero(0, validityBuffer.capacity());
   }
 
   @Override
-  public boolean isNull(int index) { return false; }
+  public void reAlloc() {
+    /* reallocate the validity buffer */
+    reallocValidityBuffer();
+    super.reAlloc();
+  }
+
+  private void reallocValidityBuffer() {
+    final int currentBufferCapacity = validityBuffer.capacity();
+    long baseSize = validityAllocationSizeInBytes;
+
+    if (baseSize < (long) currentBufferCapacity) {
+      baseSize = (long) currentBufferCapacity;
+    }
+
+    long newAllocationSize = baseSize * 2L;
+    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
+
+    if (newAllocationSize > BaseValueVector.MAX_ALLOCATION_SIZE) {
+      throw new OversizedAllocationException("Unable to expand the buffer");
+    }
+
+    final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize);
+    newBuf.setZero(0, newBuf.capacity());
+    newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity);
+    validityBuffer.release(1);
+    validityBuffer = newBuf;
+    validityAllocationSizeInBytes = (int) newAllocationSize;
+  }
+
   @Override
-  public int getNullCount() { return 0; }
+  public long getValidityBufferAddress() {
+    return validityBuffer.memoryAddress();
+  }
 
-  public void get(int index, ComplexHolder holder) {
-    reader.setPosition(index);
-    holder.reader = reader;
+  @Override
+  public long getDataBufferAddress() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public int getValueCount() {
-    return valueCount;
+  public long getOffsetBufferAddress() {
+    throw new UnsupportedOperationException();
   }
 
-  public ValueVector getVectorById(int id) {
-  return getChildByOrdinal(id);
-}
+  @Override
+  public ArrowBuf getValidityBuffer() {
+    return validityBuffer;
+  }
 
   @Override
-  public void setValueCount(int valueCount) {
-    for (final ValueVector v : getChildren()) {
-      v.setValueCount(valueCount);
-    }
-    MapVector.this.valueCount = valueCount;
+  public ArrowBuf getDataBuffer() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public void clear() {
-    for (final ValueVector v : getChildren()) {
-      v.clear();
-    }
-    valueCount = 0;
+  public ArrowBuf getOffsetBuffer() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public Field getField() {
-    List<Field> children = new ArrayList<>();
-    for (ValueVector child : getChildren()) {
-      children.add(child.getField());
+  public Object getObject(int index) {
+    if (isSet(index) == 0) {
+      return null;
+    } else {
+      return super.getObject(index);
     }
-    return new Field(name, fieldType, children);
   }
 
   @Override
-  public MinorType getMinorType() {
-    return MinorType.MAP;
+  public void get(int index, ComplexHolder holder) {
+    holder.isSet = isSet(index);
+    super.get(index, holder);
+  }
+
+  public int getNullCount() {
+    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
+  }
+
+  public boolean isNull(int index) {
+    return isSet(index) == 0;
+  }
+
+  public int isSet(int index) {
+    final int byteIndex = index >> 3;
+    final byte b = validityBuffer.getByte(byteIndex);
+    final int bitIndex = index & 7;
+    return Long.bitCount(b & (1L << bitIndex));
+  }
+
+  public void setIndexDefined(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
+    }
+    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
+  }
+
+  public void setNull(int index) {
+    while (index >= getValidityBufferValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
+    }
+    BitVectorHelper.setValidityBit(validityBuffer, index, 0);
   }
 
   @Override
-  public void close() {
-    final Collection<FieldVector> vectors = getChildren();
-    for (final FieldVector v : vectors) {
-      v.close();
+  public void setValueCount(int valueCount) {
+    assert valueCount >= 0;
+    while (valueCount > getValidityBufferValueCapacity()) {
+      /* realloc the inner buffers if needed */
+      reallocValidityBuffer();
     }
-    vectors.clear();
+    super.setValueCount(valueCount);
+    this.valueCount = valueCount;
+  }
 
+  public void reset() {
     valueCount = 0;
-
-    super.close();
   }
 
-  public void initializeChildrenFromFields(List<Field> children) {
-    for (Field field : children) {
-      FieldVector vector = (FieldVector) this.add(field.getName(), field.getFieldType());
-      vector.initializeChildrenFromFields(field.getChildren());
-    }
+  @Override
+  @Deprecated
+  public Accessor getAccessor() {
+    throw new UnsupportedOperationException("Accessor is not supported for reading from Nullable MAP");
   }
 
-  public List<FieldVector> getChildrenFromFields() {
-    return getChildren();
+  @Override
+  @Deprecated
+  public Mutator getMutator() {
+    throw new UnsupportedOperationException("Mutator is not supported for writing to Nullable MAP");
   }
 }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/NonNullableMapVector.java
similarity index 88%
copy from java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
copy to java/vector/src/main/java/org/apache/arrow/vector/complex/NonNullableMapVector.java
index 6089a67..cc3ac41 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/NonNullableMapVector.java
@@ -47,12 +47,12 @@ import org.apache.arrow.vector.util.CallBack;
 import org.apache.arrow.vector.util.JsonStringHashMap;
 import org.apache.arrow.vector.util.TransferPair;
 
-public class MapVector extends AbstractMapVector {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class);
+public class NonNullableMapVector extends AbstractMapVector {
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NonNullableMapVector.class);
 
-  public static MapVector empty(String name, BufferAllocator allocator) {
+  public static NonNullableMapVector empty(String name, BufferAllocator allocator) {
     FieldType fieldType = new FieldType(false, ArrowType.Struct.INSTANCE, null, null);
-    return new MapVector(name, allocator, fieldType, null);
+    return new NonNullableMapVector(name, allocator, fieldType, null);
   }
 
   private final SingleMapReaderImpl reader = new SingleMapReaderImpl(this);
@@ -61,11 +61,11 @@ public class MapVector extends AbstractMapVector {
 
   // deprecated, use FieldType or static constructor instead
   @Deprecated
-  public MapVector(String name, BufferAllocator allocator, CallBack callBack) {
+  public NonNullableMapVector(String name, BufferAllocator allocator, CallBack callBack) {
     this(name, allocator, new FieldType(false, ArrowType.Struct.INSTANCE, null, null), callBack);
   }
 
-  public MapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
+  public NonNullableMapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
     super(name, allocator, callBack);
     this.fieldType = checkNotNull(fieldType);
     this.valueCount = 0;
@@ -78,7 +78,7 @@ public class MapVector extends AbstractMapVector {
 
   transient private MapTransferPair ephPair;
 
-  public void copyFromSafe(int fromIndex, int thisIndex, MapVector from) {
+  public void copyFromSafe(int fromIndex, int thisIndex, NonNullableMapVector from) {
     if (ephPair == null || ephPair.from != from) {
       ephPair = (MapTransferPair) from.makeTransferPair(this);
     }
@@ -150,29 +150,29 @@ public class MapVector extends AbstractMapVector {
 
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
-    return new MapTransferPair(this, new MapVector(name, allocator, fieldType, callBack), false);
+    return new MapTransferPair(this, new NonNullableMapVector(name, allocator, fieldType, callBack), false);
   }
 
   @Override
   public TransferPair makeTransferPair(ValueVector to) {
-    return new MapTransferPair(this, (MapVector) to);
+    return new MapTransferPair(this, (NonNullableMapVector) to);
   }
 
   @Override
   public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    return new MapTransferPair(this, new MapVector(ref, allocator, fieldType, callBack), false);
+    return new MapTransferPair(this, new NonNullableMapVector(ref, allocator, fieldType, callBack), false);
   }
 
   protected static class MapTransferPair implements TransferPair {
     private final TransferPair[] pairs;
-    private final MapVector from;
-    private final MapVector to;
+    private final NonNullableMapVector from;
+    private final NonNullableMapVector to;
 
-    public MapTransferPair(MapVector from, MapVector to) {
+    public MapTransferPair(NonNullableMapVector from, NonNullableMapVector to) {
       this(from, to, true);
     }
 
-    protected MapTransferPair(MapVector from, MapVector to, boolean allocate) {
+    protected MapTransferPair(NonNullableMapVector from, NonNullableMapVector to, boolean allocate) {
       this.from = from;
       this.to = to;
       this.pairs = new TransferPair[from.size()];
@@ -301,7 +301,7 @@ public class MapVector extends AbstractMapVector {
     for (final ValueVector v : getChildren()) {
       v.setValueCount(valueCount);
     }
-    MapVector.this.valueCount = valueCount;
+    NonNullableMapVector.this.valueCount = valueCount;
   }
 
   @Override
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
deleted file mode 100644
index e223d1c..0000000
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/NullableMapVector.java
+++ /dev/null
@@ -1,512 +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
- * <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;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import com.google.common.collect.ObjectArrays;
-
-import io.netty.buffer.ArrowBuf;
-import org.apache.arrow.memory.BaseAllocator;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.*;
-import org.apache.arrow.vector.complex.impl.NullableMapReaderImpl;
-import org.apache.arrow.vector.complex.impl.NullableMapWriter;
-import org.apache.arrow.vector.holders.ComplexHolder;
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-import org.apache.arrow.vector.types.pojo.ArrowType.Struct;
-import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
-import org.apache.arrow.vector.types.pojo.FieldType;
-import org.apache.arrow.vector.types.pojo.Field;
-import org.apache.arrow.vector.util.CallBack;
-import org.apache.arrow.vector.util.OversizedAllocationException;
-import org.apache.arrow.vector.util.TransferPair;
-
-public class NullableMapVector extends MapVector implements FieldVector {
-
-  public static NullableMapVector empty(String name, BufferAllocator allocator) {
-    FieldType fieldType = FieldType.nullable(Struct.INSTANCE);
-    return new NullableMapVector(name, allocator, fieldType, null);
-  }
-
-  private final NullableMapReaderImpl reader = new NullableMapReaderImpl(this);
-  private final NullableMapWriter writer = new NullableMapWriter(this);
-
-  protected ArrowBuf validityBuffer;
-  private int validityAllocationSizeInBytes;
-
-  // deprecated, use FieldType or static constructor instead
-  @Deprecated
-  public NullableMapVector(String name, BufferAllocator allocator, CallBack callBack) {
-    this(name, allocator, FieldType.nullable(ArrowType.Struct.INSTANCE), callBack);
-  }
-
-  // deprecated, use FieldType or static constructor instead
-  @Deprecated
-  public NullableMapVector(String name, BufferAllocator allocator, DictionaryEncoding dictionary, CallBack callBack) {
-    this(name, allocator, new FieldType(true, ArrowType.Struct.INSTANCE, dictionary, null), callBack);
-  }
-
-  public NullableMapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
-    super(name, checkNotNull(allocator), fieldType, callBack);
-    this.validityBuffer = allocator.getEmpty();
-    this.validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(BaseValueVector.INITIAL_VALUE_ALLOCATION);
-  }
-
-  @Override
-  public Field getField() {
-    Field f = super.getField();
-    FieldType type = new FieldType(true, f.getType(), f.getFieldType().getDictionary(), f.getFieldType().getMetadata());
-    return new Field(f.getName(), type, f.getChildren());
-  }
-
-  @Override
-  public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers) {
-    if (ownBuffers.size() != 1) {
-      throw new IllegalArgumentException("Illegal buffer count, expected " + 1 + ", got: " + ownBuffers.size());
-    }
-
-    ArrowBuf bitBuffer = ownBuffers.get(0);
-
-    validityBuffer.release();
-    validityBuffer = BitVectorHelper.loadValidityBuffer(fieldNode, bitBuffer, allocator);
-    valueCount = fieldNode.getLength();
-    validityAllocationSizeInBytes = validityBuffer.capacity();
-  }
-
-  @Override
-  public List<ArrowBuf> getFieldBuffers() {
-    List<ArrowBuf> result = new ArrayList<>(1);
-    setReaderAndWriterIndex();
-    result.add(validityBuffer);
-
-    return result;
-  }
-
-  private void setReaderAndWriterIndex() {
-    validityBuffer.readerIndex(0);
-    validityBuffer.writerIndex(BitVectorHelper.getValidityBufferSize(valueCount));
-  }
-
-  @Override
-  @Deprecated
-  public List<BufferBacked> getFieldInnerVectors() {
-    throw new UnsupportedOperationException("There are no inner vectors. Use getFieldBuffers");
-  }
-
-  @Override
-  public NullableMapReaderImpl getReader() {
-    return reader;
-  }
-
-  public NullableMapWriter getWriter() {
-    return writer;
-  }
-
-  @Override
-  public TransferPair getTransferPair(BufferAllocator allocator) {
-    return new NullableMapTransferPair(this, new NullableMapVector(name, allocator, fieldType, null), false);
-  }
-
-  @Override
-  public TransferPair makeTransferPair(ValueVector to) {
-    return new NullableMapTransferPair(this, (NullableMapVector) to, true);
-  }
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
-    return new NullableMapTransferPair(this, new NullableMapVector(ref, allocator, fieldType, null), false);
-  }
-
-  @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator, CallBack callBack) {
-    return new NullableMapTransferPair(this, new NullableMapVector(ref, allocator, fieldType, callBack), false);
-  }
-
-  protected class NullableMapTransferPair extends MapTransferPair {
-
-    private NullableMapVector target;
-
-    protected NullableMapTransferPair(NullableMapVector from, NullableMapVector to, boolean allocate) {
-      super(from, to, allocate);
-      this.target = to;
-    }
-
-    @Override
-    public void transfer() {
-      target.clear();
-      target.validityBuffer = validityBuffer.transferOwnership(target.allocator).buffer;
-      super.transfer();
-      clear();
-    }
-
-    @Override
-    public void copyValueSafe(int fromIndex, int toIndex) {
-      while (toIndex >= target.getValidityBufferValueCapacity()) {
-        target.reallocValidityBuffer();
-      }
-      BitVectorHelper.setValidityBit(target.validityBuffer, toIndex, isSet(fromIndex));
-      super.copyValueSafe(fromIndex, toIndex);
-    }
-
-    @Override
-    public void splitAndTransfer(int startIndex, int length) {
-      target.clear();
-      splitAndTransferValidityBuffer(startIndex, length, target);
-      super.splitAndTransfer(startIndex, length);
-    }
-  }
-
-  /*
-   * transfer the validity.
-   */
-  private void splitAndTransferValidityBuffer(int startIndex, int length, NullableMapVector target) {
-    assert startIndex + length <= valueCount;
-    int firstByteSource = BitVectorHelper.byteIndex(startIndex);
-    int lastByteSource = BitVectorHelper.byteIndex(valueCount - 1);
-    int byteSizeTarget = BitVectorHelper.getValidityBufferSize(length);
-    int offset = startIndex % 8;
-
-    if (length > 0) {
-      if (offset == 0) {
-        // slice
-        if (target.validityBuffer != null) {
-          target.validityBuffer.release();
-        }
-        target.validityBuffer = validityBuffer.slice(firstByteSource, byteSizeTarget);
-        target.validityBuffer.retain(1);
-      } else {
-        /* Copy data
-         * When the first bit starts from the middle of a byte (offset != 0),
-         * copy data from src BitVector.
-         * Each byte in the target is composed by a part in i-th byte,
-         * another part in (i+1)-th byte.
-         */
-        target.allocateValidityBuffer(byteSizeTarget);
-
-        for (int i = 0; i < byteSizeTarget - 1; i++) {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer, firstByteSource + i, offset);
-          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer, firstByteSource + i + 1, offset);
-
-          target.validityBuffer.setByte(i, (b1 + b2));
-        }
-
-        /* Copying the last piece is done in the following manner:
-         * if the source vector has 1 or more bytes remaining, we copy
-         * the last piece as a byte formed by shifting data
-         * from the current byte and the next byte.
-         *
-         * if the source vector has no more bytes remaining
-         * (we are at the last byte), we copy the last piece as a byte
-         * by shifting data from the current byte.
-         */
-        if ((firstByteSource + byteSizeTarget - 1) < lastByteSource) {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
-                  firstByteSource + byteSizeTarget - 1, offset);
-          byte b2 = BitVectorHelper.getBitsFromNextByte(validityBuffer,
-                  firstByteSource + byteSizeTarget, offset);
-
-          target.validityBuffer.setByte(byteSizeTarget - 1, b1 + b2);
-        } else {
-          byte b1 = BitVectorHelper.getBitsFromCurrentByte(validityBuffer,
-                  firstByteSource + byteSizeTarget - 1, offset);
-          target.validityBuffer.setByte(byteSizeTarget - 1, b1);
-        }
-      }
-    }
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * 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) {
-      for (ArrowBuf buffer : buffers) {
-        buffer.retain();
-      }
-      clear();
-    }
-
-    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;
-    }
-    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);
-  }
-
-  @Override
-  public void setInitialCapacity(int numRecords) {
-    validityAllocationSizeInBytes = BitVectorHelper.getValidityBufferSize(numRecords);
-    super.setInitialCapacity(numRecords);
-  }
-
-  @Override
-  public boolean allocateNewSafe() {
-    /* Boolean to keep track if all the memory allocations were successful
-     * Used in the case of composite vectors when we need to allocate multiple
-     * buffers for multiple vectors. If one of the allocations failed we need to
-     * clear all the memory that we allocated
-     */
-    boolean success = false;
-    try {
-      clear();
-      allocateValidityBuffer(validityAllocationSizeInBytes);
-      success = super.allocateNewSafe();
-    } finally {
-      if (!success) {
-        clear();
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private void allocateValidityBuffer(final long size) {
-    final int curSize = (int) size;
-    validityBuffer = allocator.buffer(curSize);
-    validityBuffer.readerIndex(0);
-    validityAllocationSizeInBytes = curSize;
-    validityBuffer.setZero(0, validityBuffer.capacity());
-  }
-
-  @Override
-  public void reAlloc() {
-    /* reallocate the validity buffer */
-    reallocValidityBuffer();
-    super.reAlloc();
-  }
-
-  private void reallocValidityBuffer() {
-    final int currentBufferCapacity = validityBuffer.capacity();
-    long baseSize = validityAllocationSizeInBytes;
-
-    if (baseSize < (long) currentBufferCapacity) {
-      baseSize = (long) currentBufferCapacity;
-    }
-
-    long newAllocationSize = baseSize * 2L;
-    newAllocationSize = BaseAllocator.nextPowerOfTwo(newAllocationSize);
-
-    if (newAllocationSize > BaseValueVector.MAX_ALLOCATION_SIZE) {
-      throw new OversizedAllocationException("Unable to expand the buffer");
-    }
-
-    final ArrowBuf newBuf = allocator.buffer((int) newAllocationSize);
-    newBuf.setZero(0, newBuf.capacity());
-    newBuf.setBytes(0, validityBuffer, 0, currentBufferCapacity);
-    validityBuffer.release(1);
-    validityBuffer = newBuf;
-    validityAllocationSizeInBytes = (int) newAllocationSize;
-  }
-
-  @Override
-  public long getValidityBufferAddress() {
-    return validityBuffer.memoryAddress();
-  }
-
-  @Override
-  public long getDataBufferAddress() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getOffsetBufferAddress() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrowBuf getValidityBuffer() {
-    return validityBuffer;
-  }
-
-  @Override
-  public ArrowBuf getDataBuffer() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ArrowBuf getOffsetBuffer() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Object getObject(int index) {
-    if (isSet(index) == 0) {
-      return null;
-    } else {
-      return super.getObject(index);
-    }
-  }
-
-  @Override
-  public void get(int index, ComplexHolder holder) {
-    holder.isSet = isSet(index);
-    super.get(index, holder);
-  }
-
-  public int getNullCount() {
-    return BitVectorHelper.getNullCount(validityBuffer, valueCount);
-  }
-
-  public boolean isNull(int index) {
-    return isSet(index) == 0;
-  }
-
-  public int isSet(int index) {
-    final int byteIndex = index >> 3;
-    final byte b = validityBuffer.getByte(byteIndex);
-    final int bitIndex = index & 7;
-    return Long.bitCount(b & (1L << bitIndex));
-  }
-
-  public void setIndexDefined(int index) {
-    while (index >= getValidityBufferValueCapacity()) {
-      /* realloc the inner buffers if needed */
-      reallocValidityBuffer();
-    }
-    BitVectorHelper.setValidityBitToOne(validityBuffer, index);
-  }
-
-  public void setNull(int index) {
-    while (index >= getValidityBufferValueCapacity()) {
-      /* realloc the inner buffers if needed */
-      reallocValidityBuffer();
-    }
-    BitVectorHelper.setValidityBit(validityBuffer, index, 0);
-  }
-
-  @Override
-  public void setValueCount(int valueCount) {
-    assert valueCount >= 0;
-    while (valueCount > getValidityBufferValueCapacity()) {
-      /* realloc the inner buffers if needed */
-      reallocValidityBuffer();
-    }
-    super.setValueCount(valueCount);
-    this.valueCount = valueCount;
-  }
-
-  public void reset() {
-    valueCount = 0;
-  }
-
-  @Override
-  @Deprecated
-  public Accessor getAccessor() {
-    throw new UnsupportedOperationException("Accessor is not supported for reading from Nullable MAP");
-  }
-
-  @Override
-  @Deprecated
-  public Mutator getMutator() {
-    throw new UnsupportedOperationException("Mutator is not supported for writing to Nullable MAP");
-  }
-}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/ComplexWriterImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/ComplexWriterImpl.java
index a696023..970b90e 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/ComplexWriterImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/ComplexWriterImpl.java
@@ -20,7 +20,7 @@ package org.apache.arrow.vector.complex.impl;
 
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.StateTool;
 import org.apache.arrow.vector.complex.writer.BaseWriter.ComplexWriter;
 import org.apache.arrow.vector.types.pojo.Field;
@@ -131,7 +131,7 @@ public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWri
     switch (mode) {
 
       case INIT:
-        mapRoot = nullableMapWriterFactory.build((NullableMapVector) container);
+        mapRoot = nullableMapWriterFactory.build((MapVector) container);
         mapRoot.setPosition(idx());
         mode = Mode.MAP;
         break;
@@ -152,7 +152,7 @@ public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWri
 
       case INIT:
         // TODO allow dictionaries in complex types
-        NullableMapVector map = container.addOrGetMap(name);
+        MapVector map = container.addOrGetMap(name);
         mapRoot = nullableMapWriterFactory.build(map);
         mapRoot.setPosition(idx());
         mode = Mode.MAP;
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
index 06b0f4d..acf155a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapReaderImpl.java
@@ -20,17 +20,17 @@
 package org.apache.arrow.vector.complex.impl;
 
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.arrow.vector.types.pojo.Field;
 
 public class NullableMapReaderImpl extends SingleMapReaderImpl {
 
-  private NullableMapVector nullableMapVector;
+  private MapVector nullableMapVector;
 
   public NullableMapReaderImpl(MapVector vector) {
-    super((NullableMapVector) vector);
-    this.nullableMapVector = (NullableMapVector) vector;
+    super((MapVector) vector);
+    this.nullableMapVector = (MapVector) vector;
   }
 
   @Override
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapWriterFactory.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapWriterFactory.java
index d2dcb23..06dbf78 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapWriterFactory.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/NullableMapWriterFactory.java
@@ -18,7 +18,7 @@
 
 package org.apache.arrow.vector.complex.impl;
 
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 
 public class NullableMapWriterFactory {
   private final boolean caseSensitive;
@@ -29,7 +29,7 @@ public class NullableMapWriterFactory {
     this.caseSensitive = caseSensitive;
   }
 
-  public NullableMapWriter build(NullableMapVector container) {
+  public NullableMapWriter build(MapVector container) {
     return this.caseSensitive ? new NullableCaseSensitiveMapWriter(container) : new NullableMapWriter(container);
   }
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
index 5bd439c..9abd38d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/PromotableWriter.java
@@ -23,7 +23,7 @@ import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.ZeroVector;
 import org.apache.arrow.vector.complex.AbstractMapVector;
 import org.apache.arrow.vector.complex.ListVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.UnionVector;
 import org.apache.arrow.vector.complex.writer.FieldWriter;
 import org.apache.arrow.vector.types.Types.MinorType;
@@ -94,7 +94,7 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
     type = v.getMinorType();
     switch (type) {
       case MAP:
-        writer = nullableMapWriterFactory.build((NullableMapVector) vector);
+        writer = nullableMapWriterFactory.build((MapVector) vector);
         break;
       case LIST:
         writer = new UnionListWriter((ListVector) vector, nullableMapWriterFactory);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
index c77ca4e..0341b62 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/SingleMapReaderImpl.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.NonNullableMapVector;
 import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.arrow.vector.types.Types.MinorType;
@@ -34,10 +35,10 @@ import com.google.common.collect.Maps;
 @SuppressWarnings("unused")
 public class SingleMapReaderImpl extends AbstractFieldReader {
 
-  private final MapVector vector;
+  private final NonNullableMapVector vector;
   private final Map<String, FieldReader> fields = Maps.newHashMap();
 
-  public SingleMapReaderImpl(MapVector vector) {
+  public SingleMapReaderImpl(NonNullableMapVector vector) {
     this.vector = vector;
   }
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
index 6243a28..62bc271 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionListReader.java
@@ -20,7 +20,6 @@
 package org.apache.arrow.vector.complex.impl;
 
 import io.netty.buffer.ArrowBuf;
-import org.apache.arrow.vector.UInt4Vector;
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.reader.FieldReader;
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
index cb11a25..04d5033 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileReader.java
@@ -222,7 +222,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT1 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableTinyIntVector.TYPE_WIDTH;
+        final int size = count * TinyIntVector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -237,7 +237,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT2 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableSmallIntVector.TYPE_WIDTH;
+        final int size = count * SmallIntVector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -252,7 +252,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT4 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableIntVector.TYPE_WIDTH;
+        final int size = count * IntVector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -267,7 +267,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader INT8 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableBigIntVector.TYPE_WIDTH;
+        final int size = count * BigIntVector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -282,7 +282,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader FLOAT4 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableFloat4Vector.TYPE_WIDTH;
+        final int size = count * Float4Vector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -297,7 +297,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader FLOAT8 = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableFloat8Vector.TYPE_WIDTH;
+        final int size = count * Float8Vector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
@@ -312,7 +312,7 @@ public class JsonFileReader implements AutoCloseable, DictionaryProvider {
     BufferReader DECIMAL = new BufferReader() {
       @Override
       protected ArrowBuf read(BufferAllocator allocator, int count) throws IOException {
-        final int size = count * NullableDecimalVector.TYPE_WIDTH;
+        final int size = count * DecimalVector.TYPE_WIDTH;
         ArrowBuf buf = allocator.buffer(size);
 
         for (int i = 0; i < count; i++) {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
index 22423b8..067fb25 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/ipc/JsonFileWriter.java
@@ -160,8 +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;
+      final int scale = (vector instanceof DecimalVector) ?
+                            ((DecimalVector) vector).getScale() : 0;
       for (int v = 0; v < vectorTypes.size(); v++) {
         ArrowVectorType vectorType = vectorTypes.get(v);
         ArrowBuf vectorBuffer = vectorBuffers.get(v);
@@ -199,86 +199,86 @@ public class JsonFileWriter implements AutoCloseable {
                                      ArrowBuf offsetBuffer, FieldVector vector,
                                      final int index, final int scale) throws IOException {
     if (bufferType.equals(TYPE)) {
-      generator.writeNumber(buffer.getByte(index * NullableTinyIntVector.TYPE_WIDTH));
+      generator.writeNumber(buffer.getByte(index * TinyIntVector.TYPE_WIDTH));
     } else if (bufferType.equals(OFFSET)) {
-      generator.writeNumber(buffer.getInt(index * BaseNullableVariableWidthVector.OFFSET_WIDTH));
+      generator.writeNumber(buffer.getInt(index * BaseVariableWidthVector.OFFSET_WIDTH));
     } else if(bufferType.equals(VALIDITY)) {
       generator.writeNumber(vector.isNull(index) ? 0 : 1);
     } else if (bufferType.equals(DATA)) {
       switch (vector.getMinorType()) {
         case TINYINT:
-          generator.writeNumber(NullableTinyIntVector.get(buffer, index));
+          generator.writeNumber(TinyIntVector.get(buffer, index));
           break;
         case SMALLINT:
-          generator.writeNumber(NullableSmallIntVector.get(buffer, index));
+          generator.writeNumber(SmallIntVector.get(buffer, index));
           break;
         case INT:
-          generator.writeNumber(NullableIntVector.get(buffer, index));
+          generator.writeNumber(IntVector.get(buffer, index));
           break;
         case BIGINT:
-          generator.writeNumber(NullableBigIntVector.get(buffer, index));
+          generator.writeNumber(BigIntVector.get(buffer, index));
           break;
         case FLOAT4:
-          generator.writeNumber(NullableFloat4Vector.get(buffer, index));
+          generator.writeNumber(Float4Vector.get(buffer, index));
           break;
         case FLOAT8:
-          generator.writeNumber(NullableFloat8Vector.get(buffer, index));
+          generator.writeNumber(Float8Vector.get(buffer, index));
           break;
         case DATEDAY:
-          generator.writeNumber(NullableDateDayVector.get(buffer, index));
+          generator.writeNumber(DateDayVector.get(buffer, index));
           break;
         case DATEMILLI:
-          generator.writeNumber(NullableDateMilliVector.get(buffer, index));
+          generator.writeNumber(DateMilliVector.get(buffer, index));
           break;
         case TIMESEC:
-          generator.writeNumber(NullableTimeSecVector.get(buffer, index));
+          generator.writeNumber(TimeSecVector.get(buffer, index));
           break;
         case TIMEMILLI:
-          generator.writeNumber(NullableTimeMilliVector.get(buffer, index));
+          generator.writeNumber(TimeMilliVector.get(buffer, index));
           break;
         case TIMEMICRO:
-          generator.writeNumber(NullableTimeMicroVector.get(buffer, index));
+          generator.writeNumber(TimeMicroVector.get(buffer, index));
           break;
         case TIMENANO:
-          generator.writeNumber(NullableTimeNanoVector.get(buffer, index));
+          generator.writeNumber(TimeNanoVector.get(buffer, index));
           break;
         case TIMESTAMPSEC:
-          generator.writeNumber(NullableTimeStampSecVector.get(buffer, index));
+          generator.writeNumber(TimeStampSecVector.get(buffer, index));
           break;
         case TIMESTAMPMILLI:
-          generator.writeNumber(NullableTimeStampMilliVector.get(buffer, index));
+          generator.writeNumber(TimeStampMilliVector.get(buffer, index));
           break;
         case TIMESTAMPMICRO:
-          generator.writeNumber(NullableTimeStampMicroVector.get(buffer, index));
+          generator.writeNumber(TimeStampMicroVector.get(buffer, index));
           break;
         case TIMESTAMPNANO:
-          generator.writeNumber(NullableTimeStampNanoVector.get(buffer, index));
+          generator.writeNumber(TimeStampNanoVector.get(buffer, index));
           break;
         case TIMESTAMPSECTZ:
-          generator.writeNumber(NullableTimeStampSecTZVector.get(buffer, index));
+          generator.writeNumber(TimeStampSecTZVector.get(buffer, index));
           break;
         case TIMESTAMPMILLITZ:
-          generator.writeNumber(NullableTimeStampMilliTZVector.get(buffer, index));
+          generator.writeNumber(TimeStampMilliTZVector.get(buffer, index));
           break;
         case TIMESTAMPMICROTZ:
-          generator.writeNumber(NullableTimeStampMicroTZVector.get(buffer, index));
+          generator.writeNumber(TimeStampMicroTZVector.get(buffer, index));
           break;
         case TIMESTAMPNANOTZ:
-          generator.writeNumber(NullableTimeStampNanoTZVector.get(buffer, index));
+          generator.writeNumber(TimeStampNanoTZVector.get(buffer, index));
           break;
         case BIT:
           generator.writeNumber(BitVectorHelper.get(buffer, index));
           break;
         case VARBINARY: {
           assert offsetBuffer != null;
-          String hexString = Hex.encodeHexString(BaseNullableVariableWidthVector.get(buffer,
+          String hexString = Hex.encodeHexString(BaseVariableWidthVector.get(buffer,
                   offsetBuffer, index));
           generator.writeObject(hexString);
           break;
         }
         case VARCHAR: {
           assert offsetBuffer != null;
-          byte[] b = (BaseNullableVariableWidthVector.get(buffer, offsetBuffer, index));
+          byte[] b = (BaseVariableWidthVector.get(buffer, offsetBuffer, index));
           generator.writeString(new String(b, "UTF-8"));
           break;
         }
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java b/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
index c57dd6d..7834845 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/Types.java
@@ -24,41 +24,41 @@ import static org.apache.arrow.vector.types.UnionMode.Sparse;
 
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.NullableBigIntVector;
-import org.apache.arrow.vector.NullableBitVector;
-import org.apache.arrow.vector.NullableDateDayVector;
-import org.apache.arrow.vector.NullableDateMilliVector;
-import org.apache.arrow.vector.NullableDecimalVector;
-import org.apache.arrow.vector.NullableFloat4Vector;
-import org.apache.arrow.vector.NullableFloat8Vector;
-import org.apache.arrow.vector.NullableIntVector;
-import org.apache.arrow.vector.NullableIntervalDayVector;
-import org.apache.arrow.vector.NullableIntervalYearVector;
-import org.apache.arrow.vector.NullableSmallIntVector;
-import org.apache.arrow.vector.NullableTimeMicroVector;
-import org.apache.arrow.vector.NullableTimeMilliVector;
-import org.apache.arrow.vector.NullableTimeNanoVector;
-import org.apache.arrow.vector.NullableTimeSecVector;
-import org.apache.arrow.vector.NullableTimeStampMicroTZVector;
-import org.apache.arrow.vector.NullableTimeStampMicroVector;
-import org.apache.arrow.vector.NullableTimeStampMilliTZVector;
-import org.apache.arrow.vector.NullableTimeStampMilliVector;
-import org.apache.arrow.vector.NullableTimeStampNanoTZVector;
-import org.apache.arrow.vector.NullableTimeStampNanoVector;
-import org.apache.arrow.vector.NullableTimeStampSecTZVector;
-import org.apache.arrow.vector.NullableTimeStampSecVector;
-import org.apache.arrow.vector.NullableTinyIntVector;
-import org.apache.arrow.vector.NullableUInt1Vector;
-import org.apache.arrow.vector.NullableUInt2Vector;
-import org.apache.arrow.vector.NullableUInt4Vector;
-import org.apache.arrow.vector.NullableUInt8Vector;
-import org.apache.arrow.vector.NullableVarBinaryVector;
-import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.IntervalDayVector;
+import org.apache.arrow.vector.IntervalYearVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampNanoVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampSecVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.ZeroVector;
 import org.apache.arrow.vector.complex.FixedSizeListVector;
 import org.apache.arrow.vector.complex.ListVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.UnionVector;
 import org.apache.arrow.vector.complex.impl.BigIntWriterImpl;
 import org.apache.arrow.vector.complex.impl.BitWriterImpl;
@@ -131,304 +131,304 @@ public class Types {
     MAP(Struct.INSTANCE) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableMapVector(name, allocator, fieldType, schemaChangeCallback);
+        return new MapVector(name, allocator, fieldType, schemaChangeCallback);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new NullableMapWriter((NullableMapVector) vector);
+        return new NullableMapWriter((MapVector) vector);
       }
     },
     TINYINT(new Int(8, true)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTinyIntVector(name, fieldType, allocator);
+        return new TinyIntVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TinyIntWriterImpl((NullableTinyIntVector) vector);
+        return new TinyIntWriterImpl((TinyIntVector) vector);
       }
     },
     SMALLINT(new Int(16, true)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableSmallIntVector(name, fieldType, allocator);
+        return new SmallIntVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new SmallIntWriterImpl((NullableSmallIntVector) vector);
+        return new SmallIntWriterImpl((SmallIntVector) vector);
       }
     },
     INT(new Int(32, true)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableIntVector(name, fieldType, allocator);
+        return new IntVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new IntWriterImpl((NullableIntVector) vector);
+        return new IntWriterImpl((IntVector) vector);
       }
     },
     BIGINT(new Int(64, true)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableBigIntVector(name, fieldType, allocator);
+        return new BigIntVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new BigIntWriterImpl((NullableBigIntVector) vector);
+        return new BigIntWriterImpl((BigIntVector) vector);
       }
     },
     DATEDAY(new Date(DateUnit.DAY)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableDateDayVector(name, fieldType, allocator);
+        return new DateDayVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new DateDayWriterImpl((NullableDateDayVector) vector);
+        return new DateDayWriterImpl((DateDayVector) vector);
       }
     },
     DATEMILLI(new Date(DateUnit.MILLISECOND)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableDateMilliVector(name, fieldType, allocator);
+        return new DateMilliVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new DateMilliWriterImpl((NullableDateMilliVector) vector);
+        return new DateMilliWriterImpl((DateMilliVector) vector);
       }
     },
     TIMESEC(new Time(TimeUnit.SECOND, 32)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeSecVector(name, fieldType, allocator);
+        return new TimeSecVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeSecWriterImpl((NullableTimeSecVector) vector);
+        return new TimeSecWriterImpl((TimeSecVector) vector);
       }
     },
     TIMEMILLI(new Time(TimeUnit.MILLISECOND, 32)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeMilliVector(name, fieldType, allocator);
+        return new TimeMilliVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeMilliWriterImpl((NullableTimeMilliVector) vector);
+        return new TimeMilliWriterImpl((TimeMilliVector) vector);
       }
     },
     TIMEMICRO(new Time(TimeUnit.MICROSECOND, 64)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeMicroVector(name, fieldType, allocator);
+        return new TimeMicroVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeMicroWriterImpl((NullableTimeMicroVector) vector);
+        return new TimeMicroWriterImpl((TimeMicroVector) vector);
       }
     },
     TIMENANO(new Time(TimeUnit.NANOSECOND, 64)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeNanoVector(name, fieldType, allocator);
+        return new TimeNanoVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeNanoWriterImpl((NullableTimeNanoVector) vector);
+        return new TimeNanoWriterImpl((TimeNanoVector) vector);
       }
     },
     // time in second from the Unix epoch, 00:00:00.000000 on 1 January 1970, UTC.
     TIMESTAMPSEC(new Timestamp(org.apache.arrow.vector.types.TimeUnit.SECOND, null)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampSecVector(name, fieldType, allocator);
+        return new TimeStampSecVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampSecWriterImpl((NullableTimeStampSecVector) vector);
+        return new TimeStampSecWriterImpl((TimeStampSecVector) vector);
       }
     },
     // time in millis from the Unix epoch, 00:00:00.000 on 1 January 1970, UTC.
     TIMESTAMPMILLI(new Timestamp(org.apache.arrow.vector.types.TimeUnit.MILLISECOND, null)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampMilliVector(name, fieldType, allocator);
+        return new TimeStampMilliVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampMilliWriterImpl((NullableTimeStampMilliVector) vector);
+        return new TimeStampMilliWriterImpl((TimeStampMilliVector) vector);
       }
     },
     // time in microsecond from the Unix epoch, 00:00:00.000000 on 1 January 1970, UTC.
     TIMESTAMPMICRO(new Timestamp(org.apache.arrow.vector.types.TimeUnit.MICROSECOND, null)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampMicroVector(name, fieldType, allocator);
+        return new TimeStampMicroVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampMicroWriterImpl((NullableTimeStampMicroVector) vector);
+        return new TimeStampMicroWriterImpl((TimeStampMicroVector) vector);
       }
     },
     // time in nanosecond from the Unix epoch, 00:00:00.000000000 on 1 January 1970, UTC.
     TIMESTAMPNANO(new Timestamp(org.apache.arrow.vector.types.TimeUnit.NANOSECOND, null)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampNanoVector(name, fieldType, allocator);
+        return new TimeStampNanoVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampNanoWriterImpl((NullableTimeStampNanoVector) vector);
+        return new TimeStampNanoWriterImpl((TimeStampNanoVector) vector);
       }
     },
     INTERVALDAY(new Interval(IntervalUnit.DAY_TIME)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableIntervalDayVector(name, fieldType, allocator);
+        return new IntervalDayVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new IntervalDayWriterImpl((NullableIntervalDayVector) vector);
+        return new IntervalDayWriterImpl((IntervalDayVector) vector);
       }
     },
     INTERVALYEAR(new Interval(IntervalUnit.YEAR_MONTH)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableIntervalYearVector(name, fieldType, allocator);
+        return new IntervalYearVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new IntervalYearWriterImpl((NullableIntervalYearVector) vector);
+        return new IntervalYearWriterImpl((IntervalYearVector) vector);
       }
     },
     //  4 byte ieee 754
     FLOAT4(new FloatingPoint(SINGLE)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableFloat4Vector(name, fieldType, allocator);
+        return new Float4Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new Float4WriterImpl((NullableFloat4Vector) vector);
+        return new Float4WriterImpl((Float4Vector) vector);
       }
     },
     //  8 byte ieee 754
     FLOAT8(new FloatingPoint(DOUBLE)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableFloat8Vector(name, fieldType, allocator);
+        return new Float8Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new Float8WriterImpl((NullableFloat8Vector) vector);
+        return new Float8WriterImpl((Float8Vector) vector);
       }
     },
     BIT(Bool.INSTANCE) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableBitVector(name, fieldType, allocator);
+        return new BitVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new BitWriterImpl((NullableBitVector) vector);
+        return new BitWriterImpl((BitVector) vector);
       }
     },
     VARCHAR(Utf8.INSTANCE) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableVarCharVector(name, fieldType, allocator);
+        return new VarCharVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new VarCharWriterImpl((NullableVarCharVector) vector);
+        return new VarCharWriterImpl((VarCharVector) vector);
       }
     },
     VARBINARY(Binary.INSTANCE) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableVarBinaryVector(name, fieldType, allocator);
+        return new VarBinaryVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new VarBinaryWriterImpl((NullableVarBinaryVector) vector);
+        return new VarBinaryWriterImpl((VarBinaryVector) vector);
       }
     },
     DECIMAL(null) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableDecimalVector(name, fieldType, allocator);
+        return new DecimalVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new DecimalWriterImpl((NullableDecimalVector) vector);
+        return new DecimalWriterImpl((DecimalVector) vector);
       }
     },
     UINT1(new Int(8, false)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableUInt1Vector(name, fieldType, allocator);
+        return new UInt1Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new UInt1WriterImpl((NullableUInt1Vector) vector);
+        return new UInt1WriterImpl((UInt1Vector) vector);
       }
     },
     UINT2(new Int(16, false)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableUInt2Vector(name, fieldType, allocator);
+        return new UInt2Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new UInt2WriterImpl((NullableUInt2Vector) vector);
+        return new UInt2WriterImpl((UInt2Vector) vector);
       }
     },
     UINT4(new Int(32, false)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableUInt4Vector(name, fieldType, allocator);
+        return new UInt4Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new UInt4WriterImpl((NullableUInt4Vector) vector);
+        return new UInt4WriterImpl((UInt4Vector) vector);
       }
     },
     UINT8(new Int(64, false)) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableUInt8Vector(name, fieldType, allocator);
+        return new UInt8Vector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new UInt8WriterImpl((NullableUInt8Vector) vector);
+        return new UInt8WriterImpl((UInt8Vector) vector);
       }
     },
     LIST(List.INSTANCE) {
@@ -470,45 +470,45 @@ public class Types {
     TIMESTAMPSECTZ(null) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampSecTZVector(name, fieldType, allocator);
+        return new TimeStampSecTZVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampSecTZWriterImpl((NullableTimeStampSecTZVector) vector);
+        return new TimeStampSecTZWriterImpl((TimeStampSecTZVector) vector);
       }
     },
     TIMESTAMPMILLITZ(null) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampMilliTZVector(name, fieldType, allocator);
+        return new TimeStampMilliTZVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampMilliTZWriterImpl((NullableTimeStampMilliTZVector) vector);
+        return new TimeStampMilliTZWriterImpl((TimeStampMilliTZVector) vector);
       }
     },
     TIMESTAMPMICROTZ(null) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampMicroTZVector(name, fieldType, allocator);
+        return new TimeStampMicroTZVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampMicroTZWriterImpl((NullableTimeStampMicroTZVector) vector);
+        return new TimeStampMicroTZWriterImpl((TimeStampMicroTZVector) vector);
       }
     },
     TIMESTAMPNANOTZ(null) {
       @Override
       public FieldVector getNewVector(String name, FieldType fieldType, BufferAllocator allocator, CallBack schemaChangeCallback) {
-        return new NullableTimeStampNanoTZVector(name, fieldType, allocator);
+        return new TimeStampNanoTZVector(name, fieldType, allocator);
       }
 
       @Override
       public FieldWriter getNewFieldWriter(ValueVector vector) {
-        return new TimeStampNanoTZWriterImpl((NullableTimeStampNanoTZVector) vector);
+        return new TimeStampNanoTZWriterImpl((TimeStampNanoTZVector) vector);
       }
     };
 
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java b/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
index acf7c58..c6d734d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
@@ -19,7 +19,6 @@
 package org.apache.arrow.vector.util;
 
 import io.netty.buffer.ArrowBuf;
-import org.apache.arrow.vector.DecimalVector;
 import org.apache.arrow.vector.types.pojo.ArrowType;
 
 import java.math.BigDecimal;
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
index ada3414..36365fa 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestBitVector.java
@@ -55,17 +55,17 @@ public class TestBitVector {
       dst.allocateNew(10);
 
       for (int i = 0; i < size; i++) {
-        src.getMutator().set(i, i % 2);
+        src.set(i, i % 2);
       }
-      src.getMutator().setValueCount(size);
+      src.setValueCount(size);
 
       for (int i = 0; i < size; i++) {
         dst.copyFromSafe(i, i, src);
       }
-      dst.getMutator().setValueCount(size);
+      dst.setValueCount(size);
 
       for (int i = 0; i < size; i++) {
-        assertEquals(src.getAccessor().getObject(i), dst.getAccessor().getObject(i));
+        assertEquals(src.getObject(i), dst.getObject(i));
       }
     }
   }
@@ -74,25 +74,23 @@ public class TestBitVector {
   public void testSplitAndTransfer() throws Exception {
 
     try (final BitVector sourceVector = new BitVector("bitvector", allocator)) {
-      final BitVector.Mutator sourceMutator = sourceVector.getMutator();
-      final BitVector.Accessor sourceAccessor = sourceVector.getAccessor();
 
       sourceVector.allocateNew(40);
 
       /* populate the bitvector -- 010101010101010101010101..... */
       for (int i = 0; i < 40; i++) {
         if ((i & 1) == 1) {
-          sourceMutator.set(i, 1);
+          sourceVector.set(i, 1);
         } else {
-          sourceMutator.set(i, 0);
+          sourceVector.set(i, 0);
         }
       }
 
-      sourceMutator.setValueCount(40);
+      sourceVector.setValueCount(40);
 
       /* check the vector output */
       for (int i = 0; i < 40; i++) {
-        int result = sourceAccessor.get(i);
+        int result = sourceVector.get(i);
         if ((i & 1) == 1) {
           assertEquals(Integer.toString(1), Integer.toString(result));
         } else {
@@ -102,8 +100,6 @@ public class TestBitVector {
 
       try (final BitVector toVector = new BitVector("toVector", allocator)) {
         final TransferPair transferPair = sourceVector.makeTransferPair(toVector);
-        final BitVector.Accessor toAccessor = toVector.getAccessor();
-        final BitVector.Mutator toMutator = toVector.getMutator();
 
         /*
          * form test cases such that we cover:
@@ -123,8 +119,8 @@ public class TestBitVector {
 
           /* check the toVector output after doing splitAndTransfer */
           for (int i = 0; i < length; i++) {
-            int actual = toAccessor.get(i);
-            int expected = sourceAccessor.get(start + i);
+            int actual = toVector.get(i);
+            int expected = sourceVector.get(start + i);
             assertEquals("different data values not expected --> sourceVector index: " + (start + i) + " toVector index: " + i,
                     expected, actual);
           }
@@ -137,28 +133,24 @@ public class TestBitVector {
   public void testSplitAndTransfer1() throws Exception {
 
     try (final BitVector sourceVector = new BitVector("bitvector", allocator)) {
-      final BitVector.Mutator sourceMutator = sourceVector.getMutator();
-      final BitVector.Accessor sourceAccessor = sourceVector.getAccessor();
 
       sourceVector.allocateNew(8190);
 
       /* populate the bitvector */
       for (int i = 0; i < 8190; i++) {
-        sourceMutator.set(i, 1);
+        sourceVector.set(i, 1);
       }
 
-      sourceMutator.setValueCount(8190);
+      sourceVector.setValueCount(8190);
 
       /* check the vector output */
       for (int i = 0; i < 8190; i++) {
-        int result = sourceAccessor.get(i);
+        int result = sourceVector.get(i);
         assertEquals(Integer.toString(1), Integer.toString(result));
       }
 
       try (final BitVector toVector = new BitVector("toVector", allocator)) {
         final TransferPair transferPair = sourceVector.makeTransferPair(toVector);
-        final BitVector.Accessor toAccessor = toVector.getAccessor();
-        final BitVector.Mutator toMutator = toVector.getMutator();
 
         final int[][] transferLengths = {{0, 4095}, {4095, 4095}};
 
@@ -170,8 +162,8 @@ public class TestBitVector {
 
           /* check the toVector output after doing splitAndTransfer */
           for (int i = 0; i < length; i++) {
-            int actual = toAccessor.get(i);
-            int expected = sourceAccessor.get(start + i);
+            int actual = toVector.get(i);
+            int expected = sourceVector.get(start + i);
             assertEquals("different data values not expected --> sourceVector index: " + (start + i) + " toVector index: " + i,
                     expected, actual);
           }
@@ -184,25 +176,23 @@ public class TestBitVector {
   public void testSplitAndTransfer2() throws Exception {
 
     try (final BitVector sourceVector = new BitVector("bitvector", allocator)) {
-      final BitVector.Mutator sourceMutator = sourceVector.getMutator();
-      final BitVector.Accessor sourceAccessor = sourceVector.getAccessor();
 
       sourceVector.allocateNew(32);
 
       /* populate the bitvector */
       for (int i = 0; i < 32; i++) {
         if ((i & 1) == 1) {
-          sourceMutator.set(i, 1);
+          sourceVector.set(i, 1);
         } else {
-          sourceMutator.set(i, 0);
+          sourceVector.set(i, 0);
         }
       }
 
-      sourceMutator.setValueCount(32);
+      sourceVector.setValueCount(32);
 
       /* check the vector output */
       for (int i = 0; i < 32; i++) {
-        int result = sourceAccessor.get(i);
+        int result = sourceVector.get(i);
         if ((i & 1) == 1) {
           assertEquals(Integer.toString(1), Integer.toString(result));
         } else {
@@ -212,8 +202,6 @@ public class TestBitVector {
 
       try (final BitVector toVector = new BitVector("toVector", allocator)) {
         final TransferPair transferPair = sourceVector.makeTransferPair(toVector);
-        final BitVector.Accessor toAccessor = toVector.getAccessor();
-        final BitVector.Mutator toMutator = toVector.getMutator();
 
         final int[][] transferLengths = {{5,22}, {5,24}, {5,25}, {5,27}, {0,31}, {5,7}, {2,3}};
 
@@ -225,8 +213,8 @@ public class TestBitVector {
 
           /* check the toVector output after doing splitAndTransfer */
           for (int i = 0; i < length; i++) {
-            int actual = toAccessor.get(i);
-            int expected = sourceAccessor.get(start + i);
+            int actual = toVector.get(i);
+            int expected = sourceVector.get(start + i);
             assertEquals("different data values not expected --> sourceVector index: " + (start + i) + " toVector index: " + i,
                     expected, actual);
           }
@@ -242,62 +230,56 @@ public class TestBitVector {
       int valueCapacity = vector.getValueCapacity();
       assertEquals(4096, valueCapacity);
 
-      final BitVector.Mutator mutator = vector.getMutator();
-      final BitVector.Accessor accessor = vector.getAccessor();
-
       for (int i = 0; i < valueCapacity; i++) {
         if ((i & 1) == 1) {
-          mutator.setToOne(i);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity; i++) {
-        int val = accessor.get(i);
         if ((i & 1) == 1) {
-          assertEquals("unexpected cleared bit at index: " + i, 1, val);
+          assertEquals("unexpected cleared bit at index: " + i, 1, vector.get(i));
         }
         else {
-          assertEquals("unexpected set bit at index: " + i, 0, val);
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
       /* trigger first realloc */
-      mutator.setSafeToOne(valueCapacity);
+      vector.setSafe(valueCapacity, 1);
       assertEquals(valueCapacity * 2, vector.getValueCapacity());
 
       for (int i = valueCapacity; i < valueCapacity*2; i++) {
         if ((i & 1) == 1) {
-          mutator.setToOne(i);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity*2; i++) {
-        int val = accessor.get(i);
         if (((i & 1) == 1) || (i == valueCapacity)) {
-          assertEquals("unexpected cleared bit at index: " + i, 1, val);
+          assertEquals("unexpected cleared bit at index: " + i, 1, vector.get(i));
         }
         else {
-          assertEquals("unexpected set bit at index: " + i, 0, val);
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
       /* trigger second realloc */
-      mutator.setSafeToOne(valueCapacity*2);
+      vector.setSafe(valueCapacity*2, 1);
       assertEquals(valueCapacity * 4, vector.getValueCapacity());
 
       for (int i = valueCapacity*2; i < valueCapacity*4; i++) {
         if ((i & 1) == 1) {
-          mutator.setToOne(i);
+          vector.set(i, 1);
         }
       }
 
       for (int i = 0; i < valueCapacity*4; i++) {
-        int val = accessor.get(i);
         if (((i & 1) == 1) || (i == valueCapacity) || (i == valueCapacity*2)) {
-          assertEquals("unexpected cleared bit at index: " + i, 1, val);
+          assertEquals("unexpected cleared bit at index: " + i, 1, vector.get(i));
         }
         else {
-          assertEquals("unexpected set bit at index: " + i, 0, val);
+          assertTrue("unexpected set bit at index: " + i, vector.isNull(i));
         }
       }
 
@@ -305,27 +287,24 @@ public class TestBitVector {
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
       final BitVector toVector = (BitVector)transferPair.getTo();
-      final BitVector.Accessor toAccessor = toVector.getAccessor();
-      final BitVector.Mutator toMutator = toVector.getMutator();
 
       assertEquals(valueCapacity * 4, toVector.getValueCapacity());
 
       /* realloc the toVector */
-      toMutator.setSafeToOne(valueCapacity * 4);
+      toVector.setSafe(valueCapacity * 4, 1);
 
       for (int i = 0; i < toVector.getValueCapacity(); i++) {
-        int val = toAccessor.get(i);
         if (i <= valueCapacity * 4) {
           if (((i & 1) == 1) || (i == valueCapacity) ||
                   (i == valueCapacity*2) || (i == valueCapacity*4)) {
-            assertEquals("unexpected cleared bit at index: " + i, 1, val);
+            assertEquals("unexpected cleared bit at index: " + i, 1, toVector.get(i));
           }
           else {
-            assertEquals("unexpected set bit at index: " + i, 0, val);
+            assertTrue("unexpected set bit at index: " + i, toVector.isNull(i));
           }
         }
         else {
-          assertEquals("unexpected set bit at index: " + i, 0, val);
+          assertTrue("unexpected set bit at index: " + i, toVector.isNull(i));
         }
       }
 
@@ -335,7 +314,7 @@ public class TestBitVector {
 
   @Test
   public void testReallocAfterVectorTransfer2() {
-    try (final NullableBitVector vector = new NullableBitVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final BitVector vector = new BitVector(EMPTY_SCHEMA_PATH, allocator)) {
       vector.allocateNew(4096);
       int valueCapacity = vector.getValueCapacity();
       assertEquals(4096, valueCapacity);
@@ -396,7 +375,7 @@ public class TestBitVector {
       /* now transfer the vector */
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
-      final NullableBitVector toVector = (NullableBitVector)transferPair.getTo();
+      final BitVector toVector = (BitVector)transferPair.getTo();
 
       assertEquals(valueCapacity * 4, toVector.getValueCapacity());
 
@@ -426,81 +405,81 @@ public class TestBitVector {
   public void testBitVector() {
     // Create a new value vector for 1024 integers
     try (final BitVector vector = new BitVector(EMPTY_SCHEMA_PATH, allocator)) {
-      final BitVector.Mutator m = vector.getMutator();
       vector.allocateNew(1024);
-      m.setValueCount(1024);
+      vector.setValueCount(1024);
 
       // Put and set a few values
-      m.set(0, 1);
-      m.set(1, 0);
-      m.set(100, 0);
-      m.set(1022, 1);
+      vector.set(0, 1);
+      vector.set(1, 0);
+      vector.set(100, 0);
+      vector.set(1022, 1);
 
-      m.setValueCount(1024);
+      vector.setValueCount(1024);
 
-      final BitVector.Accessor accessor = vector.getAccessor();
-      assertEquals(1, accessor.get(0));
-      assertEquals(0, accessor.get(1));
-      assertEquals(0, accessor.get(100));
-      assertEquals(1, accessor.get(1022));
+      assertEquals(1, vector.get(0));
+      assertEquals(0, vector.get(1));
+      assertEquals(0, vector.get(100));
+      assertEquals(1, vector.get(1022));
 
-      assertEquals(1022, accessor.getNullCount());
+      assertEquals(1020, vector.getNullCount());
 
       // test setting the same value twice
-      m.set(0, 1);
-      m.set(0, 1);
-      m.set(1, 0);
-      m.set(1, 0);
-      assertEquals(1, accessor.get(0));
-      assertEquals(0, accessor.get(1));
+      vector.set(0, 1);
+      vector.set(0, 1);
+      vector.set(1, 0);
+      vector.set(1, 0);
+      assertEquals(1, vector.get(0));
+      assertEquals(0, vector.get(1));
 
       // test toggling the values
-      m.set(0, 0);
-      m.set(1, 1);
-      assertEquals(0, accessor.get(0));
-      assertEquals(1, accessor.get(1));
+      vector.set(0, 0);
+      vector.set(1, 1);
+      assertEquals(0, vector.get(0));
+      assertEquals(1, vector.get(1));
 
       // should not change
-      assertEquals(1022, accessor.getNullCount());
+      assertEquals(1020, vector.getNullCount());
 
-      // Ensure unallocated space returns 0
-      assertEquals(0, accessor.get(3));
+      // Ensure null value
+      assertTrue(vector.isNull(3));
 
       // unset the previously set bits
-      m.set(1, 0);
-      m.set(1022, 0);
+      vector.setNull(0);
+      vector.setNull(1);
+      vector.setNull(100);
+      vector.setNull(1022);
       // this should set all the array to 0
-      assertEquals(1024, accessor.getNullCount());
+      assertEquals(1024, vector.getNullCount());
 
       // set all the array to 1
       for (int i = 0; i < 1024; ++i) {
-        assertEquals(1024 - i, accessor.getNullCount());
-        m.set(i, 1);
+        assertEquals(1024 - i, vector.getNullCount());
+        vector.set(i, 1);
       }
 
-      assertEquals(0, accessor.getNullCount());
+      assertEquals(0, vector.getNullCount());
 
       vector.allocateNew(1015);
-      m.setValueCount(1015);
+      vector.setValueCount(1015);
 
       // ensure it has been zeroed
-      assertEquals(1015, accessor.getNullCount());
+      assertEquals(1015, vector.getNullCount());
 
-      m.set(0, 1);
-      m.set(1014, 1); // ensure that the last item of the last byte is allocated
+      vector.set(0, 1);
+      vector.set(1014, 1); // ensure that the last item of the last byte is allocated
 
-      assertEquals(1013, accessor.getNullCount());
+      assertEquals(1013, vector.getNullCount());
 
       vector.zeroVector();
-      assertEquals(1015, accessor.getNullCount());
+      assertEquals(1015, vector.getNullCount());
 
       // set all the array to 1
       for (int i = 0; i < 1015; ++i) {
-        assertEquals(1015 - i, accessor.getNullCount());
-        m.set(i, 1);
+        assertEquals(1015 - i, vector.getNullCount());
+        vector.set(i, 1);
       }
 
-      assertEquals(0, accessor.getNullCount());
+      assertEquals(0, vector.getNullCount());
     }
   }
 
@@ -526,15 +505,17 @@ public class TestBitVector {
     try (BitVector bitVector = new BitVector("bits", allocator)) {
       bitVector.reset();
       bitVector.allocateNew(length);
-      bitVector.getMutator().setRangeToOne(start, count);
+      for (int i = start; i < start + count; i++) {
+        bitVector.set(i, 1);
+      }
       for (int i = 0; i < start; i++) {
-        Assert.assertEquals(desc + i, 0, bitVector.getAccessor().get(i));
+        Assert.assertTrue(desc + i, bitVector.isNull(i));
       }
       for (int i = start; i < start + count; i++) {
-        Assert.assertEquals(desc + i, 1, bitVector.getAccessor().get(i));
+        Assert.assertEquals(desc + i, 1, bitVector.get(i));
       }
       for (int i = start + count; i < length; i++) {
-        Assert.assertEquals(desc + i, 0, bitVector.getAccessor().get(i));
+        Assert.assertTrue(desc + i, bitVector.isNull(i));
       }
     }
   }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java b/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java
index 4fe2861..20f6754 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestBufferOwnershipTransfer.java
@@ -38,11 +38,11 @@ public class TestBufferOwnershipTransfer {
     BufferAllocator childAllocator1 = allocator.newChildAllocator("child1", 100000, 100000);
     BufferAllocator childAllocator2 = allocator.newChildAllocator("child2", 100000, 100000);
 
-    NullableIntVector v1 = new NullableIntVector("v1", childAllocator1);
+    IntVector v1 = new IntVector("v1", childAllocator1);
     v1.allocateNew();
     v1.setValueCount(4095);
 
-    NullableIntVector v2 = new NullableIntVector("v2", childAllocator2);
+    IntVector v2 = new IntVector("v2", childAllocator2);
 
     v1.makeTransferPair(v2).transfer();
 
@@ -58,12 +58,12 @@ public class TestBufferOwnershipTransfer {
     BufferAllocator childAllocator1 = allocator.newChildAllocator("child1", 100000, 100000);
     BufferAllocator childAllocator2 = allocator.newChildAllocator("child2", 100000, 100000);
 
-    NullableVarCharVector v1 = new NullableVarCharVector("v1", childAllocator1);
+    VarCharVector v1 = new VarCharVector("v1", childAllocator1);
     v1.allocateNew();
     v1.setSafe(4094, "hello world".getBytes(), 0, 11);
     v1.setValueCount(4001);
 
-    NullableVarCharVector v2 = new NullableVarCharVector("v2", childAllocator2);
+    VarCharVector v2 = new VarCharVector("v2", childAllocator2);
 
     v1.makeTransferPair(v2).transfer();
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestCopyFrom.java b/java/vector/src/test/java/org/apache/arrow/vector/TestCopyFrom.java
index 7cdfbdf..87ffcaf 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestCopyFrom.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestCopyFrom.java
@@ -70,8 +70,8 @@ public class TestCopyFrom {
 
   @Test /* NullableVarChar */
   public void testCopyFromWithNulls() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator);
-         final NullableVarCharVector vector2 = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator)) {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator);
+         final VarCharVector vector2 = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator)) {
 
       vector.allocateNew();
       int capacity = vector.getValueCapacity();
@@ -130,8 +130,8 @@ public class TestCopyFrom {
 
   @Test /* NullableVarChar */
   public void testCopyFromWithNulls1() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator);
-         final NullableVarCharVector vector2 = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator)) {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator);
+         final VarCharVector vector2 = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, Types.MinorType.VARCHAR, allocator)) {
 
       vector.allocateNew();
       int capacity = vector.getValueCapacity();
@@ -192,10 +192,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableIntVector */
+  @Test /* IntVector */
   public void testCopyFromWithNulls2() {
-    try (final NullableIntVector vector1 = new NullableIntVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableIntVector vector2 = new NullableIntVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final IntVector vector1 = new IntVector(EMPTY_SCHEMA_PATH, allocator);
+         final IntVector vector2 = new IntVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -252,10 +252,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableBigIntVector */
+  @Test /* BigIntVector */
   public void testCopyFromWithNulls3() {
-    try (final NullableBigIntVector vector1 = new NullableBigIntVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableBigIntVector vector2 = new NullableBigIntVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final BigIntVector vector1 = new BigIntVector(EMPTY_SCHEMA_PATH, allocator);
+         final BigIntVector vector2 = new BigIntVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -314,10 +314,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableBitVector */
+  @Test /* BitVector */
   public void testCopyFromWithNulls4() {
-    try (final NullableBitVector vector1 = new NullableBitVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableBitVector vector2 = new NullableBitVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final BitVector vector1 = new BitVector(EMPTY_SCHEMA_PATH, allocator);
+         final BitVector vector2 = new BitVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -392,10 +392,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableFloat4Vector */
+  @Test /* Float4Vector */
   public void testCopyFromWithNulls5() {
-    try (final NullableFloat4Vector vector1 = new NullableFloat4Vector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableFloat4Vector vector2 = new NullableFloat4Vector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final Float4Vector vector1 = new Float4Vector(EMPTY_SCHEMA_PATH, allocator);
+         final Float4Vector vector2 = new Float4Vector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -454,10 +454,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableFloat8Vector */
+  @Test /* Float8Vector */
   public void testCopyFromWithNulls6() {
-    try (final NullableFloat8Vector vector1 = new NullableFloat8Vector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableFloat8Vector vector2 = new NullableFloat8Vector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final Float8Vector vector1 = new Float8Vector(EMPTY_SCHEMA_PATH, allocator);
+         final Float8Vector vector2 = new Float8Vector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -516,10 +516,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableIntervalDayVector */
+  @Test /* IntervalDayVector */
   public void testCopyFromWithNulls7() {
-    try (final NullableIntervalDayVector vector1 = new NullableIntervalDayVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableIntervalDayVector vector2 = new NullableIntervalDayVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final IntervalDayVector vector1 = new IntervalDayVector(EMPTY_SCHEMA_PATH, allocator);
+         final IntervalDayVector vector2 = new IntervalDayVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -582,10 +582,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableIntervalYearVector */
+  @Test /* IntervalYearVector */
   public void testCopyFromWithNulls8() {
-    try (final NullableIntervalYearVector vector1 = new NullableIntervalYearVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableIntervalYearVector vector2 = new NullableIntervalYearVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final IntervalYearVector vector1 = new IntervalYearVector(EMPTY_SCHEMA_PATH, allocator);
+         final IntervalYearVector vector2 = new IntervalYearVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -651,10 +651,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableSmallIntVector */
+  @Test /* SmallIntVector */
   public void testCopyFromWithNulls9() {
-    try (final NullableSmallIntVector vector1 = new NullableSmallIntVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableSmallIntVector vector2 = new NullableSmallIntVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final SmallIntVector vector1 = new SmallIntVector(EMPTY_SCHEMA_PATH, allocator);
+         final SmallIntVector vector2 = new SmallIntVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -714,10 +714,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableTimeMicroVector */
+  @Test /* TimeMicroVector */
   public void testCopyFromWithNulls10() {
-    try (final NullableTimeMicroVector vector1 = new NullableTimeMicroVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableTimeMicroVector vector2 = new NullableTimeMicroVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final TimeMicroVector vector1 = new TimeMicroVector(EMPTY_SCHEMA_PATH, allocator);
+         final TimeMicroVector vector2 = new TimeMicroVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -777,10 +777,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableTimeMilliVector */
+  @Test /* TimeMilliVector */
   public void testCopyFromWithNulls11() {
-    try (final NullableTimeMilliVector vector1 = new NullableTimeMilliVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableTimeMilliVector vector2 = new NullableTimeMilliVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final TimeMilliVector vector1 = new TimeMilliVector(EMPTY_SCHEMA_PATH, allocator);
+         final TimeMilliVector vector2 = new TimeMilliVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -840,10 +840,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableTinyIntVector */
+  @Test /* TinyIntVector */
   public void testCopyFromWithNulls12() {
-    try (final NullableTinyIntVector vector1 = new NullableTinyIntVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableTinyIntVector vector2 = new NullableTinyIntVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final TinyIntVector vector1 = new TinyIntVector(EMPTY_SCHEMA_PATH, allocator);
+         final TinyIntVector vector2 = new TinyIntVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -906,10 +906,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableDecimalVector */
+  @Test /* DecimalVector */
   public void testCopyFromWithNulls13() {
-    try (final NullableDecimalVector vector1 = new NullableDecimalVector(EMPTY_SCHEMA_PATH, allocator, 30, 16);
-         final NullableDecimalVector vector2 = new NullableDecimalVector(EMPTY_SCHEMA_PATH, allocator, 30, 16)) {
+    try (final DecimalVector vector1 = new DecimalVector(EMPTY_SCHEMA_PATH, allocator, 30, 16);
+         final DecimalVector vector2 = new DecimalVector(EMPTY_SCHEMA_PATH, allocator, 30, 16)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
@@ -972,10 +972,10 @@ public class TestCopyFrom {
     }
   }
 
-  @Test /* NullableTimeStampVector */
+  @Test /* TimeStampVector */
   public void testCopyFromWithNulls14() {
-    try (final NullableTimeStampVector vector1 = new NullableTimeStampMicroVector(EMPTY_SCHEMA_PATH, allocator);
-         final NullableTimeStampVector vector2 = new NullableTimeStampMicroVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final TimeStampVector vector1 = new TimeStampMicroVector(EMPTY_SCHEMA_PATH, allocator);
+         final TimeStampVector vector2 = new TimeStampMicroVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       vector1.allocateNew();
       assertEquals(4096, vector1.getValueCapacity());
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
index 4d844d6..a48c035 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
@@ -60,9 +60,9 @@ public class TestDecimalVector {
 
   @Test
   public void testValuesWriteRead() {
-    try (NullableDecimalVector decimalVector = TestUtils.newVector(NullableDecimalVector.class, "decimal", new ArrowType.Decimal(10, scale), allocator);) {
+    try (DecimalVector decimalVector = TestUtils.newVector(DecimalVector.class, "decimal", new ArrowType.Decimal(10, scale), allocator);) {
 
-      try (NullableDecimalVector oldConstructor = new NullableDecimalVector("decimal", allocator, 10, scale);) {
+      try (DecimalVector oldConstructor = new DecimalVector("decimal", allocator, 10, scale);) {
         assertEquals(decimalVector.getField().getType(), oldConstructor.getField().getType());
       }
 
@@ -85,7 +85,7 @@ public class TestDecimalVector {
 
   @Test
   public void testBigDecimalDifferentScaleAndPrecision() {
-    try (NullableDecimalVector decimalVector = TestUtils.newVector(NullableDecimalVector.class, "decimal", new ArrowType.Decimal(4, 2), allocator);) {
+    try (DecimalVector decimalVector = TestUtils.newVector(DecimalVector.class, "decimal", new ArrowType.Decimal(4, 2), allocator);) {
       decimalVector.allocateNew();
 
       // test BigDecimal with different scale
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
index 46a2baf..ba149bc 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java
@@ -18,7 +18,7 @@
 
 package org.apache.arrow.vector;
 
-import static org.apache.arrow.vector.TestUtils.newNullableVarCharVector;
+import static org.apache.arrow.vector.TestUtils.newVarCharVector;
 import static org.junit.Assert.assertEquals;
 
 import java.nio.charset.StandardCharsets;
@@ -54,8 +54,8 @@ public class TestDictionaryVector {
   @Test
   public void testEncodeStrings() {
     // Create a new value vector
-    try (final NullableVarCharVector vector = newNullableVarCharVector("foo", allocator);
-         final NullableVarCharVector dictionaryVector = newNullableVarCharVector("dict", allocator);) {
+    try (final VarCharVector vector = newVarCharVector("foo", allocator);
+         final VarCharVector dictionaryVector = newVarCharVector("dict", allocator);) {
       vector.allocateNew(512, 5);
 
       // set some values
@@ -77,9 +77,9 @@ public class TestDictionaryVector {
 
       try (final ValueVector encoded = (FieldVector) DictionaryEncoder.encode(vector, dictionary)) {
         // verify indices
-        assertEquals(NullableIntVector.class, encoded.getClass());
+        assertEquals(IntVector.class, encoded.getClass());
 
-        NullableIntVector index = ((NullableIntVector)encoded);
+        IntVector index = ((IntVector)encoded);
         assertEquals(5, index.getValueCount());
         assertEquals(0, index.get(0));
         assertEquals(1, index.get(1));
@@ -90,9 +90,9 @@ public class TestDictionaryVector {
         // now run through the decoder and verify we get the original back
         try (ValueVector decoded = DictionaryEncoder.decode(encoded, dictionary)) {
           assertEquals(vector.getClass(), decoded.getClass());
-          assertEquals(vector.getValueCount(), ((NullableVarCharVector)decoded).getValueCount());
+          assertEquals(vector.getValueCount(), ((VarCharVector)decoded).getValueCount());
           for (int i = 0; i < 5; i++) {
-            assertEquals(vector.getObject(i), ((NullableVarCharVector)decoded).getObject(i));
+            assertEquals(vector.getObject(i), ((VarCharVector)decoded).getObject(i));
           }
         }
       }
@@ -102,8 +102,8 @@ public class TestDictionaryVector {
   @Test
   public void testEncodeLargeVector() {
     // Create a new value vector
-    try (final NullableVarCharVector vector = newNullableVarCharVector("foo", allocator);
-         final NullableVarCharVector dictionaryVector = newNullableVarCharVector("dict", allocator);) {
+    try (final VarCharVector vector = newVarCharVector("foo", allocator);
+         final VarCharVector dictionaryVector = newVarCharVector("dict", allocator);) {
       vector.allocateNew();
 
       int count = 10000;
@@ -124,9 +124,9 @@ public class TestDictionaryVector {
 
       try (final ValueVector encoded = (FieldVector) DictionaryEncoder.encode(vector, dictionary)) {
         // verify indices
-        assertEquals(NullableIntVector.class, encoded.getClass());
+        assertEquals(IntVector.class, encoded.getClass());
 
-        NullableIntVector index = ((NullableIntVector) encoded);
+        IntVector index = ((IntVector) encoded);
         assertEquals(count, index.getValueCount());
         for (int i = 0; i < count; ++i) {
           assertEquals(i % 3, index.get(i));
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
index 2af6cd5..50438ce 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestFixedSizeListVector.java
@@ -53,7 +53,7 @@ public class TestFixedSizeListVector {
   @Test
   public void testIntType() {
     try (FixedSizeListVector vector = FixedSizeListVector.empty("list", 2, allocator)) {
-      NullableIntVector nested = (NullableIntVector) vector.addOrGetVector(FieldType.nullable(MinorType.INT.getType())).getVector();
+      IntVector nested = (IntVector) vector.addOrGetVector(FieldType.nullable(MinorType.INT.getType())).getVector();
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
@@ -80,7 +80,7 @@ public class TestFixedSizeListVector {
   @Test
   public void testFloatTypeNullable() {
     try (FixedSizeListVector vector = FixedSizeListVector.empty("list", 2, allocator)) {
-      NullableFloat4Vector nested = (NullableFloat4Vector) vector.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
+      Float4Vector nested = (Float4Vector) vector.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
@@ -115,7 +115,7 @@ public class TestFixedSizeListVector {
   public void testNestedInList() {
     try (ListVector vector = ListVector.empty("list", allocator)) {
       FixedSizeListVector tuples = (FixedSizeListVector) vector.addOrGetVector(FieldType.nullable(new ArrowType.FixedSizeList(2))).getVector();
-      NullableIntVector innerVector = (NullableIntVector) tuples.addOrGetVector(FieldType.nullable(MinorType.INT.getType())).getVector();
+      IntVector innerVector = (IntVector) tuples.addOrGetVector(FieldType.nullable(MinorType.INT.getType())).getVector();
       vector.allocateNew();
 
       for (int i = 0; i < 10; i++) {
@@ -157,7 +157,7 @@ public class TestFixedSizeListVector {
   public void testTransferPair() {
     try (FixedSizeListVector from = new FixedSizeListVector("from", allocator, 2, null, null);
          FixedSizeListVector to = new FixedSizeListVector("to", allocator, 2, null, null)) {
-      NullableFloat4Vector nested = (NullableFloat4Vector) from.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
+      Float4Vector nested = (Float4Vector) from.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
       from.allocateNew();
 
       for (int i = 0; i < 10; i++) {
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
index f6aa86a..1cff7fa 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestListVector.java
@@ -131,8 +131,8 @@ public class TestListVector {
       ArrowBuf validityBuffer = listVector.getValidityBuffer();
       ArrowBuf offsetBuffer = listVector.getOffsetBuffer();
 
-      /* get the underlying data vector -- NullableBigIntVector */
-      NullableBigIntVector dataVector = (NullableBigIntVector) listVector.getDataVector();
+      /* get the underlying data vector -- BigIntVector */
+      BigIntVector dataVector = (BigIntVector) listVector.getDataVector();
 
       /* check current lastSet */
       assertEquals(Integer.toString(0), Integer.toString(listVector.getLastSet()));
@@ -310,7 +310,7 @@ public class TestListVector {
       final ArrowBuf offsetBuffer = listVector.getOffsetBuffer();
 
       /* get dataVector */
-      NullableBigIntVector dataVector = (NullableBigIntVector) listVector.getDataVector();
+      BigIntVector dataVector = (BigIntVector) listVector.getDataVector();
 
       /* check the vector output */
 
@@ -418,7 +418,7 @@ public class TestListVector {
           final ArrowBuf toOffsetBuffer = toVector.getOffsetBuffer();
 
           /* get dataVector of toVector */
-          NullableBigIntVector dataVector1 = (NullableBigIntVector) toVector.getDataVector();
+          BigIntVector dataVector1 = (BigIntVector) toVector.getDataVector();
 
           for (int i = 0; i < splitLength; i++) {
             dataLength1 = offsetBuffer.getInt((start + i + 1) * ListVector.OFFSET_WIDTH) -
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestMapVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestMapVector.java
index 357df96..e568295 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestMapVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestMapVector.java
@@ -22,7 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.types.pojo.ArrowType.Struct;
 import org.apache.arrow.vector.types.pojo.FieldType;
 import org.junit.After;
@@ -50,7 +50,7 @@ public class TestMapVector {
     Map<String, String> metadata = new HashMap<>();
     metadata.put("k1", "v1");
     FieldType type = new FieldType(true, Struct.INSTANCE, null, metadata);
-    try (NullableMapVector vector = new NullableMapVector("map", allocator, type, null)) {
+    try (MapVector vector = new MapVector("map", allocator, type, null)) {
       Assert.assertEquals(vector.getField().getMetadata(), type.getMetadata());
     }
   }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestOversizedAllocationForValueVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestOversizedAllocationForValueVector.java
index ba2ebbf..f14dbd6 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestOversizedAllocationForValueVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestOversizedAllocationForValueVector.java
@@ -112,10 +112,10 @@ public class TestOversizedAllocationForValueVector {
     try {
       vector.allocateNew(expectedAllocationInBytes, 10);
       assertTrue(expectedOffsetSize <= vector.getValueCapacity());
-      assertTrue(expectedAllocationInBytes <= vector.getBuffer().capacity());
+      assertTrue(expectedAllocationInBytes <= vector.getDataBuffer().capacity());
       vector.reAlloc();
       assertTrue(expectedOffsetSize * 2 <= vector.getValueCapacity());
-      assertTrue(expectedAllocationInBytes * 2 <= vector.getBuffer().capacity());
+      assertTrue(expectedAllocationInBytes * 2 <= vector.getDataBuffer().capacity());
     } finally {
       vector.close();
     }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestSplitAndTransfer.java b/java/vector/src/test/java/org/apache/arrow/vector/TestSplitAndTransfer.java
index 7de3bcb..80d5fe1 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestSplitAndTransfer.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestSplitAndTransfer.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 
-import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.util.TransferPair;
 
 import org.junit.After;
@@ -45,9 +45,9 @@ public class TestSplitAndTransfer {
         allocator.close();
     }
 
-    @Test /* NullableVarCharVector */
+    @Test /* VarCharVector */
     public void test() throws Exception {
-        try(final NullableVarCharVector varCharVector = new NullableVarCharVector("myvector", allocator)) {
+        try(final VarCharVector varCharVector = new VarCharVector("myvector", allocator)) {
             varCharVector.allocateNew(10000, 1000);
 
             final int valueCount = 500;
@@ -61,7 +61,7 @@ public class TestSplitAndTransfer {
             varCharVector.setValueCount(valueCount);
 
             final TransferPair tp = varCharVector.getTransferPair(allocator);
-            final NullableVarCharVector newVarCharVector = (NullableVarCharVector) tp.getTo();
+            final VarCharVector newVarCharVector = (VarCharVector) tp.getTo();
             final int[][] startLengths = {{0, 201}, {201, 200}, {401, 99}};
 
             for (final int[] startLength : startLengths) {
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestUtils.java b/java/vector/src/test/java/org/apache/arrow/vector/TestUtils.java
index a148813..99a1d89 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestUtils.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestUtils.java
@@ -25,13 +25,13 @@ import org.apache.arrow.vector.types.pojo.FieldType;
 
 public class TestUtils {
 
-  public static NullableVarCharVector newNullableVarCharVector(String name, BufferAllocator allocator) {
-    return (NullableVarCharVector)
+  public static VarCharVector newVarCharVector(String name, BufferAllocator allocator) {
+    return (VarCharVector)
         FieldType.nullable(new ArrowType.Utf8()).createNewSingleVector(name, allocator, null);
   }
 
-  public static NullableVarBinaryVector newNullableVarBinaryVector(String name, BufferAllocator allocator) {
-    return (NullableVarBinaryVector)
+  public static VarBinaryVector newVarBinaryVector(String name, BufferAllocator allocator) {
+    return (VarBinaryVector)
         FieldType.nullable(new ArrowType.Binary()).createNewSingleVector(name, allocator, null);
   }
 
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
index f51a874..336ae1c 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestValueVector.java
@@ -19,8 +19,8 @@
 package org.apache.arrow.vector;
 import org.apache.arrow.vector.util.OversizedAllocationException;
 
-import static org.apache.arrow.vector.TestUtils.newNullableVarBinaryVector;
-import static org.apache.arrow.vector.TestUtils.newNullableVarCharVector;
+import static org.apache.arrow.vector.TestUtils.newVarBinaryVector;
+import static org.apache.arrow.vector.TestUtils.newVarCharVector;
 import static org.apache.arrow.vector.TestUtils.newVector;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -88,9 +88,9 @@ public class TestValueVector {
    *  -- Float4Vector
    *  -- Float8Vector
    *
-   *  -- NullableUInt4Vector
-   *  -- NullableIntVector
-   *  -- NullableFloat4Vector
+   *  -- UInt4Vector
+   *  -- IntVector
+   *  -- Float4Vector
    *
    * TODO:
    *
@@ -107,28 +107,26 @@ public class TestValueVector {
 
       boolean error = false;
       int initialCapacity = 0;
-      final UInt4Vector.Mutator mutator = vector.getMutator();
-      final UInt4Vector.Accessor accessor = vector.getAccessor();
 
       vector.allocateNew(1024);
       initialCapacity = vector.getValueCapacity();
       assertEquals(1024, initialCapacity);
 
       // Put and set a few values
-      mutator.setSafe(0, 100);
-      mutator.setSafe(1, 101);
-      mutator.setSafe(100, 102);
-      mutator.setSafe(1022, 103);
-      mutator.setSafe(1023, 104);
-
-      assertEquals(100, accessor.get(0));
-      assertEquals(101, accessor.get(1));
-      assertEquals(102, accessor.get(100));
-      assertEquals(103, accessor.get(1022));
-      assertEquals(104, accessor.get(1023));
+      vector.setSafe(0, 100);
+      vector.setSafe(1, 101);
+      vector.setSafe(100, 102);
+      vector.setSafe(1022, 103);
+      vector.setSafe(1023, 104);
+
+      assertEquals(100, vector.get(0));
+      assertEquals(101, vector.get(1));
+      assertEquals(102, vector.get(100));
+      assertEquals(103, vector.get(1022));
+      assertEquals(104, vector.get(1023));
 
       try {
-        mutator.set(1024, 10000);
+        vector.set(1024, 10000);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -139,7 +137,7 @@ public class TestValueVector {
       }
 
       try {
-        accessor.get(1024);
+        vector.get(1024);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -150,18 +148,18 @@ public class TestValueVector {
       }
 
       /* this should trigger a realloc() */
-      mutator.setSafe(1024, 10000);
+      vector.setSafe(1024, 10000);
 
       /* underlying buffer should now be able to store double the number of values */
       assertEquals(initialCapacity * 2, vector.getValueCapacity());
 
       /* check vector data after realloc */
-      assertEquals(100, accessor.get(0));
-      assertEquals(101, accessor.get(1));
-      assertEquals(102, accessor.get(100));
-      assertEquals(103, accessor.get(1022));
-      assertEquals(104, accessor.get(1023));
-      assertEquals(10000, accessor.get(1024));
+      assertEquals(100, vector.get(0));
+      assertEquals(101, vector.get(1));
+      assertEquals(102, vector.get(100));
+      assertEquals(103, vector.get(1022));
+      assertEquals(104, vector.get(1023));
+      assertEquals(10000, vector.get(1024));
 
       /* reset the vector */
       vector.reset();
@@ -171,7 +169,8 @@ public class TestValueVector {
 
       /* vector data should have been zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertEquals("non-zero data not expected at index: " + i, 0, accessor.get(i));
+        // TODO: test vector.get(i) is 0 after unsafe get added
+        assertEquals("non-zero data not expected at index: " + i, true, vector.isNull(i));
       }
     }
   }
@@ -179,8 +178,6 @@ public class TestValueVector {
   @Test /* IntVector */
   public void testFixedType2() {
     try (final IntVector intVector = new IntVector(EMPTY_SCHEMA_PATH, allocator)) {
-      final IntVector.Mutator mutator = intVector.getMutator();
-      final IntVector.Accessor accessor = intVector.getAccessor();
       boolean error = false;
       int initialCapacity = 16;
 
@@ -211,12 +208,12 @@ public class TestValueVector {
       /* populate the vector */
       int j = 1;
       for(int i = 0; i < 16; i += 2) {
-        mutator.set(i, j);
+        intVector.set(i, j);
         j++;
       }
 
       try {
-        mutator.set(16, 9);
+        intVector.set(16, 9);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -229,12 +226,12 @@ public class TestValueVector {
       /* check vector contents */
       j = 1;
       for(int i = 0; i < 16; i += 2) {
-        assertEquals("unexpected value at index: " + i, j, accessor.get(i));
+        assertEquals("unexpected value at index: " + i, j, intVector.get(i));
         j++;
       }
 
       try {
-        accessor.get(16);
+        intVector.get(16);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -245,7 +242,7 @@ public class TestValueVector {
       }
 
       /* this should trigger a realloc() */
-      mutator.setSafe(16, 9);
+      intVector.setSafe(16, 9);
 
       /* underlying buffer should now be able to store double the number of values */
       assertEquals(initialCapacity * 2, intVector.getValueCapacity());
@@ -253,7 +250,7 @@ public class TestValueVector {
       /* vector data should still be intact after realloc */
       j = 1;
       for(int i = 0; i <= 16; i += 2) {
-        assertEquals("unexpected value at index: " + i, j, accessor.get(i));
+        assertEquals("unexpected value at index: " + i, j, intVector.get(i));
         j++;
       }
 
@@ -265,7 +262,7 @@ public class TestValueVector {
 
       /* vector data should have been zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertEquals("non-zero data not expected at index: " + i, 0, accessor.get(i));
+        assertEquals("non-zero data not expected at index: " + i, true, intVector.isNull(i));
       }
     }
   }
@@ -273,8 +270,6 @@ public class TestValueVector {
   @Test /* Float4Vector */
   public void testFixedType3() {
     try (final Float4Vector floatVector = new Float4Vector(EMPTY_SCHEMA_PATH, allocator)) {
-      final Float4Vector.Mutator mutator = floatVector.getMutator();
-      final Float4Vector.Accessor accessor = floatVector.getAccessor();
       boolean error = false;
       int initialCapacity = 16;
 
@@ -304,18 +299,18 @@ public class TestValueVector {
 
       floatVector.zeroVector();
 
-      /* populate the vector */
-      mutator.set(0, 1.5f);
-      mutator.set(2, 2.5f);
-      mutator.set(4, 3.3f);
-      mutator.set(6, 4.8f);
-      mutator.set(8, 5.6f);
-      mutator.set(10, 6.6f);
-      mutator.set(12, 7.8f);
-      mutator.set(14, 8.5f);
+      /* populate the floatVector */
+      floatVector.set(0, 1.5f);
+      floatVector.set(2, 2.5f);
+      floatVector.set(4, 3.3f);
+      floatVector.set(6, 4.8f);
+      floatVector.set(8, 5.6f);
+      floatVector.set(10, 6.6f);
+      floatVector.set(12, 7.8f);
+      floatVector.set(14, 8.5f);
 
       try {
-        mutator.set(16, 9.5f);
+        floatVector.set(16, 9.5f);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -326,17 +321,17 @@ public class TestValueVector {
       }
 
       /* check vector contents */
-      assertEquals(1.5f, accessor.get(0), 0);
-      assertEquals(2.5f, accessor.get(2), 0);
-      assertEquals(3.3f, accessor.get(4), 0);
-      assertEquals(4.8f, accessor.get(6), 0);
-      assertEquals(5.6f, accessor.get(8), 0);
-      assertEquals(6.6f, accessor.get(10), 0);
-      assertEquals(7.8f, accessor.get(12), 0);
-      assertEquals(8.5f, accessor.get(14), 0);
+      assertEquals(1.5f, floatVector.get(0), 0);
+      assertEquals(2.5f, floatVector.get(2), 0);
+      assertEquals(3.3f, floatVector.get(4), 0);
+      assertEquals(4.8f, floatVector.get(6), 0);
+      assertEquals(5.6f, floatVector.get(8), 0);
+      assertEquals(6.6f, floatVector.get(10), 0);
+      assertEquals(7.8f, floatVector.get(12), 0);
+      assertEquals(8.5f, floatVector.get(14), 0);
 
       try {
-        accessor.get(16);
+        floatVector.get(16);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -347,21 +342,21 @@ public class TestValueVector {
       }
 
       /* this should trigger a realloc() */
-      mutator.setSafe(16, 9.5f);
+      floatVector.setSafe(16, 9.5f);
 
       /* underlying buffer should now be able to store double the number of values */
       assertEquals(initialCapacity * 2, floatVector.getValueCapacity());
 
       /* vector data should still be intact after realloc */
-      assertEquals(1.5f, accessor.get(0), 0);
-      assertEquals(2.5f, accessor.get(2), 0);
-      assertEquals(3.3f, accessor.get(4), 0);
-      assertEquals(4.8f, accessor.get(6), 0);
-      assertEquals(5.6f, accessor.get(8), 0);
-      assertEquals(6.6f, accessor.get(10), 0);
-      assertEquals(7.8f, accessor.get(12), 0);
-      assertEquals(8.5f, accessor.get(14), 0);
-      assertEquals(9.5f, accessor.get(16), 0);
+      assertEquals(1.5f, floatVector.get(0), 0);
+      assertEquals(2.5f, floatVector.get(2), 0);
+      assertEquals(3.3f, floatVector.get(4), 0);
+      assertEquals(4.8f, floatVector.get(6), 0);
+      assertEquals(5.6f, floatVector.get(8), 0);
+      assertEquals(6.6f, floatVector.get(10), 0);
+      assertEquals(7.8f, floatVector.get(12), 0);
+      assertEquals(8.5f, floatVector.get(14), 0);
+      assertEquals(9.5f, floatVector.get(16), 0);
 
       /* reset the vector */
       floatVector.reset();
@@ -371,7 +366,7 @@ public class TestValueVector {
 
       /* vector data should be zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertEquals("non-zero data not expected at index: " + i, 0, accessor.get(i), 0);
+        assertEquals("non-zero data not expected at index: " + i, true, floatVector.isNull(i));
       }
     }
   }
@@ -379,8 +374,6 @@ public class TestValueVector {
   @Test /* Float8Vector */
   public void testFixedType4() {
     try (final Float8Vector floatVector = new Float8Vector(EMPTY_SCHEMA_PATH, allocator)) {
-      final Float8Vector.Mutator mutator = floatVector.getMutator();
-      final Float8Vector.Accessor accessor = floatVector.getAccessor();
       boolean error = false;
       int initialCapacity = 16;
 
@@ -409,17 +402,17 @@ public class TestValueVector {
       assertEquals(initialCapacity, floatVector.getValueCapacity());
 
       /* populate the vector */
-      mutator.set(0, 1.55);
-      mutator.set(2, 2.53);
-      mutator.set(4, 3.36);
-      mutator.set(6, 4.82);
-      mutator.set(8, 5.67);
-      mutator.set(10, 6.67);
-      mutator.set(12, 7.87);
-      mutator.set(14, 8.56);
+      floatVector.set(0, 1.55);
+      floatVector.set(2, 2.53);
+      floatVector.set(4, 3.36);
+      floatVector.set(6, 4.82);
+      floatVector.set(8, 5.67);
+      floatVector.set(10, 6.67);
+      floatVector.set(12, 7.87);
+      floatVector.set(14, 8.56);
 
       try {
-        mutator.set(16, 9.53);
+        floatVector.set(16, 9.53);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -429,18 +422,18 @@ public class TestValueVector {
         error = false;
       }
 
-      /* check vector contents */
-      assertEquals(1.55, accessor.get(0), 0);
-      assertEquals(2.53, accessor.get(2), 0);
-      assertEquals(3.36, accessor.get(4), 0);
-      assertEquals(4.82, accessor.get(6), 0);
-      assertEquals(5.67, accessor.get(8), 0);
-      assertEquals(6.67, accessor.get(10), 0);
-      assertEquals(7.87, accessor.get(12), 0);
-      assertEquals(8.56, accessor.get(14), 0);
+      /* check floatVector contents */
+      assertEquals(1.55, floatVector.get(0), 0);
+      assertEquals(2.53, floatVector.get(2), 0);
+      assertEquals(3.36, floatVector.get(4), 0);
+      assertEquals(4.82, floatVector.get(6), 0);
+      assertEquals(5.67, floatVector.get(8), 0);
+      assertEquals(6.67, floatVector.get(10), 0);
+      assertEquals(7.87, floatVector.get(12), 0);
+      assertEquals(8.56, floatVector.get(14), 0);
 
       try {
-        accessor.get(16);
+        floatVector.get(16);
       }
       catch (IndexOutOfBoundsException ie) {
         error = true;
@@ -451,21 +444,21 @@ public class TestValueVector {
       }
 
       /* this should trigger a realloc() */
-      mutator.setSafe(16, 9.53);
+      floatVector.setSafe(16, 9.53);
 
       /* underlying buffer should now be able to store double the number of values */
       assertEquals(initialCapacity * 2, floatVector.getValueCapacity());
 
       /* vector data should still be intact after realloc */
-      assertEquals(1.55, accessor.get(0), 0);
-      assertEquals(2.53, accessor.get(2), 0);
-      assertEquals(3.36, accessor.get(4), 0);
-      assertEquals(4.82, accessor.get(6), 0);
-      assertEquals(5.67, accessor.get(8), 0);
-      assertEquals(6.67, accessor.get(10), 0);
-      assertEquals(7.87, accessor.get(12), 0);
-      assertEquals(8.56, accessor.get(14), 0);
-      assertEquals(9.53, accessor.get(16), 0);
+      assertEquals(1.55, floatVector.get(0), 0);
+      assertEquals(2.53, floatVector.get(2), 0);
+      assertEquals(3.36, floatVector.get(4), 0);
+      assertEquals(4.82, floatVector.get(6), 0);
+      assertEquals(5.67, floatVector.get(8), 0);
+      assertEquals(6.67, floatVector.get(10), 0);
+      assertEquals(7.87, floatVector.get(12), 0);
+      assertEquals(8.56, floatVector.get(14), 0);
+      assertEquals(9.53, floatVector.get(16), 0);
 
       /* reset the vector */
       floatVector.reset();
@@ -475,16 +468,16 @@ public class TestValueVector {
 
       /* vector data should be zeroed out */
       for(int i = 0; i < (initialCapacity * 2); i++) {
-        assertEquals("non-zero data not expected at index: " + i, 0, accessor.get(i), 0);
+        assertEquals("non-zero data not expected at index: " + i, true, floatVector.isNull(i));
       }
     }
   }
 
-  @Test /* NullableUInt4Vector */
+  @Test /* UInt4Vector */
   public void testNullableFixedType1() {
 
     // Create a new value vector for 1024 integers.
-    try (final NullableUInt4Vector vector = newVector(NullableUInt4Vector.class, EMPTY_SCHEMA_PATH, new ArrowType.Int(32, false), allocator);) {
+    try (final UInt4Vector vector = newVector(UInt4Vector.class, EMPTY_SCHEMA_PATH, new ArrowType.Int(32, false), allocator);) {
       boolean error = false;
       int initialCapacity = 1024;
 
@@ -582,10 +575,10 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableFloat4Vector */
+  @Test /* Float4Vector */
   public void testNullableFixedType2() {
     // Create a new value vector for 1024 integers
-    try (final NullableFloat4Vector vector = newVector(NullableFloat4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator);) {
+    try (final Float4Vector vector = newVector(Float4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator);) {
       boolean error = false;
       int initialCapacity = 16;
 
@@ -683,10 +676,10 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableIntVector */
+  @Test /* IntVector */
   public void testNullableFixedType3() {
     // Create a new value vector for 1024 integers
-    try (final NullableIntVector vector = newVector(NullableIntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, allocator)) {
+    try (final IntVector vector = newVector(IntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, allocator)) {
       boolean error = false;
       int initialCapacity = 1024;
 
@@ -777,9 +770,9 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableIntVector */
+  @Test /* IntVector */
   public void testNullableFixedType4() {
-    try (final NullableIntVector vector = newVector(NullableIntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, allocator)) {
+    try (final IntVector vector = newVector(IntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, allocator)) {
 
       /* no memory allocation has happened yet */
       assertEquals(0, vector.getValueCapacity());
@@ -870,8 +863,8 @@ public class TestValueVector {
    *
    * Covered types as of now
    *
-   *  -- NullableVarCharVector
-   *  -- NullableVarBinaryVector
+   *  -- VarCharVector
+   *  -- VarBinaryVector
    *
    * TODO:
    *
@@ -879,11 +872,11 @@ public class TestValueVector {
    *  -- VarBinaryVector
    */
 
-  @Test /* NullableVarCharVector */
+  @Test /* VarCharVector */
   public void testNullableVarType1() {
 
     // Create a new value vector for 1024 integers.
-    try (final NullableVarCharVector vector = newNullableVarCharVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final VarCharVector vector = newVarCharVector(EMPTY_SCHEMA_PATH, allocator)) {
       vector.allocateNew(1024 * 10, 1024);
 
       vector.set(0, STR1);
@@ -916,11 +909,11 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableVarBinaryVector */
+  @Test /* VarBinaryVector */
   public void testNullableVarType2() {
 
     // Create a new value vector for 1024 integers.
-    try (final NullableVarBinaryVector vector = newNullableVarBinaryVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final VarBinaryVector vector = newVarBinaryVector(EMPTY_SCHEMA_PATH, allocator)) {
       vector.allocateNew(1024 * 10, 1024);
 
       vector.set(0, STR1);
@@ -971,8 +964,6 @@ public class TestValueVector {
   @Test /* Float8Vector */
   public void testReallocAfterVectorTransfer1() {
     try (final Float8Vector vector = new Float8Vector(EMPTY_SCHEMA_PATH, allocator)) {
-      final Float8Vector.Mutator mutator = vector.getMutator();
-      final Float8Vector.Accessor accessor = vector.getAccessor();
       final int initialDefaultCapacity = 4096;
       boolean error = false;
 
@@ -984,7 +975,7 @@ public class TestValueVector {
       double baseValue = 100.375;
 
       for (int i = 0; i < initialDefaultCapacity; i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       /* the above setSafe calls should not have triggered a realloc as
@@ -993,33 +984,33 @@ public class TestValueVector {
       assertEquals(initialDefaultCapacity, vector.getValueCapacity());
 
       for (int i = 0; i < initialDefaultCapacity; i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
       /* this should trigger a realloc */
-      mutator.setSafe(initialDefaultCapacity, baseValue + (double)initialDefaultCapacity);
+      vector.setSafe(initialDefaultCapacity, baseValue + (double)initialDefaultCapacity);
       assertEquals(initialDefaultCapacity * 2, vector.getValueCapacity());
 
       for (int i = initialDefaultCapacity + 1; i < (initialDefaultCapacity * 2); i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       for (int i = 0; i < (initialDefaultCapacity * 2); i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
       /* this should trigger a realloc */
-      mutator.setSafe(initialDefaultCapacity * 2, baseValue + (double)(initialDefaultCapacity * 2));
+      vector.setSafe(initialDefaultCapacity * 2, baseValue + (double)(initialDefaultCapacity * 2));
       assertEquals(initialDefaultCapacity * 4, vector.getValueCapacity());
 
       for (int i = (initialDefaultCapacity * 2) + 1; i < (initialDefaultCapacity * 4); i++) {
-        mutator.setSafe(i, baseValue + (double)i);
+        vector.setSafe(i, baseValue + (double)i);
       }
 
       for (int i = 0; i < (initialDefaultCapacity * 4); i++) {
-        double value = accessor.get(i);
+        double value = vector.get(i);
         assertEquals(baseValue + (double)i, value, 0);
       }
 
@@ -1036,15 +1027,12 @@ public class TestValueVector {
       toVector.reAlloc();
       assertEquals(initialDefaultCapacity * 8, toVector.getValueCapacity());
 
-      final Float8Vector.Accessor toAccessor = toVector.getAccessor();
-
       for (int i = 0; i < (initialDefaultCapacity * 8); i++) {
-        double value = toAccessor.get(i);
         if (i < (initialDefaultCapacity * 4)) {
-          assertEquals(baseValue + (double)i, value, 0);
+          assertEquals(baseValue + (double)i, toVector.get(i), 0);
         }
         else {
-          assertEquals(0, value, 0);
+          assertTrue(toVector.isNull(i));
         }
       }
 
@@ -1052,9 +1040,9 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableFloat8Vector */
+  @Test /* Float8Vector */
   public void testReallocAfterVectorTransfer2() {
-    try (final NullableFloat8Vector vector = new NullableFloat8Vector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final Float8Vector vector = new Float8Vector(EMPTY_SCHEMA_PATH, allocator)) {
       final int initialDefaultCapacity = 4096;
       boolean error = false;
 
@@ -1111,7 +1099,7 @@ public class TestValueVector {
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
 
-      NullableFloat8Vector toVector = (NullableFloat8Vector)transferPair.getTo();
+      Float8Vector toVector = (Float8Vector)transferPair.getTo();
 
       /* check toVector contents before realloc */
       for (int i = 0; i < (initialDefaultCapacity * 4); i++) {
@@ -1139,9 +1127,9 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableVarCharVector */
+  @Test /* VarCharVector */
   public void testReallocAfterVectorTransfer3() {
-    try (final NullableVarCharVector vector = new NullableVarCharVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final VarCharVector vector = new VarCharVector(EMPTY_SCHEMA_PATH, allocator)) {
       /* 4096 values with 10 byte per record */
       vector.allocateNew(4096 * 10, 4096);
       int valueCapacity = vector.getValueCapacity();
@@ -1223,7 +1211,7 @@ public class TestValueVector {
 
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
-      NullableVarCharVector toVector = (NullableVarCharVector)transferPair.getTo();
+      VarCharVector toVector = (VarCharVector)transferPair.getTo();
       valueCapacity = toVector.getValueCapacity();
 
       for (int i = 0; i < valueCapacity; i++) {
@@ -1239,9 +1227,9 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableIntVector */
+  @Test /* IntVector */
   public void testReallocAfterVectorTransfer4() {
-    try (final NullableIntVector vector = new NullableIntVector(EMPTY_SCHEMA_PATH, allocator)) {
+    try (final IntVector vector = new IntVector(EMPTY_SCHEMA_PATH, allocator)) {
 
       /* 4096 values  */
       vector.allocateNew(4096);
@@ -1316,7 +1304,7 @@ public class TestValueVector {
 
       TransferPair transferPair = vector.getTransferPair(allocator);
       transferPair.transfer();
-      NullableIntVector toVector = (NullableIntVector)transferPair.getTo();
+      IntVector toVector = (IntVector)transferPair.getTo();
       /* value capacity of source and target vectors should be same after
        * the transfer.
        */
@@ -1336,9 +1324,9 @@ public class TestValueVector {
   }
 
   @Test
-  public void testReAllocNullableFixedWidthVector() {
+  public void testReAllocFixedWidthVector() {
     // Create a new value vector for 1024 integers
-    try (final NullableFloat4Vector vector = newVector(NullableFloat4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator)) {
+    try (final Float4Vector vector = newVector(Float4Vector.class, EMPTY_SCHEMA_PATH, MinorType.FLOAT4, allocator)) {
       vector.allocateNew(1024);
 
       assertEquals(1024, vector.getValueCapacity());
@@ -1359,7 +1347,7 @@ public class TestValueVector {
       assertEquals(104.5f, vector.get(1023), 0);
       assertEquals(105.5f, vector.get(2000), 0);
 
-      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors
+      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for ValueVectors
       // as we don't call setSafe for null values, but we do call setValueCount when all values are inserted into the
       // vector
       vector.setValueCount(vector.getValueCapacity() + 200);
@@ -1367,8 +1355,8 @@ public class TestValueVector {
   }
 
   @Test
-  public void testReAllocNullableVariableWidthVector() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
+  public void testReAllocVariableWidthVector() {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
       vector.allocateNew();
 
       int initialCapacity = vector.getValueCapacity();
@@ -1392,7 +1380,7 @@ public class TestValueVector {
       assertArrayEquals(STR2, vector.get(initialCapacity - 1));
       assertArrayEquals(STR3, vector.get(initialCapacity + 200));
 
-      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for NullableValueVectors
+      // Set the valueCount to be more than valueCapacity of current allocation. This is possible for ValueVectors
       // as we don't call setSafe for null values, but we do call setValueCount when the current batch is processed.
       vector.setValueCount(vector.getValueCapacity() + 200);
     }
@@ -1400,7 +1388,7 @@ public class TestValueVector {
 
   @Test
   public void testFillEmptiesNotOverfill() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
       vector.allocateNew();
 
       int initialCapacity = vector.getValueCapacity();
@@ -1420,8 +1408,8 @@ public class TestValueVector {
 
   @Test
   public void testCopyFromWithNulls() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator);
-         final NullableVarCharVector vector2 = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator);
+         final VarCharVector vector2 = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
 
       vector.allocateNew();
       int capacity = vector.getValueCapacity();
@@ -1480,8 +1468,8 @@ public class TestValueVector {
 
   @Test
   public void testCopyFromWithNulls1() {
-    try (final NullableVarCharVector vector = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator);
-         final NullableVarCharVector vector2 = newVector(NullableVarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
+    try (final VarCharVector vector = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator);
+         final VarCharVector vector2 = newVector(VarCharVector.class, EMPTY_SCHEMA_PATH, MinorType.VARCHAR, allocator)) {
 
       vector.allocateNew();
       int capacity = vector.getValueCapacity();
@@ -1544,7 +1532,7 @@ public class TestValueVector {
 
   @Test
   public void testSetLastSetUsage() {
-    try (final NullableVarCharVector vector = new NullableVarCharVector("myvector", allocator)) {
+    try (final VarCharVector vector = new VarCharVector("myvector", allocator)) {
       vector.allocateNew(1024 * 10, 1024);
 
       setBytes(0, STR1, vector);
@@ -1657,7 +1645,7 @@ public class TestValueVector {
   @Test
   public void testVectorLoadUnload() {
 
-    try (final NullableVarCharVector vector1 = new NullableVarCharVector("myvector", allocator)) {
+    try (final VarCharVector vector1 = new VarCharVector("myvector", allocator)) {
       vector1.allocateNew(1024 * 10, 1024);
 
       vector1.set(0, STR1);
@@ -1701,7 +1689,7 @@ public class TestValueVector {
         VectorLoader vectorLoader = new VectorLoader(schemaRoot2);
         vectorLoader.load(recordBatch);
 
-        NullableVarCharVector vector2 = (NullableVarCharVector) schemaRoot2.getVector(fieldName);
+        VarCharVector vector2 = (VarCharVector) schemaRoot2.getVector(fieldName);
         /*
          * lastSet would have internally been set by VectorLoader.load() when it invokes
          * loadFieldBuffers.
@@ -1723,7 +1711,7 @@ public class TestValueVector {
 
   @Test
   public void testFillEmptiesUsage() {
-    try (final NullableVarCharVector vector = new NullableVarCharVector("myvector", allocator)) {
+    try (final VarCharVector vector = new VarCharVector("myvector", allocator)) {
 
       vector.allocateNew(1024 * 10, 1024);
 
@@ -1830,10 +1818,10 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableVarCharVector */
+  @Test /* VarCharVector */
   public void testGetBufferAddress1() {
 
-    try (final NullableVarCharVector vector = new NullableVarCharVector("myvector", allocator)) {
+    try (final VarCharVector vector = new VarCharVector("myvector", allocator)) {
       vector.allocateNew(1024 * 10, 1024);
 
       /* populate the vector */
@@ -1866,9 +1854,9 @@ public class TestValueVector {
     }
   }
 
-  @Test /* NullableIntVector */
+  @Test /* IntVector */
   public void testGetBufferAddress2() {
-    try (final NullableIntVector vector = new NullableIntVector("myvector", allocator)) {
+    try (final IntVector vector = new IntVector("myvector", allocator)) {
       boolean error = false;
       vector.allocateNew(16);
 
@@ -1905,7 +1893,7 @@ public class TestValueVector {
   @Test
   public void testMultipleClose() {
     BufferAllocator vectorAllocator = allocator.newChildAllocator("vector_allocator", 0, Long.MAX_VALUE);
-    NullableIntVector vector = newVector(NullableIntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, vectorAllocator);
+    IntVector vector = newVector(IntVector.class, EMPTY_SCHEMA_PATH, MinorType.INT, vectorAllocator);
     vector.close();
     vectorAllocator.close();
     vector.close();
@@ -1916,7 +1904,7 @@ public class TestValueVector {
    * lastSet. The method is to test the lastSet property and that's why we load the vector
    * in a way that lastSet is not set automatically.
    */
-  public static void setBytes(int index, byte[] bytes, NullableVarCharVector vector) {
+  public static void setBytes(int index, byte[] bytes, VarCharVector vector) {
     final int currentOffset = vector.offsetBuffer.getInt(index * vector.OFFSET_WIDTH);
 
     BitVectorHelper.setValidityBitToOne(vector.validityBuffer, index);
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
index 293ffbf..c0df488 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReAlloc.java
@@ -26,7 +26,7 @@ import java.nio.charset.StandardCharsets;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.vector.complex.ListVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.FieldType;
@@ -53,14 +53,13 @@ public class TestVectorReAlloc {
   @Test
   public void testFixedType() {
     try (final UInt4Vector vector = new UInt4Vector("", allocator)) {
-      final UInt4Vector.Mutator m = vector.getMutator();
       vector.setInitialCapacity(512);
       vector.allocateNew();
 
       assertEquals(512, vector.getValueCapacity());
 
       try {
-        m.set(512, 0);
+        vector.set(512, 0);
         Assert.fail("Expected out of bounds exception");
       } catch (Exception e) {
         // ok
@@ -69,14 +68,14 @@ public class TestVectorReAlloc {
       vector.reAlloc();
       assertEquals(1024, vector.getValueCapacity());
 
-      m.set(512, 100);
-      assertEquals(100, vector.getAccessor().get(512));
+      vector.set(512, 100);
+      assertEquals(100, vector.get(512));
     }
   }
 
   @Test
   public void testNullableType() {
-    try (final NullableVarCharVector vector = new NullableVarCharVector("", allocator)) {
+    try (final VarCharVector vector = new VarCharVector("", allocator)) {
       vector.setInitialCapacity(512);
       vector.allocateNew();
 
@@ -108,7 +107,7 @@ public class TestVectorReAlloc {
       assertEquals(1023, vector.getValueCapacity());
 
       try {
-        vector.getOffsetVector().getAccessor().get(2014);
+        vector.getInnerValueCountAt(2014);
         Assert.fail("Expected out of bounds exception");
       } catch (Exception e) {
         // ok
@@ -122,8 +121,8 @@ public class TestVectorReAlloc {
 
   @Test
   public void testMapType() {
-    try (final NullableMapVector vector = NullableMapVector.empty("", allocator)) {
-      vector.addOrGet("", FieldType.nullable(MinorType.INT.getType()), NullableIntVector.class);
+    try (final MapVector vector = MapVector.empty("", allocator)) {
+      vector.addOrGet("", FieldType.nullable(MinorType.INT.getType()), IntVector.class);
 
       vector.setInitialCapacity(512);
       vector.allocateNew();
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReset.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReset.java
index d53f694..28903b1 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReset.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorReset.java
@@ -43,12 +43,11 @@ public class TestVectorReset {
   @Test
   public void testFixedTypeReset() {
     try (final UInt4Vector vector = new UInt4Vector("", allocator)) {
-      final UInt4Vector.Mutator m = vector.getMutator();
       vector.allocateNew();
-      final int sizeBefore = vector.getAllocationSize();
+      final int sizeBefore = vector.getBufferSize();
       vector.reAlloc();
       vector.reset();
-      final int sizeAfter = vector.getAllocationSize();
+      final int sizeAfter = vector.getBufferSize();
       assertEquals(sizeBefore, sizeAfter);
     }
   }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
index e61dbec..439a627 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestVectorUnloadLoad.java
@@ -236,8 +236,8 @@ public class TestVectorUnloadLoad {
 
       vectorLoader.load(recordBatch);
 
-      NullableIntVector intDefinedVector = (NullableIntVector) newRoot.getVector("intDefined");
-      NullableIntVector intNullVector = (NullableIntVector) newRoot.getVector("intNull");
+      IntVector intDefinedVector = (IntVector) newRoot.getVector("intDefined");
+      IntVector intNullVector = (IntVector) newRoot.getVector("intNull");
       for (int i = 0; i < count; i++) {
         assertFalse("#" + i, intDefinedVector.isNull(i));
         assertEquals("#" + i, i, intDefinedVector.get(i));
@@ -283,7 +283,7 @@ public class TestVectorUnloadLoad {
         FieldVector vector = field.createVector(originalVectorsAllocator);
         vector.allocateNew();
         sources.add(vector);
-        NullableIntVector intVector = (NullableIntVector)vector;
+        IntVector intVector = (IntVector)vector;
         for (int i = 0; i < count; i++) {
           intVector.set(i, i);
         }
@@ -302,8 +302,8 @@ public class TestVectorUnloadLoad {
           List<FieldVector> targets = newRoot.getFieldVectors();
           Assert.assertEquals(sources.size(), targets.size());
           for (int k = 0; k < sources.size(); k++) {
-            NullableIntVector src = (NullableIntVector) sources.get(k);
-            NullableIntVector tgt = (NullableIntVector) targets.get(k);
+            IntVector src = (IntVector) sources.get(k);
+            IntVector tgt = (IntVector) targets.get(k);
             Assert.assertEquals(src.getValueCount(), tgt.getValueCount());
             for (int i = 0; i < count; i++) {
               Assert.assertEquals(src.get(i), tgt.get(i));
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java b/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
index b0d6cf5..38b7842 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/complex/impl/TestPromotableWriter.java
@@ -25,7 +25,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.DirtyRootAllocator;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.UnionVector;
 import org.apache.arrow.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.arrow.vector.types.pojo.ArrowType;
@@ -55,7 +55,7 @@ public class TestPromotableWriter {
   public void testPromoteToUnion() throws Exception {
 
     try (final MapVector container = MapVector.empty(EMPTY_SCHEMA_PATH, allocator);
-         final NullableMapVector v = container.addOrGetMap("test");
+         final MapVector v = container.addOrGetMap("test");
          final PromotableWriter writer = new PromotableWriter(v, container)) {
 
       container.allocateNew();
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java b/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
index 52defdc..bd8489e 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/complex/writer/TestComplexWriter.java
@@ -28,13 +28,13 @@ import io.netty.buffer.ArrowBuf;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.vector.SchemaChangeCallBack;
-import org.apache.arrow.vector.NullableFloat8Vector;
-import org.apache.arrow.vector.NullableFloat4Vector;
-import org.apache.arrow.vector.NullableBigIntVector;
-import org.apache.arrow.vector.NullableIntVector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.IntVector;
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.NonNullableMapVector;
 import org.apache.arrow.vector.complex.UnionVector;
 import org.apache.arrow.vector.complex.impl.ComplexWriterImpl;
 import org.apache.arrow.vector.complex.impl.SingleMapReaderImpl;
@@ -830,7 +830,7 @@ public class TestComplexWriter {
     rootWriter.end();
     writer.setValueCount(1);
 
-    NullableMapVector mapVector = (NullableMapVector) parent.getChild("root");
+    MapVector mapVector = (MapVector) parent.getChild("root");
     TransferPair tp = mapVector.getTransferPair(allocator);
     tp.splitAndTransfer(0, 1);
     MapVector toMapVector = (MapVector) tp.getTo();
@@ -849,7 +849,7 @@ public class TestComplexWriter {
     /* initialize a SingleMapWriter with empty MapVector and then lazily
      * create all vectors with expected initialCapacity.
      */
-    MapVector parent = MapVector.empty("parent", allocator);
+    NonNullableMapVector parent = NonNullableMapVector.empty("parent", allocator);
     SingleMapWriter singleMapWriter = new SingleMapWriter(parent);
 
     int initialCapacity = 1024;
@@ -885,10 +885,10 @@ public class TestComplexWriter {
       singleMapWriter.end();
     }
 
-    NullableIntVector intVector = (NullableIntVector)parent.getChild("intField");
-    NullableBigIntVector bigIntVector = (NullableBigIntVector)parent.getChild("bigIntField");
-    NullableFloat4Vector float4Vector = (NullableFloat4Vector)parent.getChild("float4Field");
-    NullableFloat8Vector float8Vector = (NullableFloat8Vector)parent.getChild("float8Field");
+    IntVector intVector = (IntVector)parent.getChild("intField");
+    BigIntVector bigIntVector = (BigIntVector)parent.getChild("bigIntField");
+    Float4Vector float4Vector = (Float4Vector)parent.getChild("float4Field");
+    Float8Vector float8Vector = (Float8Vector)parent.getChild("float8Field");
 
     assertEquals(initialCapacity, singleMapWriter.getValueCapacity());
     assertEquals(initialCapacity, intVector.getValueCapacity());
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
index 233b682..3514aca 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/BaseFileTest.java
@@ -28,17 +28,17 @@ import com.google.common.collect.ImmutableList;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.NullableDateMilliVector;
-import org.apache.arrow.vector.NullableDecimalVector;
-import org.apache.arrow.vector.NullableIntVector;
-import org.apache.arrow.vector.NullableTimeMilliVector;
-import org.apache.arrow.vector.NullableVarBinaryVector;
-import org.apache.arrow.vector.NullableVarCharVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.ValueVector.Accessor;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.complex.ListVector;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.impl.ComplexWriterImpl;
 import org.apache.arrow.vector.complex.impl.UnionListWriter;
 import org.apache.arrow.vector.complex.reader.FieldReader;
@@ -71,7 +71,7 @@ import org.slf4j.LoggerFactory;
 
 import io.netty.buffer.ArrowBuf;
 
-import static org.apache.arrow.vector.TestUtils.newNullableVarCharVector;
+import static org.apache.arrow.vector.TestUtils.newVarCharVector;
 
 /**
  * Helps testing the file formats
@@ -172,7 +172,7 @@ public class BaseFileTest {
     return new LocalDateTime(2000 + i, 1 + i, 1 + i, i, i, i, i);
   }
 
-  protected void writeDateTimeData(int count, NullableMapVector parent) {
+  protected void writeDateTimeData(int count, MapVector parent) {
     Assert.assertTrue(count < 100);
     ComplexWriter writer = new ComplexWriterImpl("root", parent);
     MapWriter rootWriter = writer.rootAsMap();
@@ -202,11 +202,11 @@ public class BaseFileTest {
     Assert.assertEquals(count, root.getRowCount());
     printVectors(root.getFieldVectors());
     for (int i = 0; i < count; i++) {
-      long dateVal = ((NullableDateMilliVector) root.getVector("date")).get(i);
+      long dateVal = ((DateMilliVector) root.getVector("date")).get(i);
       LocalDateTime dt = makeDateTimeFromCount(i);
       LocalDateTime dateExpected = dt.minusMillis(dt.getMillisOfDay());
       Assert.assertEquals(DateUtility.toMillis(dateExpected), dateVal);
-      long timeVal = ((NullableTimeMilliVector) root.getVector("time")).get(i);
+      long timeVal = ((TimeMilliVector) root.getVector("time")).get(i);
       Assert.assertEquals(dt.getMillisOfDay(), timeVal);
       Object timestampMilliVal = root.getVector("timestamp-milli").getObject(i);
       Assert.assertEquals(dt, timestampMilliVal);
@@ -218,7 +218,7 @@ public class BaseFileTest {
   protected VectorSchemaRoot writeFlatDictionaryData(BufferAllocator bufferAllocator, DictionaryProvider.MapDictionaryProvider provider) {
 
     // Define dictionaries and add to provider
-    NullableVarCharVector dictionary1Vector = newNullableVarCharVector("D1", bufferAllocator);
+    VarCharVector dictionary1Vector = newVarCharVector("D1", bufferAllocator);
     dictionary1Vector.allocateNewSafe();
     dictionary1Vector.set(0, "foo".getBytes(StandardCharsets.UTF_8));
     dictionary1Vector.set(1, "bar".getBytes(StandardCharsets.UTF_8));
@@ -228,7 +228,7 @@ public class BaseFileTest {
     Dictionary dictionary1 = new Dictionary(dictionary1Vector, new DictionaryEncoding(1L, false, null));
     provider.put(dictionary1);
 
-    NullableVarCharVector dictionary2Vector = newNullableVarCharVector("D2", bufferAllocator);
+    VarCharVector dictionary2Vector = newVarCharVector("D2", bufferAllocator);
     dictionary2Vector.allocateNewSafe();
     dictionary2Vector.set(0, "micro".getBytes(StandardCharsets.UTF_8));
     dictionary2Vector.set(1, "small".getBytes(StandardCharsets.UTF_8));
@@ -239,7 +239,7 @@ public class BaseFileTest {
     provider.put(dictionary2);
 
     // Populate the vectors
-    NullableVarCharVector vector1A = newNullableVarCharVector("varcharA", bufferAllocator);
+    VarCharVector vector1A = newVarCharVector("varcharA", bufferAllocator);
     vector1A.allocateNewSafe();
     vector1A.set(0, "foo".getBytes(StandardCharsets.UTF_8));
     vector1A.set(1, "bar".getBytes(StandardCharsets.UTF_8));
@@ -252,7 +252,7 @@ public class BaseFileTest {
     vector1A.close();  // Done with this vector after encoding
 
     // Write this vector using indices instead of encoding
-    NullableIntVector encodedVector1B = new NullableIntVector("varcharB", bufferAllocator);
+    IntVector encodedVector1B = new IntVector("varcharB", bufferAllocator);
     encodedVector1B.allocateNewSafe();
     encodedVector1B.set(0, 2);  // "baz"
     encodedVector1B.set(1, 1);  // "bar"
@@ -261,7 +261,7 @@ public class BaseFileTest {
     encodedVector1B.set(5, 0);  // "foo"
     encodedVector1B.setValueCount(6);
 
-    NullableVarCharVector vector2 = newNullableVarCharVector("sizes", bufferAllocator);
+    VarCharVector vector2 = newVarCharVector("sizes", bufferAllocator);
     vector2.allocateNewSafe();
     vector2.set(1, "large".getBytes(StandardCharsets.UTF_8));
     vector2.set(2, "small".getBytes(StandardCharsets.UTF_8));
@@ -327,7 +327,7 @@ public class BaseFileTest {
 
     Dictionary dictionary1 = provider.lookup(1L);
     Assert.assertNotNull(dictionary1);
-    NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary1.getVector());
+    VarCharVector dictionaryVector = ((VarCharVector) dictionary1.getVector());
     Assert.assertEquals(3, dictionaryVector.getValueCount());
     Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
     Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
@@ -335,7 +335,7 @@ public class BaseFileTest {
 
     Dictionary dictionary2 = provider.lookup(2L);
     Assert.assertNotNull(dictionary2);
-    dictionaryVector = ((NullableVarCharVector) dictionary2.getVector());
+    dictionaryVector = ((VarCharVector) dictionary2.getVector());
     Assert.assertEquals(3, dictionaryVector.getValueCount());
     Assert.assertEquals(new Text("micro"), dictionaryVector.getObject(0));
     Assert.assertEquals(new Text("small"), dictionaryVector.getObject(1));
@@ -345,7 +345,7 @@ public class BaseFileTest {
   protected VectorSchemaRoot writeNestedDictionaryData(BufferAllocator bufferAllocator, DictionaryProvider.MapDictionaryProvider provider) {
 
     // Define the dictionary and add to the provider
-    NullableVarCharVector dictionaryVector = newNullableVarCharVector("D2", bufferAllocator);
+    VarCharVector dictionaryVector = newVarCharVector("D2", bufferAllocator);
     dictionaryVector.allocateNewSafe();
     dictionaryVector.set(0, "foo".getBytes(StandardCharsets.UTF_8));
     dictionaryVector.set(1, "bar".getBytes(StandardCharsets.UTF_8));
@@ -395,16 +395,16 @@ public class BaseFileTest {
 
     Dictionary dictionary = provider.lookup(2L);
     Assert.assertNotNull(dictionary);
-    NullableVarCharVector dictionaryVector = ((NullableVarCharVector) dictionary.getVector());
+    VarCharVector dictionaryVector = ((VarCharVector) dictionary.getVector());
     Assert.assertEquals(2, dictionaryVector.getValueCount());
     Assert.assertEquals(new Text("foo"), dictionaryVector.getObject(0));
     Assert.assertEquals(new Text("bar"), dictionaryVector.getObject(1));
   }
 
   protected VectorSchemaRoot writeDecimalData(BufferAllocator bufferAllocator) {
-    NullableDecimalVector decimalVector1 = new NullableDecimalVector("decimal1", bufferAllocator, 10, 3);
-    NullableDecimalVector decimalVector2 = new NullableDecimalVector("decimal2", bufferAllocator, 4, 2);
-    NullableDecimalVector decimalVector3 = new NullableDecimalVector("decimal3", bufferAllocator, 16, 8);
+    DecimalVector decimalVector1 = new DecimalVector("decimal1", bufferAllocator, 10, 3);
+    DecimalVector decimalVector2 = new DecimalVector("decimal2", bufferAllocator, 4, 2);
+    DecimalVector decimalVector3 = new DecimalVector("decimal3", bufferAllocator, 16, 8);
 
     int count = 10;
     decimalVector1.allocateNew(count);
@@ -427,9 +427,9 @@ public class BaseFileTest {
   }
 
   protected void validateDecimalData(VectorSchemaRoot root) {
-    NullableDecimalVector decimalVector1 = (NullableDecimalVector) root.getVector("decimal1");
-    NullableDecimalVector decimalVector2 = (NullableDecimalVector) root.getVector("decimal2");
-    NullableDecimalVector decimalVector3 = (NullableDecimalVector) root.getVector("decimal3");
+    DecimalVector decimalVector1 = (DecimalVector) root.getVector("decimal1");
+    DecimalVector decimalVector2 = (DecimalVector) root.getVector("decimal2");
+    DecimalVector decimalVector3 = (DecimalVector) root.getVector("decimal3");
     int count = 10;
     Assert.assertEquals(count, root.getRowCount());
 
@@ -491,7 +491,7 @@ public class BaseFileTest {
     }
   }
 
-  public void writeUnionData(int count, NullableMapVector parent) {
+  public void writeUnionData(int count, MapVector parent) {
     ArrowBuf varchar = allocator.buffer(3);
     varchar.readerIndex(0);
     varchar.setByte(0, 'a');
@@ -534,7 +534,7 @@ public class BaseFileTest {
     varchar.release();
   }
 
-  protected void writeVarBinaryData(int count, NullableMapVector parent) {
+  protected void writeVarBinaryData(int count, MapVector parent) {
     Assert.assertTrue(count < 100);
     ComplexWriter writer = new ComplexWriterImpl("root", parent);
     MapWriter rootWriter = writer.rootAsMap();
@@ -577,8 +577,8 @@ public class BaseFileTest {
     // ListVector lastSet should be the index of last value + 1
     Assert.assertEquals(listVector.getLastSet(), count);
 
-    // NullableVarBinaryVector lastSet should be the index of last value
-    NullableVarBinaryVector binaryVector = (NullableVarBinaryVector) listVector.getChildrenFromFields().get(0);
+    // VarBinaryVector lastSet should be the index of last value
+    VarBinaryVector binaryVector = (VarBinaryVector) listVector.getChildrenFromFields().get(0);
     Assert.assertEquals(binaryVector.getLastSet(), numVarBinaryValues - 1);
   }
 }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowFile.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowFile.java
index 4387db0..0cfc9ba 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowFile.java
@@ -36,14 +36,14 @@ import com.google.common.collect.Lists;
 
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.FieldVector;
-import org.apache.arrow.vector.NullableFloat4Vector;
-import org.apache.arrow.vector.NullableIntVector;
-import org.apache.arrow.vector.NullableTinyIntVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TinyIntVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.VectorUnloader;
 import org.apache.arrow.vector.complex.FixedSizeListVector;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.dictionary.DictionaryProvider.MapDictionaryProvider;
 import org.apache.arrow.vector.ipc.message.ArrowBlock;
 import org.apache.arrow.vector.ipc.message.ArrowBuffer;
@@ -82,7 +82,7 @@ public class TestArrowFile extends BaseFileTest {
     int count = COUNT;
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeComplexData(count, parent);
       FieldVector root = parent.getChild("root");
       validateComplexContent(count, new VectorSchemaRoot(root));
@@ -268,7 +268,7 @@ public class TestArrowFile extends BaseFileTest {
 
     // write
     try (BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-         NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+         MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeUnionData(count, parent);
       validateUnionData(count, new VectorSchemaRoot(parent.getChild("root")));
       write(parent.getChild("root"), file, stream);
@@ -304,7 +304,7 @@ public class TestArrowFile extends BaseFileTest {
 
     try (VectorSchemaRoot root = VectorSchemaRoot.create(MessageSerializerTest.testSchema(), allocator)) {
       root.getFieldVectors().get(0).allocateNew();
-      NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
+      TinyIntVector vector = (TinyIntVector) root.getFieldVectors().get(0);
       for (int i = 0; i < 16; i++) {
         vector.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
       }
@@ -352,7 +352,7 @@ public class TestArrowFile extends BaseFileTest {
 
   private void validateTinyData(VectorSchemaRoot root) {
     Assert.assertEquals(16, root.getRowCount());
-    NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
+    TinyIntVector vector = (TinyIntVector) root.getFieldVectors().get(0);
     for (int i = 0; i < 16; i++) {
       if (i < 8) {
         Assert.assertEquals((byte) (i + 1), vector.get(i));
@@ -382,7 +382,7 @@ public class TestArrowFile extends BaseFileTest {
 
     // write
     try (BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-         NullableMapVector vector = (NullableMapVector) field.createVector(originalVectorAllocator)) {
+         MapVector vector = (MapVector) field.createVector(originalVectorAllocator)) {
       vector.allocateNewSafe();
       vector.setValueCount(0);
 
@@ -558,10 +558,10 @@ public class TestArrowFile extends BaseFileTest {
 
     // write
     try (BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-         NullableMapVector parent = NullableMapVector.empty("parent", originalVectorAllocator)) {
+         MapVector parent = MapVector.empty("parent", originalVectorAllocator)) {
       FixedSizeListVector tuples = parent.addOrGet("float-pairs", FieldType.nullable(new FixedSizeList(2)), FixedSizeListVector.class);
-      NullableFloat4Vector floats = (NullableFloat4Vector) tuples.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
-      NullableIntVector ints = parent.addOrGet("ints", FieldType.nullable(new Int(32, true)), NullableIntVector.class);
+      Float4Vector floats = (Float4Vector) tuples.addOrGetVector(FieldType.nullable(MinorType.FLOAT4.getType())).getVector();
+      IntVector ints = parent.addOrGet("ints", FieldType.nullable(new Int(32, true)), IntVector.class);
       parent.allocateNew();
 
       for (int i = 0; i < 10; i++) {
@@ -618,7 +618,7 @@ public class TestArrowFile extends BaseFileTest {
     // write
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeVarBinaryData(count, parent);
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateVarBinary(count, root);
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStream.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStream.java
index 7a8586a..f87a0eb 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStream.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStream.java
@@ -26,7 +26,9 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
-import org.apache.arrow.vector.NullableTinyIntVector;
+import io.netty.buffer.ArrowBuf;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.TinyIntVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.ipc.ArrowStreamReader;
 import org.apache.arrow.vector.ipc.ArrowStreamWriter;
@@ -65,7 +67,7 @@ public class TestArrowStream extends BaseFileTest {
       int numBatches = 1;
 
       root.getFieldVectors().get(0).allocateNew();
-      NullableTinyIntVector vector = (NullableTinyIntVector)root.getFieldVectors().get(0);
+      TinyIntVector vector = (TinyIntVector)root.getFieldVectors().get(0);
       for (int i = 0; i < 16; i++) {
         vector.set(i, i < 8 ? 1 : 0, (byte) (i + 1));
       }
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStreamPipe.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStreamPipe.java
index 65e6cea..bd1ec94 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStreamPipe.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestArrowStreamPipe.java
@@ -28,7 +28,7 @@ import java.nio.channels.WritableByteChannel;
 
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
-import org.apache.arrow.vector.NullableTinyIntVector;
+import org.apache.arrow.vector.TinyIntVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.ipc.ArrowStreamReader;
 import org.apache.arrow.vector.ipc.ArrowStreamWriter;
@@ -61,7 +61,7 @@ public class TestArrowStreamPipe {
         writer.start();
         for (int j = 0; j < numBatches; j++) {
           root.getFieldVectors().get(0).allocateNew();
-          NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
+          TinyIntVector vector = (TinyIntVector) root.getFieldVectors().get(0);
           // Send a changing batch id first
           vector.set(0, j);
           for (int i = 1; i < 16; i++) {
@@ -103,10 +103,9 @@ public class TestArrowStreamPipe {
             done = true;
             return false;
           }
-
           VectorSchemaRoot root = getVectorSchemaRoot();
           Assert.assertEquals(16, root.getRowCount());
-          NullableTinyIntVector vector = (NullableTinyIntVector) root.getFieldVectors().get(0);
+          TinyIntVector vector = (TinyIntVector) root.getFieldVectors().get(0);
           Assert.assertEquals((byte) (batchesRead - 1), vector.get(0));
           for (int i = 1; i < 16; i++) {
             if (i < 8) {
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
index c3e0b79..6257170 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/ipc/TestJSONFile.java
@@ -25,7 +25,7 @@ import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.complex.MapVector;
-import org.apache.arrow.vector.complex.NullableMapVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.dictionary.DictionaryProvider.MapDictionaryProvider;
 import org.apache.arrow.vector.types.pojo.Schema;
@@ -73,7 +73,7 @@ public class TestJSONFile extends BaseFileTest {
     int count = COUNT;
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeComplexData(count, parent);
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateComplexContent(root.getRowCount(), root);
@@ -95,7 +95,7 @@ public class TestJSONFile extends BaseFileTest {
     int count = COUNT;
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeUnionData(count, parent);
       printVectors(parent.getChildrenFromFields());
 
@@ -127,7 +127,7 @@ public class TestJSONFile extends BaseFileTest {
     // write
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
 
       writeDateTimeData(count, parent);
 
@@ -292,7 +292,7 @@ public class TestJSONFile extends BaseFileTest {
     // write
     try (
         BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-        NullableMapVector parent = NullableMapVector.empty("parent", vectorAllocator)) {
+        MapVector parent = MapVector.empty("parent", vectorAllocator)) {
       writeVarBinaryData(count, parent);
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateVarBinary(count, root);

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