You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by em...@apache.org on 2019/06/11 05:35:36 UTC

[arrow] branch master updated: ARROW-1261: [Java] Add MapVector with reader and writer

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

emkornfield 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 f3fde09  ARROW-1261: [Java] Add MapVector with reader and writer
f3fde09 is described below

commit f3fde098b5126ca097a7975e07aa58f92e6915ba
Author: Bryan Cutler <cu...@gmail.com>
AuthorDate: Mon Jun 10 22:34:02 2019 -0700

    ARROW-1261: [Java] Add MapVector with reader and writer
    
    This adds `MapVector` as a subclass of `ListVector` where the data vector is a Struct with 2 fields: "key" and "value".  A new writer `UnionMapWriter` is added that extends `UnionListWriter` to simplify writing key, value fields. Similarly, the `UnionMapReader` is added to read key, value fields.
    
    Author: Bryan Cutler <cu...@gmail.com>
    
    Closes #4444 from BryanCutler/java-map-type-ARROW-1279 and squashes the following commits:
    
    f53d11ed1 <Bryan Cutler> Added test to write data as list with different field names
    e68acd33f <Bryan Cutler> Expanded java docs for MapVector and UnionMapWriter
    1b153e468 <Bryan Cutler> make StructVector respect nullable flag
    f627ed073 <Bryan Cutler> revert changes from using NonNullableStructVector
    372764307 <Bryan Cutler> use Preconditions.checkArgument in MapVector
    7f602b8fe <Bryan Cutler> Added split and transfer test
    afe89e29e <Bryan Cutler> fix style checks and add javadocs
    03c380f6b <Bryan Cutler> fixed initializeChildrenFromFields in MapVector
    3a9c19471 <Bryan Cutler> fix imports
    c90347e79 <Bryan Cutler> Now using StructVector with nullable false for struct and key vectors, writers can set a nullable flag to create non-nullable vectors
    3e620d9c7 <Bryan Cutler> make MapVector use NonNullableStructVector
    e19f6cf27 <Bryan Cutler> Added roundtrip tests for MapVector Java IPC
    4dcb622c8 <Bryan Cutler> initial MapVector with reader and writer
---
 java/vector/src/main/codegen/data/ArrowTypes.tdd   |   5 +
 .../codegen/templates/AbstractFieldWriter.java     |  14 +
 .../src/main/codegen/templates/ComplexCopier.java  |   1 +
 .../src/main/codegen/templates/StructWriters.java  |  15 +-
 .../main/codegen/templates/UnionListWriter.java    |   4 +-
 .../src/main/codegen/templates/UnionMapWriter.java | 195 ++++++
 .../java/org/apache/arrow/vector/TypeLayout.java   |  10 +
 .../apache/arrow/vector/complex/ListVector.java    |   4 +-
 .../org/apache/arrow/vector/complex/MapVector.java | 140 ++++
 .../apache/arrow/vector/complex/StructVector.java  |   7 -
 .../vector/complex/impl/PromotableWriter.java      |  11 +-
 .../arrow/vector/complex/impl/UnionMapReader.java  |  77 +++
 .../java/org/apache/arrow/vector/types/Types.java  |  22 +
 .../org/apache/arrow/vector/TestMapVector.java     | 703 +++++++++++++++++++++
 .../org/apache/arrow/vector/ipc/BaseFileTest.java  | 144 +++++
 .../org/apache/arrow/vector/ipc/TestArrowFile.java |  92 +++
 .../org/apache/arrow/vector/ipc/TestJSONFile.java  |  32 +
 17 files changed, 1458 insertions(+), 18 deletions(-)

diff --git a/java/vector/src/main/codegen/data/ArrowTypes.tdd b/java/vector/src/main/codegen/data/ArrowTypes.tdd
index 8c4702a..468db26 100644
--- a/java/vector/src/main/codegen/data/ArrowTypes.tdd
+++ b/java/vector/src/main/codegen/data/ArrowTypes.tdd
@@ -41,6 +41,11 @@
       complex: true
     },
     {
+      name: "Map",
+      fields: [{name: "keysSorted", type: boolean}],
+      complex: true
+    },
+    {
       name: "Int",
       fields: [{name: "bitWidth", type: int}, {name: "isSigned", type: boolean}],
       complex: false
diff --git a/java/vector/src/main/codegen/templates/AbstractFieldWriter.java b/java/vector/src/main/codegen/templates/AbstractFieldWriter.java
index 8e98400..c4ac32c 100644
--- a/java/vector/src/main/codegen/templates/AbstractFieldWriter.java
+++ b/java/vector/src/main/codegen/templates/AbstractFieldWriter.java
@@ -30,6 +30,20 @@ package org.apache.arrow.vector.complex.impl;
  */
 @SuppressWarnings("unused")
 abstract class AbstractFieldWriter extends AbstractBaseWriter implements FieldWriter {
+
+  protected boolean addVectorAsNullable = true;
+
+  /**
+   * Set flag to control the FieldType.nullable property when a writer creates a new vector.
+   * If true then vectors created will be nullable, this is the default behavior. If false then
+   * vectors created will be non-nullable.
+   *
+   * @param nullable Whether or not to create nullable vectors (default behavior is true)
+   */
+  public void setAddVectorAsNullable(boolean nullable) {
+    addVectorAsNullable = nullable;
+  }
+
   @Override
   public void start() {
     throw new IllegalStateException(String.format("You tried to start when you are using a ValueWriter of type %s.", this.getClass().getSimpleName()));
diff --git a/java/vector/src/main/codegen/templates/ComplexCopier.java b/java/vector/src/main/codegen/templates/ComplexCopier.java
index 1b21c4a..620e81e 100644
--- a/java/vector/src/main/codegen/templates/ComplexCopier.java
+++ b/java/vector/src/main/codegen/templates/ComplexCopier.java
@@ -46,6 +46,7 @@ public class ComplexCopier {
       switch (mt) {
 
       case LIST:
+      case MAP:
         if (reader.isSet()) {
           writer.startList();
           while (reader.next()) {
diff --git a/java/vector/src/main/codegen/templates/StructWriters.java b/java/vector/src/main/codegen/templates/StructWriters.java
index 957b19b..fec172e 100644
--- a/java/vector/src/main/codegen/templates/StructWriters.java
+++ b/java/vector/src/main/codegen/templates/StructWriters.java
@@ -65,7 +65,8 @@ public class ${mode}StructWriter extends AbstractFieldWriter {
         list(child.getName());
         break;
       case UNION:
-        UnionWriter writer = new UnionWriter(container.addOrGet(child.getName(), FieldType.nullable(MinorType.UNION.getType()), UnionVector.class), getNullableStructWriterFactory());
+        FieldType fieldType = new FieldType(addVectorAsNullable, MinorType.UNION.getType(), null, null);
+        UnionWriter writer = new UnionWriter(container.addOrGet(child.getName(), fieldType, UnionVector.class), getNullableStructWriterFactory());
         fields.put(handleCase(child.getName()), writer);
         break;
 <#list vv.types as type><#list type.minor as minor>
@@ -122,7 +123,8 @@ public class ${mode}StructWriter extends AbstractFieldWriter {
     FieldWriter writer = fields.get(finalName);
     if(writer == null){
       int vectorCount=container.size();
-      StructVector vector = container.addOrGet(name, FieldType.nullable(MinorType.STRUCT.getType()), StructVector.class);
+      FieldType fieldType = new FieldType(addVectorAsNullable, MinorType.STRUCT.getType(), null, null);
+      StructVector vector = container.addOrGet(name, fieldType, StructVector.class);
       writer = new PromotableWriter(vector, container, getNullableStructWriterFactory());
       if(vectorCount != container.size()) {
         writer.allocate();
@@ -166,7 +168,8 @@ public class ${mode}StructWriter extends AbstractFieldWriter {
     FieldWriter writer = fields.get(finalName);
     int vectorCount = container.size();
     if(writer == null) {
-      writer = new PromotableWriter(container.addOrGet(name, FieldType.nullable(MinorType.LIST.getType()), ListVector.class), container, getNullableStructWriterFactory());
+      FieldType fieldType = new FieldType(addVectorAsNullable, MinorType.LIST.getType(), null, null);
+      writer = new PromotableWriter(container.addOrGet(name, fieldType, ListVector.class), container, getNullableStructWriterFactory());
       if (container.size() > vectorCount) {
         writer.allocate();
       }
@@ -232,8 +235,8 @@ public class ${mode}StructWriter extends AbstractFieldWriter {
     if(writer == null) {
       ValueVector vector;
       ValueVector currentVector = container.getChild(name);
-      ${vectName}Vector v = container.addOrGet(name, 
-          FieldType.nullable(
+      ${vectName}Vector v = container.addOrGet(name,
+          new FieldType(addVectorAsNullable,
           <#if minor.typeParams??>
             <#if minor.arrowTypeConstructorParams??>
               <#assign constructorParams = minor.arrowTypeConstructorParams />
@@ -247,7 +250,7 @@ public class ${mode}StructWriter extends AbstractFieldWriter {
           <#else>
             MinorType.${upperName}.getType()
           </#if>
-          ),
+          ,null, null),
           ${vectName}Vector.class);
       writer = new PromotableWriter(v, container, getNullableStructWriterFactory());
       vector = v;
diff --git a/java/vector/src/main/codegen/templates/UnionListWriter.java b/java/vector/src/main/codegen/templates/UnionListWriter.java
index 13c1fd9..a9c40b3 100644
--- a/java/vector/src/main/codegen/templates/UnionListWriter.java
+++ b/java/vector/src/main/codegen/templates/UnionListWriter.java
@@ -39,8 +39,8 @@ package org.apache.arrow.vector.complex.impl;
 @SuppressWarnings("unused")
 public class UnionListWriter extends AbstractFieldWriter {
 
-  private ListVector vector;
-  private PromotableWriter writer;
+  protected ListVector vector;
+  protected PromotableWriter writer;
   private boolean inStruct = false;
   private String structName;
   private int lastIndex = 0;
diff --git a/java/vector/src/main/codegen/templates/UnionMapWriter.java b/java/vector/src/main/codegen/templates/UnionMapWriter.java
new file mode 100644
index 0000000..df9d1d1
--- /dev/null
+++ b/java/vector/src/main/codegen/templates/UnionMapWriter.java
@@ -0,0 +1,195 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import org.apache.arrow.vector.complex.writer.DecimalWriter;
+import org.apache.arrow.vector.holders.DecimalHolder;
+
+import java.lang.UnsupportedOperationException;
+import java.math.BigDecimal;
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="/org/apache/arrow/vector/complex/impl/UnionMapWriter.java" />
+
+
+<#include "/@includes/license.ftl" />
+
+package org.apache.arrow.vector.complex.impl;
+
+<#include "/@includes/vv_imports.ftl" />
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+/**
+ * <p>Writer for MapVectors. This extends UnionListWriter to simplify writing map entries to a list
+ * of struct elements, with "key" and "value" fields. The procedure for writing a map begin with
+ * {@link #startMap()} followed by {@link #startEntry()}. An entry is written by using the
+ * {@link #key()} writer to write the key, then the {@link #value()} writer to write a value. After
+ * writing the value, call {@link #endEntry()} to complete the entry. Each map can have 1 or more
+ * entries. When done writing entries, call {@link #endMap()} to complete the map.
+ *
+ * <p>NOTE: the MapVector can have NULL values by not writing to position. If a map is started with
+ * {@link #startMap()}, then it must have a key written. The value of a map entry can be NULL by
+ * not using the {@link #value()} writer.
+ *
+ * <p>Example to write the following map to position 5 of a vector
+ * <pre>{@code
+ *   // {
+ *   //   1 -> 3,
+ *   //   2 -> 4,
+ *   //   3 -> NULL
+ *   // }
+ *
+ *   UnionMapWriter writer = ...
+ *
+ *   writer.setPosition(5);
+ *   writer.startMap();
+ *   writer.startEntry();
+ *   writer.key().integer().writeInt(1);
+ *   writer.value().integer().writeInt(3);
+ *   writer.endEntry();
+ *   writer.startEntry();
+ *   writer.key().integer().writeInt(2);
+ *   writer.value().integer().writeInt(4);
+ *   writer.endEntry();
+ *   writer.startEntry();
+ *   writer.key().integer().writeInt(3);
+ *   writer.endEntry();
+ *   writer.endMap();
+ * </pre>
+ * </p>
+ */
+@SuppressWarnings("unused")
+public class UnionMapWriter extends UnionListWriter {
+
+  /**
+   * Current mode for writing map entries, set by calling {@link #key()} or {@link #value()}
+   * and reset with a call to {@link #endEntry()}. With KEY mode, a struct writer with field
+   * named "key" is returned. With VALUE mode, a struct writer with field named "value" is
+   * returned. In OFF mode, the writer will behave like a standard UnionListWriter
+   */
+  private enum MapWriteMode {
+    OFF,
+    KEY,
+    VALUE,
+  }
+
+  private MapWriteMode mode = MapWriteMode.OFF;
+  private StructWriter entryWriter;
+
+  public UnionMapWriter(MapVector vector) {
+    super(vector);
+    entryWriter = struct();
+  }
+
+  /** Start writing a map that consists of 1 or more entries. */
+  public void startMap() {
+    startList();
+  }
+
+  /** Complete the map. */
+  public void endMap() {
+    endList();
+  }
+
+  /**
+   * Start a map entry that should be followed by calls to {@link #key()} and {@link #value()}
+   * writers. Call {@link #endEntry()} to complete the entry.
+   */
+  public void startEntry() {
+    writer.setAddVectorAsNullable(false);
+    entryWriter.start();
+  }
+
+  /** Complete the map entry. */
+  public void endEntry() {
+    entryWriter.end();
+    mode = MapWriteMode.OFF;
+    writer.setAddVectorAsNullable(true);
+  }
+
+  /** Return the key writer that is used to write to the "key" field. */
+  public UnionMapWriter key() {
+    writer.setAddVectorAsNullable(false);
+    mode = MapWriteMode.KEY;
+    return this;
+  }
+
+  /** Return the value writer that is used to write to the "value" field. */
+  public UnionMapWriter value() {
+    writer.setAddVectorAsNullable(true);
+    mode = MapWriteMode.VALUE;
+    return this;
+  }
+
+  <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
+  <#assign fields = minor.fields!type.fields />
+  <#assign uncappedName = name?uncap_first/>
+  <#if uncappedName == "int" ><#assign uncappedName = "integer" /></#if>
+  <#if !minor.typeParams?? >
+  @Override
+  public ${name}Writer ${uncappedName}() {
+    switch (mode) {
+      case KEY:
+        return entryWriter.${uncappedName}(MapVector.KEY_NAME);
+      case VALUE:
+        return entryWriter.${uncappedName}(MapVector.VALUE_NAME);
+      default:
+        return this;
+    }
+  }
+
+  </#if>
+  </#list></#list>
+  @Override
+  public DecimalWriter decimal() {
+    switch (mode) {
+      case KEY:
+        return entryWriter.decimal(MapVector.KEY_NAME);
+      case VALUE:
+        return entryWriter.decimal(MapVector.VALUE_NAME);
+      default:
+        return this;
+    }
+  }
+
+  @Override
+  public StructWriter struct() {
+    switch (mode) {
+      case KEY:
+        return entryWriter.struct(MapVector.KEY_NAME);
+      case VALUE:
+        return entryWriter.struct(MapVector.VALUE_NAME);
+      default:
+        return super.struct();
+    }
+  }
+
+  @Override
+  public ListWriter list() {
+    switch (mode) {
+      case KEY:
+        return entryWriter.list(MapVector.KEY_NAME);
+      case VALUE:
+        return entryWriter.list(MapVector.VALUE_NAME);
+      default:
+        return super.list();
+    }
+  }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/TypeLayout.java b/java/vector/src/main/java/org/apache/arrow/vector/TypeLayout.java
index 9fb49f7..6f1316a 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/TypeLayout.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/TypeLayout.java
@@ -37,6 +37,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType.FixedSizeList;
 import org.apache.arrow.vector.types.pojo.ArrowType.FloatingPoint;
 import org.apache.arrow.vector.types.pojo.ArrowType.Int;
 import org.apache.arrow.vector.types.pojo.ArrowType.Interval;
+import org.apache.arrow.vector.types.pojo.ArrowType.Map;
 import org.apache.arrow.vector.types.pojo.ArrowType.Null;
 import org.apache.arrow.vector.types.pojo.ArrowType.Struct;
 import org.apache.arrow.vector.types.pojo.ArrowType.Time;
@@ -116,6 +117,15 @@ public class TypeLayout {
       }
 
       @Override
+      public TypeLayout visit(Map type) {
+        List<BufferLayout> vectors = asList(
+            BufferLayout.validityVector(),
+            BufferLayout.offsetBuffer()
+        );
+        return new TypeLayout(vectors);
+      }
+
+      @Override
       public TypeLayout visit(FloatingPoint type) {
         int bitWidth;
         switch (type.getPrecision()) {
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
index c19ff68..6945e91 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/ListVector.java
@@ -68,7 +68,7 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
   }
 
   protected ArrowBuf validityBuffer;
-  private UnionListReader reader;
+  protected UnionListReader reader;
   private CallBack callBack;
   private final FieldType fieldType;
   private int validityAllocationSizeInBytes;
@@ -94,7 +94,7 @@ public class ListVector extends BaseRepeatedValueVector implements FieldVector,
    * Constructs a new instance.
    *
    * @param name The name of the instance.
-   * @param allocator The allocator to use to allocating/reallocating buffers.
+   * @param allocator The allocator to use for allocating/reallocating buffers.
    * @param fieldType The type of this list.
    * @param callBack A schema change callback.
    */
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
new file mode 100644
index 0000000..340fb2a
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/MapVector.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.vector.complex;
+
+import static org.apache.arrow.util.Preconditions.checkArgument;
+
+import java.util.List;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.AddOrGetResult;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.complex.impl.UnionMapReader;
+import org.apache.arrow.vector.complex.impl.UnionMapWriter;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType.Map;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.CallBack;
+
+/**
+ * A MapVector is used to store entries of key/value pairs. It is a container vector that is
+ * composed of a list of struct values with "key" and "value" fields. The MapVector is nullable,
+ * but if a map is set at a given index, there must be an entry. In other words, the StructVector
+ * data is non-nullable. Also for a given entry, the "key" is non-nullable, however the "value" can
+ * be null.
+ */
+public class MapVector extends ListVector {
+
+  public static final String KEY_NAME = "key";
+  public static final String VALUE_NAME = "value";
+
+  /**
+   * Construct an empty MapVector with no data. Child vectors must be added subsequently.
+   *
+   * @param name The name of the vector.
+   * @param allocator The allocator used for allocating/reallocating buffers.
+   * @param keysSorted True if the map keys have been pre-sorted.
+   * @return a new instance of MapVector.
+   */
+  public static MapVector empty(String name, BufferAllocator allocator, boolean keysSorted) {
+    return new MapVector(name, allocator, FieldType.nullable(new Map(keysSorted)), null);
+  }
+
+  /**
+   * Construct a MapVector instance.
+   *
+   * @param name The name of the vector.
+   * @param allocator The allocator used for allocating/reallocating buffers.
+   * @param fieldType The type definition of the MapVector.
+   * @param callBack A schema change callback.
+   */
+  public MapVector(String name, BufferAllocator allocator, FieldType fieldType, CallBack callBack) {
+    super(name, allocator, fieldType, callBack);
+    reader = new UnionMapReader(this);
+  }
+
+  /**
+   * Initialize child vectors of the map from the given list of fields.
+   *
+   * @param children List of fields that will be children of this MapVector.
+   */
+  @Override
+  public void initializeChildrenFromFields(List<Field> children) {
+    checkArgument(children.size() == 1, "Maps have one List child. Found: " + children);
+
+    Field structField = children.get(0);
+    MinorType minorType = Types.getMinorTypeForArrowType(structField.getType());
+    checkArgument(minorType == MinorType.STRUCT && !structField.isNullable(),
+        "Map data should be a non-nullable struct type");
+    checkArgument(structField.getChildren().size() == 2,
+        "Map data should be a struct with 2 children. Found: " + children);
+
+    Field keyField = structField.getChildren().get(0);
+    checkArgument(!keyField.isNullable(), "Map data key type should be a non-nullable");
+
+    AddOrGetResult<FieldVector> addOrGetVector = addOrGetVector(structField.getFieldType());
+    checkArgument(addOrGetVector.isCreated(), "Child vector already existed: " + addOrGetVector.getVector());
+
+    addOrGetVector.getVector().initializeChildrenFromFields(structField.getChildren());
+  }
+
+  /**
+   * Get the writer for this MapVector instance.
+   */
+  @Override
+  public UnionMapWriter getWriter() {
+    return new UnionMapWriter(this);
+  }
+
+  /**
+   * Get the reader for this MapVector instance.
+   */
+  @Override
+  public UnionMapReader getReader() {
+    return (UnionMapReader)reader;
+  }
+
+  /**
+   * Add a child vector that will be the list vector, or get the vector if already added.
+   *
+   * @param fieldType The field type of the child vector.
+   * @param <T> Type of the resulting vector.
+   * @return return an AddOrGetResult instance that contains the vector and created flag.
+   */
+  @Override
+  public <T extends ValueVector> AddOrGetResult<T> addOrGetVector(FieldType fieldType) {
+    AddOrGetResult<T> result = super.addOrGetVector(fieldType);
+    reader = new UnionMapReader(this);
+    return result;
+  }
+
+  /**
+   * Promote this MapVector to a UnionVector.
+   *
+   * @return the new UnionVector.
+   */
+  @Override
+  public UnionVector promoteToUnion() {
+    UnionVector result = super.promoteToUnion();
+    reader = new UnionMapReader(this);
+    return result;
+  }
+}
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java
index 03cb359..7481fa8 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/StructVector.java
@@ -95,13 +95,6 @@ public class StructVector extends NonNullableStructVector implements FieldVector
   }
 
   @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());
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 f382b13..ce3948c 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
@@ -129,6 +129,14 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
     }
   }
 
+  @Override
+  public void setAddVectorAsNullable(boolean nullable) {
+    super.setAddVectorAsNullable(nullable);
+    if (writer instanceof AbstractFieldWriter) {
+      ((AbstractFieldWriter) writer).setAddVectorAsNullable(nullable);
+    }
+  }
+
   private void setWriter(ValueVector v) {
     setWriter(v, null);
   }
@@ -180,7 +188,8 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
       if (arrowType == null) {
         arrowType = type.getType();
       }
-      ValueVector v = listVector.addOrGetVector(FieldType.nullable(arrowType)).getVector();
+      FieldType fieldType = new FieldType(addVectorAsNullable, arrowType, null, null);
+      ValueVector v = listVector.addOrGetVector(fieldType).getVector();
       v.allocateNew();
       setWriter(v, arrowType);
       writer.setPosition(position);
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionMapReader.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionMapReader.java
new file mode 100644
index 0000000..7a1bdce
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/impl/UnionMapReader.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.vector.complex.impl;
+
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.types.Types.MinorType;
+
+/**
+ * Reader for a MapVector.
+ */
+public class UnionMapReader extends UnionListReader {
+
+  private String keyName = MapVector.KEY_NAME;
+  private String valueName = MapVector.VALUE_NAME;
+
+  /**
+   * Construct a new reader for the given vector.
+   *
+   * @param vector Vector to read from.
+   */
+  public UnionMapReader(MapVector vector) {
+    super(vector);
+  }
+
+  /**
+   * Set the key, value field names to read.
+   *
+   * @param key Field name for key.
+   * @param value Field name for value.
+   */
+  public void setKeyValueNames(String key, String value) {
+    keyName = key;
+    valueName = value;
+  }
+
+  /**
+   * Start reading a key from the map entry.
+   *
+   * @return reader that can be used to read the key.
+   */
+  public FieldReader key() {
+    return reader().reader(keyName);
+  }
+
+  /**
+   * Start reading a value element from the map entry.
+   *
+   * @return reader that can be used to read the value.
+   */
+  public FieldReader value() {
+    return reader().reader(valueName);
+  }
+
+  /**
+   * Return the MinorType of the reader as MAP.
+   */
+  @Override
+  public MinorType getMinorType() {
+    return MinorType.MAP;
+  }
+}
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 f8ec4ca..7096104 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
@@ -59,6 +59,7 @@ import org.apache.arrow.vector.VarCharVector;
 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.MapVector;
 import org.apache.arrow.vector.complex.StructVector;
 import org.apache.arrow.vector.complex.UnionVector;
 import org.apache.arrow.vector.complex.impl.BigIntWriterImpl;
@@ -110,6 +111,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType.FloatingPoint;
 import org.apache.arrow.vector.types.pojo.ArrowType.Int;
 import org.apache.arrow.vector.types.pojo.ArrowType.Interval;
 import org.apache.arrow.vector.types.pojo.ArrowType.List;
+import org.apache.arrow.vector.types.pojo.ArrowType.Map;
 import org.apache.arrow.vector.types.pojo.ArrowType.Null;
 import org.apache.arrow.vector.types.pojo.ArrowType.Struct;
 import org.apache.arrow.vector.types.pojo.ArrowType.Time;
@@ -634,6 +636,21 @@ public class Types {
         return new UnionWriter((UnionVector) vector);
       }
     },
+    MAP(null) {
+      @Override
+      public FieldVector getNewVector(
+          String name,
+          FieldType fieldType,
+          BufferAllocator allocator,
+          CallBack schemaChangeCallback) {
+        return new MapVector(name, allocator, fieldType, schemaChangeCallback);
+      }
+
+      @Override
+      public FieldWriter getNewFieldWriter(ValueVector vector) {
+        return new UnionListWriter((MapVector) vector);
+      }
+    },
     TIMESTAMPSECTZ(null) {
       @Override
       public FieldVector getNewVector(
@@ -752,6 +769,11 @@ public class Types {
       }
 
       @Override
+      public MinorType visit(Map type) {
+        return MinorType.MAP;
+      }
+
+      @Override
       public MinorType visit(Int type) {
         switch (type.getBitWidth()) {
           case 8:
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
new file mode 100644
index 0000000..a16390c
--- /dev/null
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestMapVector.java
@@ -0,0 +1,703 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.vector;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Map;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.impl.UnionMapReader;
+import org.apache.arrow.vector.complex.impl.UnionMapWriter;
+import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.complex.writer.BaseWriter.ListWriter;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.JsonStringArrayList;
+import org.apache.arrow.vector.util.TransferPair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ArrowBuf;
+
+public class TestMapVector {
+
+  private BufferAllocator allocator;
+
+  @Before
+  public void init() {
+    allocator = new DirtyRootAllocator(Long.MAX_VALUE, (byte) 100);
+  }
+
+  @After
+  public void terminate() throws Exception {
+    allocator.close();
+  }
+
+  public <T> T getResultKey(Map<?, T> resultStruct) {
+    assertTrue(resultStruct.containsKey(MapVector.KEY_NAME));
+    return resultStruct.get(MapVector.KEY_NAME);
+  }
+
+  public <T> T getResultValue(Map<?, T> resultStruct) {
+    assertTrue(resultStruct.containsKey(MapVector.VALUE_NAME));
+    return resultStruct.get(MapVector.VALUE_NAME);
+  }
+
+  @Test
+  public void testBasicOperation() {
+    int count = 5;
+    try (MapVector mapVector = MapVector.empty("map", allocator, false)) {
+      mapVector.allocateNew();
+      UnionMapWriter mapWriter = mapVector.getWriter();
+      for (int i = 0; i < count; i++) {
+        mapWriter.startMap();
+        for (int j = 0; j < i + 1; j++) {
+          mapWriter.startEntry();
+          mapWriter.key().bigInt().writeBigInt(j);
+          mapWriter.value().integer().writeInt(j);
+          mapWriter.endEntry();
+        }
+        mapWriter.endMap();
+      }
+      mapWriter.setValueCount(count);
+      UnionMapReader mapReader = mapVector.getReader();
+      for (int i = 0; i < count; i++) {
+        mapReader.setPosition(i);
+        for (int j = 0; j < i + 1; j++) {
+          mapReader.next();
+          assertEquals("record: " + i, j, mapReader.key().readLong().longValue());
+          assertEquals(j, mapReader.value().readInteger().intValue());
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testBasicOperationNulls() {
+    int count = 6;
+    try (MapVector mapVector = MapVector.empty("map", allocator, false)) {
+      mapVector.allocateNew();
+      UnionMapWriter mapWriter = mapVector.getWriter();
+      for (int i = 0; i < count; i++) {
+        // i == 1 is a NULL
+        if (i != 1) {
+          mapWriter.setPosition(i);
+          mapWriter.startMap();
+          // i == 3 is an empty map
+          if (i != 3) {
+            for (int j = 0; j < i + 1; j++) {
+              mapWriter.startEntry();
+              mapWriter.key().bigInt().writeBigInt(j);
+              // i == 5 maps to a NULL value
+              if (i != 5) {
+                mapWriter.value().integer().writeInt(j);
+              }
+              mapWriter.endEntry();
+            }
+          }
+          mapWriter.endMap();
+        }
+      }
+      mapWriter.setValueCount(count);
+      UnionMapReader mapReader = mapVector.getReader();
+      for (int i = 0; i < count; i++) {
+        mapReader.setPosition(i);
+        if (i == 1) {
+          assertFalse(mapReader.isSet());
+        } else {
+          if (i == 3) {
+            JsonStringArrayList<?> result = (JsonStringArrayList<?>) mapReader.readObject();
+            assertTrue(result.isEmpty());
+          } else {
+            for (int j = 0; j < i + 1; j++) {
+              mapReader.next();
+              assertEquals("record: " + i, j, mapReader.key().readLong().longValue());
+              if (i == 5) {
+                assertFalse(mapReader.value().isSet());
+              } else {
+                assertEquals(j, mapReader.value().readInteger().intValue());
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testCopyFrom() throws Exception {
+    try (MapVector inVector = MapVector.empty("input", allocator, false);
+         MapVector outVector = MapVector.empty("output", allocator, false)) {
+      UnionMapWriter writer = inVector.getWriter();
+      writer.allocate();
+
+      // populate input vector with the following records
+      // {1 -> 11, 2 -> 22, 3 -> 33}
+      // null
+      // {2 -> null}
+      writer.setPosition(0); // optional
+      writer.startMap();
+      writer.startEntry();
+      writer.key().bigInt().writeBigInt(1);
+      writer.value().bigInt().writeBigInt(11);
+      writer.endEntry();
+      writer.startEntry();
+      writer.key().bigInt().writeBigInt(2);
+      writer.value().bigInt().writeBigInt(22);
+      writer.endEntry();
+      writer.startEntry();
+      writer.key().bigInt().writeBigInt(3);
+      writer.value().bigInt().writeBigInt(33);
+      writer.endEntry();
+      writer.endMap();
+
+      writer.setPosition(2);
+      writer.startMap();
+      writer.startEntry();
+      writer.key().bigInt().writeBigInt(2);
+      writer.endEntry();
+      writer.endMap();
+
+      writer.setValueCount(3);
+
+      // copy values from input to output
+      outVector.allocateNew();
+      for (int i = 0; i < 3; i++) {
+        outVector.copyFrom(i, i, inVector);
+      }
+      outVector.setValueCount(3);
+
+      // assert the output vector is correct
+      FieldReader reader = outVector.getReader();
+      assertTrue("shouldn't be null", reader.isSet());
+      reader.setPosition(1);
+      assertFalse("should be null", reader.isSet());
+      reader.setPosition(2);
+      assertTrue("shouldn't be null", reader.isSet());
+
+
+      /* index 0 */
+      Object result = outVector.getObject(0);
+      ArrayList<?> resultSet = (ArrayList<?>) result;
+      assertEquals(3, resultSet.size());
+      Map<?, ?> resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(1L, getResultKey(resultStruct));
+      assertEquals(11L, getResultValue(resultStruct));
+      resultStruct = (Map<?, ?>) resultSet.get(1);
+      assertEquals(2L, getResultKey(resultStruct));
+      assertEquals(22L, getResultValue(resultStruct));
+      resultStruct = (Map<?, ?>) resultSet.get(2);
+      assertEquals(3L, getResultKey(resultStruct));
+      assertEquals(33L, getResultValue(resultStruct));
+
+      /* index 1 */
+      result = outVector.getObject(1);
+      assertNull(result);
+
+      /* index 2 */
+      result = outVector.getObject(2);
+      resultSet = (ArrayList<?>) result;
+      assertEquals(1, resultSet.size());
+      resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(2L, getResultKey(resultStruct));
+      assertFalse(resultStruct.containsKey(MapVector.VALUE_NAME));
+    }
+  }
+
+  @Test
+  public void testSplitAndTransfer() throws Exception {
+    try (MapVector mapVector = MapVector.empty("sourceVector", allocator, false)) {
+
+      /* Explicitly add the map child vectors */
+      FieldType type = new FieldType(false, ArrowType.Struct.INSTANCE, null, null);
+      AddOrGetResult<StructVector> addResult = mapVector.addOrGetVector(type);
+      FieldType keyType = new FieldType(false, MinorType.BIGINT.getType(), null, null);
+      FieldType valueType = FieldType.nullable(MinorType.FLOAT8.getType());
+      addResult.getVector().addOrGet(MapVector.KEY_NAME, keyType, BigIntVector.class);
+      addResult.getVector().addOrGet(MapVector.VALUE_NAME, valueType, Float8Vector.class);
+
+      UnionMapWriter mapWriter = mapVector.getWriter();
+
+      /* allocate memory */
+      mapWriter.allocate();
+
+      /* populate data */
+      mapWriter.setPosition(0);
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(10);
+      mapWriter.value().float8().writeFloat8(1.0);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(11);
+      mapWriter.value().float8().writeFloat8(1.1);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(12);
+      mapWriter.value().float8().writeFloat8(1.2);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setPosition(1);
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(13);
+      mapWriter.value().float8().writeFloat8(1.3);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(14);
+      mapWriter.value().float8().writeFloat8(1.4);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setPosition(2);
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(15);
+      mapWriter.value().float8().writeFloat8(1.5);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(16);
+      mapWriter.value().float8().writeFloat8(1.6);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(17);
+      mapWriter.value().float8().writeFloat8(1.7);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(18);
+      mapWriter.value().float8().writeFloat8(1.8);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setPosition(3);
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(19);
+      mapWriter.value().float8().writeFloat8(1.9);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setPosition(4);
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(20);
+      mapWriter.value().float8().writeFloat8(2.0);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(21);
+      mapWriter.value().float8().writeFloat8(2.1);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(22);
+      mapWriter.value().float8().writeFloat8(2.2);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(23);
+      mapWriter.value().float8().writeFloat8(2.3);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapVector.setValueCount(5);
+
+      assertEquals(5, mapVector.getLastSet());
+
+      /* get offset buffer */
+      final ArrowBuf offsetBuffer = mapVector.getOffsetBuffer();
+
+      /* get dataVector */
+      StructVector dataVector = (StructVector) mapVector.getDataVector();
+
+      /* check the vector output */
+      int index = 0;
+      int offset = 0;
+      Map<?, ?> result = null;
+
+      /* index 0 */
+      assertFalse(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(0), Integer.toString(offset));
+
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(10L, getResultKey(result));
+      assertEquals(1.0, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(11L, getResultKey(result));
+      assertEquals(1.1, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(12L, getResultKey(result));
+      assertEquals(1.2, getResultValue(result));
+
+      /* index 1 */
+      index++;
+      assertFalse(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(3), Integer.toString(offset));
+
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(13L, getResultKey(result));
+      assertEquals(1.3, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(14L, getResultKey(result));
+      assertEquals(1.4, getResultValue(result));
+
+      /* index 2 */
+      index++;
+      assertFalse(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(5), Integer.toString(offset));
+
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(15L, getResultKey(result));
+      assertEquals(1.5, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(16L, getResultKey(result));
+      assertEquals(1.6, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(17L, getResultKey(result));
+      assertEquals(1.7, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(18L, getResultKey(result));
+      assertEquals(1.8, getResultValue(result));
+
+      /* index 3 */
+      index++;
+      assertFalse(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(9), Integer.toString(offset));
+
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(19L, getResultKey(result));
+      assertEquals(1.9, getResultValue(result));
+
+      /* index 4 */
+      index++;
+      assertFalse(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(10), Integer.toString(offset));
+
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(20L, getResultKey(result));
+      assertEquals(2.0, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(21L, getResultKey(result));
+      assertEquals(2.1, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(22L, getResultKey(result));
+      assertEquals(2.2, getResultValue(result));
+      offset++;
+      result = (Map<?, ?>) dataVector.getObject(offset);
+      assertEquals(23L, getResultKey(result));
+      assertEquals(2.3, getResultValue(result));
+
+      /* index 5 */
+      index++;
+      assertTrue(mapVector.isNull(index));
+      offset = offsetBuffer.getInt(index * MapVector.OFFSET_WIDTH);
+      assertEquals(Integer.toString(14), Integer.toString(offset));
+
+      /* do split and transfer */
+      try (MapVector toVector = MapVector.empty("toVector", allocator, false)) {
+
+        TransferPair transferPair = mapVector.makeTransferPair(toVector);
+
+        int[][] transferLengths = {{0, 2}, {3, 1}, {4, 1}};
+
+        for (final int[] transferLength : transferLengths) {
+          int start = transferLength[0];
+          int splitLength = transferLength[1];
+
+          int dataLength1 = 0;
+          int dataLength2 = 0;
+
+          int offset1 = 0;
+          int offset2 = 0;
+
+          transferPair.splitAndTransfer(start, splitLength);
+
+          /* get offsetBuffer of toVector */
+          final ArrowBuf toOffsetBuffer = toVector.getOffsetBuffer();
+
+          /* get dataVector of toVector */
+          StructVector dataVector1 = (StructVector) toVector.getDataVector();
+
+          for (int i = 0; i < splitLength; i++) {
+            dataLength1 = offsetBuffer.getInt((start + i + 1) * MapVector.OFFSET_WIDTH) -
+                    offsetBuffer.getInt((start + i) * MapVector.OFFSET_WIDTH);
+            dataLength2 = toOffsetBuffer.getInt((i + 1) * MapVector.OFFSET_WIDTH) -
+                    toOffsetBuffer.getInt(i * MapVector.OFFSET_WIDTH);
+
+            assertEquals("Different data lengths at index: " + i + " and start: " + start,
+                    dataLength1, dataLength2);
+
+            offset1 = offsetBuffer.getInt((start + i) * MapVector.OFFSET_WIDTH);
+            offset2 = toOffsetBuffer.getInt(i * MapVector.OFFSET_WIDTH);
+
+            for (int j = 0; j < dataLength1; j++) {
+              assertEquals("Different data at indexes: " + offset1 + " and " + offset2,
+                      dataVector.getObject(offset1), dataVector1.getObject(offset2));
+
+              offset1++;
+              offset2++;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testMapWithListValue() throws Exception {
+    try (MapVector mapVector = MapVector.empty("sourceVector", allocator, false)) {
+
+      UnionMapWriter mapWriter = mapVector.getWriter();
+      ListWriter valueWriter;
+
+      /* allocate memory */
+      mapWriter.allocate();
+
+      /* the dataVector that backs a listVector will also be a
+       * listVector for this test.
+       */
+
+      /* write one or more maps index 0 */
+      mapWriter.setPosition(0);
+      mapWriter.startMap();
+
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(1);
+      valueWriter = mapWriter.value().list();
+      valueWriter.startList();
+      valueWriter.bigInt().writeBigInt(50);
+      valueWriter.bigInt().writeBigInt(100);
+      valueWriter.bigInt().writeBigInt(200);
+      valueWriter.endList();
+      mapWriter.endEntry();
+
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(2);
+      valueWriter = mapWriter.value().list();
+      valueWriter.startList();
+      valueWriter.bigInt().writeBigInt(75);
+      valueWriter.bigInt().writeBigInt(125);
+      valueWriter.bigInt().writeBigInt(150);
+      valueWriter.bigInt().writeBigInt(175);
+      valueWriter.endList();
+      mapWriter.endEntry();
+
+      mapWriter.endMap();
+
+      /* write one or more maps at index 1 */
+      mapWriter.setPosition(1);
+      mapWriter.startMap();
+
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(3);
+      valueWriter = mapWriter.value().list();
+      valueWriter.startList();
+      valueWriter.bigInt().writeBigInt(10);
+      valueWriter.endList();
+      mapWriter.endEntry();
+
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(4);
+      valueWriter = mapWriter.value().list();
+      valueWriter.startList();
+      valueWriter.bigInt().writeBigInt(15);
+      valueWriter.bigInt().writeBigInt(20);
+      valueWriter.endList();
+      mapWriter.endEntry();
+
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(5);
+      valueWriter = mapWriter.value().list();
+      valueWriter.startList();
+      valueWriter.bigInt().writeBigInt(25);
+      valueWriter.bigInt().writeBigInt(30);
+      valueWriter.bigInt().writeBigInt(35);
+      valueWriter.endList();
+      mapWriter.endEntry();
+
+      mapWriter.endMap();
+
+      assertEquals(2, mapVector.getLastSet());
+
+      mapWriter.setValueCount(2);
+
+      assertEquals(2, mapVector.getValueCount());
+
+      // Get mapVector element at index 0
+      Object result = mapVector.getObject(0);
+      ArrayList<?> resultSet = (ArrayList<?>) result;
+
+      // 2 map entries at index 0
+      assertEquals(2, resultSet.size());
+
+      // First Map entry
+      Map<?, ?> resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(1L, getResultKey(resultStruct));
+      ArrayList<Long> list = (ArrayList<Long>) getResultValue(resultStruct);
+      assertEquals(3, list.size());  // value is a list with 3 elements
+      assertEquals(new Long(50), list.get(0));
+      assertEquals(new Long(100), list.get(1));
+      assertEquals(new Long(200), list.get(2));
+
+      // Second Map entry
+      resultStruct = (Map<?, ?>) resultSet.get(1);
+      list = (ArrayList<Long>) getResultValue(resultStruct);
+      assertEquals(4, list.size());  // value is a list with 4 elements
+      assertEquals(new Long(75), list.get(0));
+      assertEquals(new Long(125), list.get(1));
+      assertEquals(new Long(150), list.get(2));
+      assertEquals(new Long(175), list.get(3));
+
+      // Get mapVector element at index 1
+      result = mapVector.getObject(1);
+      resultSet = (ArrayList<?>) result;
+
+      // First Map entry
+      resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(3L, getResultKey(resultStruct));
+      list = (ArrayList<Long>) getResultValue(resultStruct);
+      assertEquals(1, list.size());  // value is a list with 1 element
+      assertEquals(new Long(10), list.get(0));
+
+      // Second Map entry
+      resultStruct = (Map<?, ?>) resultSet.get(1);
+      assertEquals(4L, getResultKey(resultStruct));
+      list = (ArrayList<Long>) getResultValue(resultStruct);
+      assertEquals(2, list.size());  // value is a list with 1 element
+      assertEquals(new Long(15), list.get(0));
+      assertEquals(new Long(20), list.get(1));
+
+      // Third Map entry
+      resultStruct = (Map<?, ?>) resultSet.get(2);
+      assertEquals(5L, getResultKey(resultStruct));
+      list = (ArrayList<Long>) getResultValue(resultStruct);
+      assertEquals(3, list.size());  // value is a list with 1 element
+      assertEquals(new Long(25), list.get(0));
+      assertEquals(new Long(30), list.get(1));
+      assertEquals(new Long(35), list.get(2));
+
+      /* check underlying bitVector */
+      assertFalse(mapVector.isNull(0));
+      assertFalse(mapVector.isNull(1));
+
+      /* check underlying offsets */
+      final ArrowBuf offsetBuffer = mapVector.getOffsetBuffer();
+
+      /* mapVector has 2 entries at index 0 and 3 entries at index 1 */
+      assertEquals(0, offsetBuffer.getInt(0 * MapVector.OFFSET_WIDTH));
+      assertEquals(2, offsetBuffer.getInt(1 * MapVector.OFFSET_WIDTH));
+      assertEquals(5, offsetBuffer.getInt(2 * MapVector.OFFSET_WIDTH));
+    }
+  }
+
+  @Test
+  public void testClearAndReuse() {
+    try (final MapVector vector = MapVector.empty("map", allocator, false)) {
+      vector.allocateNew();
+      UnionMapWriter mapWriter = vector.getWriter();
+
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(1);
+      mapWriter.value().integer().writeInt(11);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(2);
+      mapWriter.value().integer().writeInt(22);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setValueCount(2);
+
+      Object result = vector.getObject(0);
+      ArrayList<?> resultSet = (ArrayList<?>) result;
+      Map<?, ?> resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(1L, getResultKey(resultStruct));
+      assertEquals(11, getResultValue(resultStruct));
+
+      result = vector.getObject(1);
+      resultSet = (ArrayList<?>) result;
+      resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(2L, getResultKey(resultStruct));
+      assertEquals(22, getResultValue(resultStruct));
+
+      // Clear and release the buffers to trigger a realloc when adding next value
+      vector.clear();
+      mapWriter = new UnionMapWriter(vector);
+
+      // The map vector should reuse a buffer when reallocating the offset buffer
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(3);
+      mapWriter.value().integer().writeInt(33);
+      mapWriter.endEntry();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(4);
+      mapWriter.value().integer().writeInt(44);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.startMap();
+      mapWriter.startEntry();
+      mapWriter.key().bigInt().writeBigInt(5);
+      mapWriter.value().integer().writeInt(55);
+      mapWriter.endEntry();
+      mapWriter.endMap();
+
+      mapWriter.setValueCount(2);
+
+      result = vector.getObject(0);
+      resultSet = (ArrayList<?>) result;
+      resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(3L, getResultKey(resultStruct));
+      assertEquals(33, getResultValue(resultStruct));
+      resultStruct = (Map<?, ?>) resultSet.get(1);
+      assertEquals(4L, getResultKey(resultStruct));
+      assertEquals(44, getResultValue(resultStruct));
+
+      result = vector.getObject(1);
+      resultSet = (ArrayList<?>) result;
+      resultStruct = (Map<?, ?>) resultSet.get(0);
+      assertEquals(5L, getResultKey(resultStruct));
+      assertEquals(55, getResultValue(resultStruct));
+    }
+  }
+}
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 e44996d..83d15d1 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
@@ -19,6 +19,7 @@ package org.apache.arrow.vector.ipc;
 
 import static org.apache.arrow.vector.TestUtils.newVarCharVector;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -35,6 +36,7 @@ import java.util.List;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.memory.RootAllocator;
 import org.apache.arrow.util.Collections2;
+import org.apache.arrow.vector.BigIntVector;
 import org.apache.arrow.vector.DateMilliVector;
 import org.apache.arrow.vector.DecimalVector;
 import org.apache.arrow.vector.FieldVector;
@@ -48,9 +50,12 @@ import org.apache.arrow.vector.VarBinaryVector;
 import org.apache.arrow.vector.VarCharVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
 import org.apache.arrow.vector.complex.StructVector;
 import org.apache.arrow.vector.complex.impl.ComplexWriterImpl;
 import org.apache.arrow.vector.complex.impl.UnionListWriter;
+import org.apache.arrow.vector.complex.impl.UnionMapReader;
+import org.apache.arrow.vector.complex.impl.UnionMapWriter;
 import org.apache.arrow.vector.complex.reader.FieldReader;
 import org.apache.arrow.vector.complex.writer.BaseWriter.ComplexWriter;
 import org.apache.arrow.vector.complex.writer.BaseWriter.ListWriter;
@@ -75,6 +80,7 @@ import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.JsonStringArrayList;
 import org.apache.arrow.vector.util.Text;
 import org.junit.After;
 import org.junit.Assert;
@@ -683,4 +689,142 @@ public class BaseFileTest {
     assertEquals(vector.get(1), 1);
     assertEquals(vector.get(2), 2);
   }
+
+  protected VectorSchemaRoot writeMapData(BufferAllocator bufferAllocator) {
+    MapVector mapVector = MapVector.empty("map", bufferAllocator, false);
+    MapVector sortedMapVector = MapVector.empty("mapSorted", bufferAllocator, true);
+    mapVector.allocateNew();
+    sortedMapVector.allocateNew();
+    UnionMapWriter mapWriter = mapVector.getWriter();
+    UnionMapWriter sortedMapWriter = sortedMapVector.getWriter();
+
+    final int count = 10;
+    for (int i = 0; i < count; i++) {
+      // Write mapVector with NULL values
+      // i == 1 is a NULL
+      if (i != 1) {
+        mapWriter.setPosition(i);
+        mapWriter.startMap();
+        // i == 3 is an empty map
+        if (i != 3) {
+          for (int j = 0; j < i + 1; j++) {
+            mapWriter.startEntry();
+            mapWriter.key().bigInt().writeBigInt(j);
+            // i == 5 maps to a NULL value
+            if (i != 5) {
+              mapWriter.value().integer().writeInt(j);
+            }
+            mapWriter.endEntry();
+          }
+        }
+        mapWriter.endMap();
+      }
+      // Write sortedMapVector
+      sortedMapWriter.setPosition(i);
+      sortedMapWriter.startMap();
+      for (int j = 0; j < i + 1; j++) {
+        sortedMapWriter.startEntry();
+        sortedMapWriter.key().bigInt().writeBigInt(j);
+        sortedMapWriter.value().integer().writeInt(j);
+        sortedMapWriter.endEntry();
+      }
+      sortedMapWriter.endMap();
+    }
+    mapWriter.setValueCount(COUNT);
+    sortedMapWriter.setValueCount(COUNT);
+
+    List<Field> fields = Collections2.asImmutableList(mapVector.getField(), sortedMapVector.getField());
+    List<FieldVector> vectors = Collections2.asImmutableList(mapVector, sortedMapVector);
+    return new VectorSchemaRoot(fields, vectors, count);
+  }
+
+  protected void validateMapData(VectorSchemaRoot root) {
+    MapVector mapVector = (MapVector) root.getVector("map");
+    MapVector sortedMapVector = (MapVector) root.getVector("mapSorted");
+
+    final int count = 10;
+    Assert.assertEquals(count, root.getRowCount());
+
+    UnionMapReader mapReader = new UnionMapReader(mapVector);
+    UnionMapReader sortedMapReader = new UnionMapReader(sortedMapVector);
+    for (int i = 0; i < count; i++) {
+      // Read mapVector with NULL values
+      mapReader.setPosition(i);
+      if (i == 1) {
+        assertFalse(mapReader.isSet());
+      } else {
+        if (i == 3) {
+          JsonStringArrayList<?> result = (JsonStringArrayList<?>) mapReader.readObject();
+          assertTrue(result.isEmpty());
+        } else {
+          for (int j = 0; j < i + 1; j++) {
+            mapReader.next();
+            assertEquals(j, mapReader.key().readLong().longValue());
+            if (i == 5) {
+              assertFalse(mapReader.value().isSet());
+            } else {
+              assertEquals(j, mapReader.value().readInteger().intValue());
+            }
+          }
+        }
+      }
+      // Read sortedMapVector
+      sortedMapReader.setPosition(i);
+      for (int j = 0; j < i + 1; j++) {
+        sortedMapReader.next();
+        assertEquals(j, sortedMapReader.key().readLong().longValue());
+        assertEquals(j, sortedMapReader.value().readInteger().intValue());
+      }
+    }
+  }
+
+  protected VectorSchemaRoot writeListAsMapData(BufferAllocator bufferAllocator) {
+    ListVector mapEntryList = ListVector.empty("entryList", bufferAllocator);
+    FieldType mapEntryType = new FieldType(false, ArrowType.Struct.INSTANCE, null, null);
+    StructVector mapEntryData = new StructVector("entryData", bufferAllocator, mapEntryType, null);
+    mapEntryData.addOrGet("myKey", new FieldType(false, new ArrowType.Int(64, true), null), BigIntVector.class);
+    mapEntryData.addOrGet("myValue", FieldType.nullable(new ArrowType.Int(32, true)), IntVector.class);
+    mapEntryList.initializeChildrenFromFields(Collections2.asImmutableList(mapEntryData.getField()));
+    UnionListWriter entryWriter = mapEntryList.getWriter();
+    entryWriter.allocate();
+
+    final int count = 10;
+    for (int i = 0; i < count; i++) {
+      entryWriter.setPosition(i);
+      entryWriter.startList();
+      for (int j = 0; j < i + 1; j++) {
+        entryWriter.struct().start();
+        entryWriter.struct().bigInt("myKey").writeBigInt(j);
+        entryWriter.struct().integer("myValue").writeInt(j);
+        entryWriter.struct().end();
+      }
+      entryWriter.endList();
+    }
+    entryWriter.setValueCount(COUNT);
+
+    MapVector mapVector = MapVector.empty("map", bufferAllocator, false);
+    mapEntryList.makeTransferPair(mapVector).transfer();
+
+    List<Field> fields = Collections2.asImmutableList(mapVector.getField());
+    List<FieldVector> vectors = Collections2.asImmutableList(mapVector);
+    return new VectorSchemaRoot(fields, vectors, count);
+  }
+
+  protected void validateListAsMapData(VectorSchemaRoot root) {
+    MapVector sortedMapVector = (MapVector) root.getVector("map");
+
+    final int count = 10;
+    Assert.assertEquals(count, root.getRowCount());
+
+    UnionMapReader sortedMapReader = new UnionMapReader(sortedMapVector);
+    sortedMapReader.setKeyValueNames("myKey", "myValue");
+    for (int i = 0; i < count; i++) {
+      sortedMapReader.setPosition(i);
+      for (int j = 0; j < i + 1; j++) {
+        sortedMapReader.next();
+        assertEquals(j, sortedMapReader.key().readLong().longValue());
+        assertEquals(j, sortedMapReader.value().readInteger().intValue());
+      }
+    }
+  }
 }
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 604a244..db51b76 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
@@ -749,6 +749,98 @@ public class TestArrowFile extends BaseFileTest {
     }
   }
 
+  @Test
+  public void testWriteReadMapVector() throws IOException {
+    File file = new File("target/mytest_map.arrow");
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+
+    // write
+    try (BufferAllocator originalVectorAllocator =
+           allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)) {
+
+      try (VectorSchemaRoot root = writeMapData(originalVectorAllocator);
+           FileOutputStream fileOutputStream = new FileOutputStream(file);
+           ArrowFileWriter fileWriter = new ArrowFileWriter(root, null, fileOutputStream.getChannel());
+           ArrowStreamWriter streamWriter = new ArrowStreamWriter(root, null, stream)) {
+        LOGGER.debug("writing schema: " + root.getSchema());
+        fileWriter.start();
+        streamWriter.start();
+        fileWriter.writeBatch();
+        streamWriter.writeBatch();
+        fileWriter.end();
+        streamWriter.end();
+      }
+    }
+
+    // read from file
+    try (BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+         FileInputStream fileInputStream = new FileInputStream(file);
+         ArrowFileReader arrowReader = new ArrowFileReader(fileInputStream.getChannel(), readerAllocator)) {
+      VectorSchemaRoot root = arrowReader.getVectorSchemaRoot();
+      Schema schema = root.getSchema();
+      LOGGER.debug("reading schema: " + schema);
+      Assert.assertTrue(arrowReader.loadNextBatch());
+      validateMapData(root);
+    }
+
+    // Read from stream
+    try (BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+         ByteArrayInputStream input = new ByteArrayInputStream(stream.toByteArray());
+         ArrowStreamReader arrowReader = new ArrowStreamReader(input, readerAllocator)) {
+      VectorSchemaRoot root = arrowReader.getVectorSchemaRoot();
+      Schema schema = root.getSchema();
+      LOGGER.debug("reading schema: " + schema);
+      Assert.assertTrue(arrowReader.loadNextBatch());
+      validateMapData(root);
+    }
+  }
+
+  @Test
+  public void testWriteReadListAsMap() throws IOException {
+    File file = new File("target/mytest_list_as_map.arrow");
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+
+    // write
+    try (BufferAllocator originalVectorAllocator =
+           allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)) {
+
+      try (VectorSchemaRoot root = writeListAsMapData(originalVectorAllocator);
+           FileOutputStream fileOutputStream = new FileOutputStream(file);
+           ArrowFileWriter fileWriter = new ArrowFileWriter(root, null, fileOutputStream.getChannel());
+           ArrowStreamWriter streamWriter = new ArrowStreamWriter(root, null, stream)) {
+        LOGGER.debug("writing schema: " + root.getSchema());
+        fileWriter.start();
+        streamWriter.start();
+        fileWriter.writeBatch();
+        streamWriter.writeBatch();
+        fileWriter.end();
+        streamWriter.end();
+      }
+    }
+
+    // read from file
+    try (BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+         FileInputStream fileInputStream = new FileInputStream(file);
+         ArrowFileReader arrowReader = new ArrowFileReader(fileInputStream.getChannel(), readerAllocator)) {
+      VectorSchemaRoot root = arrowReader.getVectorSchemaRoot();
+      Schema schema = root.getSchema();
+      LOGGER.debug("reading schema: " + schema);
+      Assert.assertTrue(arrowReader.loadNextBatch());
+      validateListAsMapData(root);
+    }
+
+    // Read from stream
+    try (BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+         ByteArrayInputStream input = new ByteArrayInputStream(stream.toByteArray());
+         ArrowStreamReader arrowReader = new ArrowStreamReader(input, readerAllocator)) {
+      VectorSchemaRoot root = arrowReader.getVectorSchemaRoot();
+      Schema schema = root.getSchema();
+      LOGGER.debug("reading schema: " + schema);
+      Assert.assertTrue(arrowReader.loadNextBatch());
+      validateListAsMapData(root);
+    }
+  }
+
   /**
    * Writes the contents of parents to file. If outStream is non-null, also writes it
    * to outStream in the streaming serialized format.
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 67ac375..b482725 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
@@ -369,4 +369,36 @@ public class TestJSONFile extends BaseFileTest {
       reader.close();
     }
   }
+
+  @Test
+  public void testWriteReadMapJSON() throws IOException {
+    File file = new File("target/mytest_map.json");
+
+    // write
+    try (
+      BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)
+    ) {
+
+      try (VectorSchemaRoot root = writeMapData(vectorAllocator)) {
+        printVectors(root.getFieldVectors());
+        validateMapData(root);
+        writeJSON(file, root, null);
+      }
+    }
+
+    // read
+    try (
+      BufferAllocator readerAllocator = allocator.newChildAllocator("reader", 0, Integer.MAX_VALUE);
+    ) {
+      JsonFileReader reader = new JsonFileReader(file, readerAllocator);
+      Schema schema = reader.start();
+      LOGGER.debug("reading schema: " + schema);
+
+      // initialize vectors
+      try (VectorSchemaRoot root = reader.read();) {
+        validateMapData(root);
+      }
+      reader.close();
+    }
+  }
 }