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/07/04 22:19:05 UTC

arrow git commit: ARROW-693: [Java] Add dictionary support to JSON reader and writer

Repository: arrow
Updated Branches:
  refs/heads/master cbbd04bfa -> 7d86c28e0


ARROW-693: [Java] Add dictionary support to JSON reader and writer

Added support for Dictionary encoded vectors for reading and writing JSON.  Also added validation to compare DictionaryEncodings and DictionaryBatches.

New unit test for round-trip of flat dictionary type.

Author: Bryan Cutler <cu...@gmail.com>

Closes #752 from BryanCutler/java-json-dict-support-ARROW-693 and squashes the following commits:

9f9eae92 [Bryan Cutler] Adding back in Dictionary.equals, this should be a separate issue
923c3d02 [Bryan Cutler] Changed integration.py script to produce correct Field name for dictionary encoded fields, enabled dictionary_generation
90c59090 [Bryan Cutler] Removed equals from Dictionary since it does not compare the FieldVector properly, fixed Integration to check FieldVectors of Dictionary
e60b7f18 [Bryan Cutler] Merge remote-tracking branch 'upstream/master' into java-json-dict-support-ARROW-693
b8fb5430 [Bryan Cutler] Removed need for dictionary fields to have a specific name, now name is not checked
577ec665 [Bryan Cutler] Merge remote-tracking branch 'upstream/master' into java-json-dict-support-ARROW-693
7c56c0ca [Bryan Cutler] fixed up Validator error messages
552c26cd [Bryan Cutler] expanded dictionary test to use multiple dictionaries, and 2 fields using same dict
a7c67447 [Bryan Cutler] Tests working with nested dictionary
8ab4dd38 [Bryan Cutler] cleaned up validator for integration tests
8a323cf1 [Bryan Cutler] cleanup of unused imports
22e849ae [Bryan Cutler] Removed WIP on DictionaryWriter for now
933a78c6 [Bryan Cutler] JsonFileReader and JsonFileWriter are working for Java, tested with flat Dictionary type


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/7d86c28e
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/7d86c28e
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/7d86c28e

Branch: refs/heads/master
Commit: 7d86c28e09f6c402ea1dbb9e67dcee2507b62986
Parents: cbbd04b
Author: Bryan Cutler <cu...@gmail.com>
Authored: Tue Jul 4 18:19:00 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Tue Jul 4 18:19:00 2017 -0400

----------------------------------------------------------------------
 integration/integration_test.py                 |   2 +
 .../org/apache/arrow/tools/Integration.java     |  27 ++-
 .../vector/dictionary/DictionaryProvider.java   |   5 +
 .../apache/arrow/vector/file/ArrowReader.java   |  52 +---
 .../apache/arrow/vector/file/ArrowWriter.java   |  62 ++---
 .../arrow/vector/file/json/JsonFileReader.java  |  78 +++++-
 .../arrow/vector/file/json/JsonFileWriter.java  |  54 ++++-
 .../vector/types/pojo/DictionaryEncoding.java   |  10 +-
 .../arrow/vector/util/DictionaryUtility.java    | 113 +++++++++
 .../org/apache/arrow/vector/util/Validator.java | 237 ++++++++++--------
 .../apache/arrow/vector/file/BaseFileTest.java  | 238 +++++++++++++++++++
 .../apache/arrow/vector/file/TestArrowFile.java | 153 ++----------
 .../arrow/vector/file/json/TestJSONFile.java    |  93 +++++++-
 13 files changed, 790 insertions(+), 334 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/integration/integration_test.py
----------------------------------------------------------------------
diff --git a/integration/integration_test.py b/integration/integration_test.py
index 7609966..215ba58 100644
--- a/integration/integration_test.py
+++ b/integration/integration_test.py
@@ -581,6 +581,8 @@ class DictionaryType(DataType):
         return self.index_type._get_type_layout()
 
     def generate_column(self, size, name=None):
+        if name is None:
+            name = self.name
         return self.index_type.generate_range(size, 0, len(self.dictionary),
                                               name=name)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
----------------------------------------------------------------------
diff --git a/java/tools/src/main/java/org/apache/arrow/tools/Integration.java b/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
index e8266d5..7d4c86f 100644
--- a/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
+++ b/java/tools/src/main/java/org/apache/arrow/tools/Integration.java
@@ -27,6 +27,8 @@ import org.apache.arrow.vector.file.ArrowFileReader;
 import org.apache.arrow.vector.file.ArrowFileWriter;
 import org.apache.arrow.vector.file.json.JsonFileReader;
 import org.apache.arrow.vector.file.json.JsonFileWriter;
+import org.apache.arrow.vector.types.pojo.DictionaryEncoding;
+import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
 import org.apache.arrow.vector.util.Validator;
 import org.apache.commons.cli.CommandLine;
@@ -41,6 +43,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
@@ -90,11 +93,21 @@ public class Integration {
     return f;
   }
 
+  static void extractDictionaryEncodings(List<Field> fields, List<DictionaryEncoding> encodings) {
+    for (Field field: fields) {
+      DictionaryEncoding encoding = field.getDictionary();
+      if (encoding != null) {
+        encodings.add(encoding);
+      }
+
+      extractDictionaryEncodings(field.getChildren(), encodings);
+    }
+  }
+
   void run(String[] args) throws ParseException, IOException {
     CommandLineParser parser = new PosixParser();
     CommandLine cmd = parser.parse(options, args, false);
 
-
     Command command = toCommand(cmd.getOptionValue("command"));
     File arrowFile = validateFile("arrow", cmd.getOptionValue("arrow"), command.arrowExists);
     File jsonFile = validateFile("json", cmd.getOptionValue("json"), command.jsonExists);
@@ -124,7 +137,7 @@ public class Integration {
           LOGGER.debug("Found schema: " + schema);
           try (JsonFileWriter writer = new JsonFileWriter(jsonFile, JsonFileWriter.config()
               .pretty(true))) {
-            writer.start(schema);
+            writer.start(schema, arrowReader);
             for (ArrowBlock rbBlock : arrowReader.getRecordBlocks()) {
               if (!arrowReader.loadRecordBatch(rbBlock)) {
                 throw new IOException("Expected to load record batch");
@@ -147,7 +160,7 @@ public class Integration {
           try (FileOutputStream fileOutputStream = new FileOutputStream(arrowFile);
                VectorSchemaRoot root = VectorSchemaRoot.create(schema, allocator);
                // TODO json dictionaries
-               ArrowFileWriter arrowWriter = new ArrowFileWriter(root, null, fileOutputStream
+               ArrowFileWriter arrowWriter = new ArrowFileWriter(root, reader, fileOutputStream
                    .getChannel())) {
             arrowWriter.start();
             while (reader.read(root)) {
@@ -189,6 +202,14 @@ public class Integration {
             jsonRoot.close();
             totalBatches++;
           }
+
+          // Validate Dictionaries after ArrowFileReader has read batches
+          List<DictionaryEncoding> encodingsJson = new ArrayList<>();
+          extractDictionaryEncodings(jsonSchema.getFields(), encodingsJson);
+          List<DictionaryEncoding> encodingsArrow = new ArrayList<>();
+          extractDictionaryEncodings(arrowSchema.getFields(), encodingsArrow);
+          Validator.compareDictionaries(encodingsJson, encodingsArrow, jsonReader, arrowReader);
+
           boolean hasMoreJSON = jsonRoot != null;
           boolean hasMoreArrow = iterator.hasNext();
           if (hasMoreJSON || hasMoreArrow) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryProvider.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryProvider.java b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryProvider.java
index 63fde25..87516c9 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryProvider.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/dictionary/DictionaryProvider.java
@@ -19,6 +19,7 @@ package org.apache.arrow.vector.dictionary;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 public interface DictionaryProvider {
 
@@ -39,6 +40,10 @@ public interface DictionaryProvider {
       map.put(dictionary.getEncoding().getId(), dictionary);
     }
 
+    public final Set<Long> getDictionaryIds() {
+      return map.keySet();
+    }
+
     @Override
     public Dictionary lookup(long id) {
       return map.get(id);

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowReader.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowReader.java b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowReader.java
index b331ea7..f6b1041 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowReader.java
@@ -36,12 +36,9 @@ import org.apache.arrow.vector.schema.ArrowDictionaryBatch;
 import org.apache.arrow.vector.schema.ArrowMessage;
 import org.apache.arrow.vector.schema.ArrowMessage.ArrowMessageVisitor;
 import org.apache.arrow.vector.schema.ArrowRecordBatch;
-import org.apache.arrow.vector.types.pojo.ArrowType;
-import org.apache.arrow.vector.types.pojo.ArrowType.Int;
-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.types.pojo.Schema;
+import org.apache.arrow.vector.util.DictionaryUtility;
 
 public abstract class ArrowReader<T extends ReadChannel> implements DictionaryProvider, AutoCloseable {
 
@@ -83,11 +80,11 @@ public abstract class ArrowReader<T extends ReadChannel> implements DictionaryPr
 
   @Override
   public Dictionary lookup(long id) {
-    if (initialized) {
-      return dictionaries.get(id);
-    } else {
-      return null;
+    if (!initialized) {
+      throw new IllegalStateException("Unable to lookup until reader has been initialized");
     }
+
+    return dictionaries.get(id);
   }
 
   // Returns true if a batch was read, false on EOS
@@ -155,8 +152,9 @@ public abstract class ArrowReader<T extends ReadChannel> implements DictionaryPr
     List<FieldVector> vectors = new ArrayList<>();
     Map<Long, Dictionary> dictionaries = new HashMap<>();
 
+    // Convert fields with dictionaries to have the index type
     for (Field field: originalSchema.getFields()) {
-      Field updated = toMemoryFormat(field, dictionaries);
+      Field updated = DictionaryUtility.toMemoryFormat(field, allocator, dictionaries);
       fields.add(updated);
       vectors.add(updated.createVector(allocator));
     }
@@ -167,42 +165,6 @@ public abstract class ArrowReader<T extends ReadChannel> implements DictionaryPr
     this.dictionaries = Collections.unmodifiableMap(dictionaries);
   }
 
-  // in the message format, fields have the dictionary type
-  // in the memory format, they have the index type
-  private Field toMemoryFormat(Field field, Map<Long, Dictionary> dictionaries) {
-    DictionaryEncoding encoding = field.getDictionary();
-    List<Field> children = field.getChildren();
-
-    if (encoding == null && children.isEmpty()) {
-      return field;
-    }
-
-    List<Field> updatedChildren = new ArrayList<>(children.size());
-    for (Field child: children) {
-      updatedChildren.add(toMemoryFormat(child, dictionaries));
-    }
-
-    ArrowType type;
-    if (encoding == null) {
-      type = field.getType();
-    } else {
-      // re-type the field for in-memory format
-      type = encoding.getIndexType();
-      if (type == null) {
-        type = new Int(32, true);
-      }
-      // get existing or create dictionary vector
-      if (!dictionaries.containsKey(encoding.getId())) {
-        // create a new dictionary vector for the values
-        Field dictionaryField = new Field(field.getName(), new FieldType(field.isNullable(), field.getType(), null, null), children);
-        FieldVector dictionaryVector = dictionaryField.createVector(allocator);
-        dictionaries.put(encoding.getId(), new Dictionary(dictionaryVector, encoding));
-      }
-    }
-
-    return new Field(field.getName(), new FieldType(field.isNullable(), type, encoding, field.getMetadata()), updatedChildren);
-  }
-
   private void load(ArrowDictionaryBatch dictionaryBatch) {
     long id = dictionaryBatch.getDictionaryId();
     Dictionary dictionary = dictionaries.get(id);

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
index 4abaed0..3b37071 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/ArrowWriter.java
@@ -20,10 +20,9 @@ package org.apache.arrow.vector.file;
 import java.io.IOException;
 import java.nio.channels.WritableByteChannel;
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
 
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
@@ -33,11 +32,9 @@ import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.schema.ArrowDictionaryBatch;
 import org.apache.arrow.vector.schema.ArrowRecordBatch;
 import org.apache.arrow.vector.stream.MessageSerializer;
-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.types.pojo.Schema;
+import org.apache.arrow.vector.util.DictionaryUtility;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,53 +69,26 @@ public abstract class ArrowWriter implements AutoCloseable {
     this.out = new WriteChannel(out);
 
     List<Field> fields = new ArrayList<>(root.getSchema().getFields().size());
-    Map<Long, ArrowDictionaryBatch> dictionaryBatches = new HashMap<>();
+    Set<Long> dictionaryIdsUsed = new HashSet<>();
 
+    // Convert fields with dictionaries to have dictionary type
     for (Field field: root.getSchema().getFields()) {
-      fields.add(toMessageFormat(field, provider, dictionaryBatches));
+      fields.add(DictionaryUtility.toMessageFormat(field, provider, dictionaryIdsUsed));
     }
 
-    this.schema = new Schema(fields, root.getSchema().getCustomMetadata());
-    this.dictionaries = Collections.unmodifiableList(new ArrayList<>(dictionaryBatches.values()));
-  }
-
-  // in the message format, fields have the dictionary type
-  // in the memory format, they have the index type
-  private Field toMessageFormat(Field field, DictionaryProvider provider, Map<Long, ArrowDictionaryBatch> batches) {
-    DictionaryEncoding encoding = field.getDictionary();
-    List<Field> children = field.getChildren();
-
-    if (encoding == null && children.isEmpty()) {
-      return field;
-    }
-
-    List<Field> updatedChildren = new ArrayList<>(children.size());
-    for (Field child: children) {
-      updatedChildren.add(toMessageFormat(child, provider, batches));
-    }
-
-    ArrowType type;
-    if (encoding == null) {
-      type = field.getType();
-    } else {
-      long id = encoding.getId();
+    // Create a record batch for each dictionary
+    this.dictionaries = new ArrayList<>(dictionaryIdsUsed.size());
+    for (long id: dictionaryIdsUsed) {
       Dictionary dictionary = provider.lookup(id);
-      if (dictionary == null) {
-        throw new IllegalArgumentException("Could not find dictionary with ID " + id);
-      }
-      type = dictionary.getVectorType();
-
-      if (!batches.containsKey(id)) {
-        FieldVector vector = dictionary.getVector();
-        int count = vector.getAccessor().getValueCount();
-        VectorSchemaRoot root = new VectorSchemaRoot(ImmutableList.of(field), ImmutableList.of(vector), count);
-        VectorUnloader unloader = new VectorUnloader(root);
-        ArrowRecordBatch batch = unloader.getRecordBatch();
-        batches.put(id, new ArrowDictionaryBatch(id, batch));
-      }
+      FieldVector vector = dictionary.getVector();
+      int count = vector.getAccessor().getValueCount();
+      VectorSchemaRoot dictRoot = new VectorSchemaRoot(ImmutableList.of(vector.getField()), ImmutableList.of(vector), count);
+      VectorUnloader unloader = new VectorUnloader(dictRoot);
+      ArrowRecordBatch batch = unloader.getRecordBatch();
+      this.dictionaries.add(new ArrowDictionaryBatch(id, batch));
     }
 
-    return new Field(field.getName(), new FieldType(field.isNullable(), type, encoding, field.getMetadata()), updatedChildren);
+    this.schema = new Schema(fields, root.getSchema().getCustomMetadata());
   }
 
   public void start() throws IOException {

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
index 3ef1484..364d273 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileReader.java
@@ -26,8 +26,12 @@ import static org.apache.arrow.vector.schema.ArrowVectorType.OFFSET;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.BigIntVector;
 import org.apache.arrow.vector.BitVector;
@@ -62,9 +66,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.NullableMapVector;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.schema.ArrowVectorType;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.arrow.vector.util.DictionaryUtility;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.binary.Hex;
 
@@ -74,11 +81,13 @@ import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.databind.MappingJsonFactory;
 import com.google.common.base.Objects;
 
-public class JsonFileReader implements AutoCloseable {
+public class JsonFileReader implements AutoCloseable, DictionaryProvider {
   private final File inputFile;
   private final JsonParser parser;
   private final BufferAllocator allocator;
   private Schema schema;
+  private Map<Long, Dictionary> dictionaries;
+  private Boolean started = false;
 
   public JsonFileReader(File inputFile, BufferAllocator allocator) throws JsonParseException, IOException {
     super();
@@ -88,13 +97,68 @@ public class JsonFileReader implements AutoCloseable {
     this.parser = jsonFactory.createParser(inputFile);
   }
 
+  @Override
+  public Dictionary lookup(long id) {
+    if (!started) {
+      throw new IllegalStateException("Unable to lookup until after read() has started");
+    }
+
+    return dictionaries.get(id);
+  }
+
   public Schema start() throws JsonParseException, IOException {
     readToken(START_OBJECT);
     {
-      this.schema = readNextField("schema", Schema.class);
+      Schema originalSchema = readNextField("schema", Schema.class);
+      List<Field> fields = new ArrayList<>();
+      dictionaries = new HashMap<>();
+
+      // Convert fields with dictionaries to have the index type
+      for (Field field: originalSchema.getFields()) {
+        fields.add(DictionaryUtility.toMemoryFormat(field, allocator, dictionaries));
+      }
+      this.schema = new Schema(fields, originalSchema.getCustomMetadata());
+
+      if (!dictionaries.isEmpty()) {
+        nextFieldIs("dictionaries");
+        readDictionaryBatches();
+      }
+
       nextFieldIs("batches");
       readToken(START_ARRAY);
-      return schema;
+      started = true;
+      return this.schema;
+    }
+  }
+
+  private void readDictionaryBatches() throws JsonParseException, IOException {
+    readToken(START_ARRAY);
+    JsonToken token = parser.nextToken();
+    boolean haveDictionaryBatch = token == START_OBJECT;
+    while (haveDictionaryBatch) {
+
+      // Lookup what dictionary for the batch about to be read
+      long id = readNextField("id", Long.class);
+      Dictionary dict = dictionaries.get(id);
+      if (dict == null) {
+        throw new IllegalArgumentException("Dictionary with id: " + id + " missing encoding from schema Field");
+      }
+
+      // Read the dictionary record batch
+      nextFieldIs("data");
+      FieldVector vector = dict.getVector();
+      List<Field> fields = ImmutableList.of(vector.getField());
+      List<FieldVector> vectors = ImmutableList.of(vector);
+      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getAccessor().getValueCount());
+      read(root);
+
+      readToken(END_OBJECT);
+      token = parser.nextToken();
+      haveDictionaryBatch = token == START_OBJECT;
+    }
+
+    if (token != END_ARRAY) {
+      throw new IllegalArgumentException("Invalid token: " + token + " expected end of array at " + parser.getTokenLocation());
     }
   }
 
@@ -107,7 +171,7 @@ public class JsonFileReader implements AutoCloseable {
         nextFieldIs("columns");
         readToken(START_ARRAY);
         {
-          for (Field field : schema.getFields()) {
+          for (Field field : root.getSchema().getFields()) {
             FieldVector vector = root.getVector(field.getName());
             readVector(field, vector);
           }
@@ -158,8 +222,9 @@ public class JsonFileReader implements AutoCloseable {
     }
     readToken(START_OBJECT);
     {
+      // If currently reading dictionaries, field name is not important so don't check
       String name = readNextField("name", String.class);
-      if (!Objects.equal(field.getName(), name)) {
+      if (started && !Objects.equal(field.getName(), name)) {
         throw new IllegalArgumentException("Expected field " + field.getName() + " but got " + name);
       }
       int count = readNextField("count", Integer.class);
@@ -302,6 +367,9 @@ public class JsonFileReader implements AutoCloseable {
   @Override
   public void close() throws IOException {
     parser.close();
+    for (Dictionary dictionary: dictionaries.values()) {
+      dictionary.getVector().close();
+    }
   }
 
   private <T> T readNextField(String expectedFieldName, Class<T> c) throws IOException, JsonParseException {

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
index d86b3de..befa922 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/file/json/JsonFileWriter.java
@@ -19,8 +19,12 @@ package org.apache.arrow.vector.file.json;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.arrow.vector.BitVector;
 import org.apache.arrow.vector.BufferBacked;
 import org.apache.arrow.vector.DateDayVector;
@@ -38,6 +42,8 @@ import org.apache.arrow.vector.ValueVector;
 import org.apache.arrow.vector.ValueVector.Accessor;
 import org.apache.arrow.vector.VarBinaryVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.schema.ArrowVectorType;
 import org.apache.arrow.vector.types.pojo.Field;
 import org.apache.arrow.vector.types.pojo.Schema;
@@ -47,6 +53,7 @@ import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
 import com.fasterxml.jackson.core.util.DefaultPrettyPrinter.NopIndenter;
 import com.fasterxml.jackson.databind.MappingJsonFactory;
+import org.apache.arrow.vector.util.DictionaryUtility;
 import org.apache.commons.codec.binary.Hex;
 
 public class JsonFileWriter implements AutoCloseable {
@@ -85,22 +92,61 @@ public class JsonFileWriter implements AutoCloseable {
     }
   }
 
-  public void start(Schema schema) throws IOException {
-    this.schema = schema;
+  public void start(Schema schema, DictionaryProvider provider) throws IOException {
+    List<Field> fields = new ArrayList<>(schema.getFields().size());
+    Set<Long> dictionaryIdsUsed = new HashSet<>();
+    this.schema = schema;  // Store original Schema to ensure batches written match
+
+    // Convert fields with dictionaries to have dictionary type
+    for (Field field: schema.getFields()) {
+      fields.add(DictionaryUtility.toMessageFormat(field, provider, dictionaryIdsUsed));
+    }
+    Schema updatedSchema = new Schema(fields, schema.getCustomMetadata());
+
     generator.writeStartObject();
-    generator.writeObjectField("schema", schema);
+    generator.writeObjectField("schema", updatedSchema);
+
+    // Write all dictionaries that were used
+    if (!dictionaryIdsUsed.isEmpty()) {
+      writeDictionaryBatches(generator, dictionaryIdsUsed, provider);
+    }
+
+    // Start writing of record batches
     generator.writeArrayFieldStart("batches");
   }
 
+  private void writeDictionaryBatches(JsonGenerator generator, Set<Long> dictionaryIdsUsed, DictionaryProvider provider) throws IOException {
+    generator.writeArrayFieldStart("dictionaries");
+    for (Long id: dictionaryIdsUsed) {
+      generator.writeStartObject();
+      generator.writeObjectField("id", id);
+
+      generator.writeFieldName("data");
+      Dictionary dictionary = provider.lookup(id);
+      FieldVector vector = dictionary.getVector();
+      List<Field> fields = ImmutableList.of(vector.getField());
+      List<FieldVector> vectors = ImmutableList.of(vector);
+      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, vector.getAccessor().getValueCount());
+      writeBatch(root);
+
+      generator.writeEndObject();
+    }
+    generator.writeEndArray();
+  }
+
   public void write(VectorSchemaRoot recordBatch) throws IOException {
     if (!recordBatch.getSchema().equals(schema)) {
       throw new IllegalArgumentException("record batches must have the same schema: " + schema);
     }
+    writeBatch(recordBatch);
+  }
+
+  private void writeBatch(VectorSchemaRoot recordBatch) throws IOException {
     generator.writeStartObject();
     {
       generator.writeObjectField("count", recordBatch.getRowCount());
       generator.writeArrayFieldStart("columns");
-      for (Field field : schema.getFields()) {
+      for (Field field : recordBatch.getSchema().getFields()) {
         FieldVector vector = recordBatch.getVector(field.getName());
         writeVector(field, vector);
       }

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/DictionaryEncoding.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/DictionaryEncoding.java b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/DictionaryEncoding.java
index 32568d3..8a0224d 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/DictionaryEncoding.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/DictionaryEncoding.java
@@ -20,6 +20,9 @@ package org.apache.arrow.vector.types.pojo;
 
 import java.util.Objects;
 
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonGetter;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.arrow.vector.types.pojo.ArrowType.Int;
 
 public class DictionaryEncoding {
@@ -28,7 +31,11 @@ public class DictionaryEncoding {
   private final boolean ordered;
   private final Int indexType;
 
-  public DictionaryEncoding(long id, boolean ordered, Int indexType) {
+  @JsonCreator
+  public DictionaryEncoding(
+      @JsonProperty("id") long id,
+      @JsonProperty("isOrdered") boolean ordered,
+      @JsonProperty("indexType") Int indexType) {
     this.id = id;
     this.ordered = ordered;
     this.indexType = indexType == null ? new Int(32, true) : indexType;
@@ -38,6 +45,7 @@ public class DictionaryEncoding {
     return id;
   }
 
+  @JsonGetter("isOrdered")
   public boolean isOrdered() {
     return ordered;
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/util/DictionaryUtility.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/DictionaryUtility.java b/java/vector/src/main/java/org/apache/arrow/vector/util/DictionaryUtility.java
new file mode 100644
index 0000000..4108dc4
--- /dev/null
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/DictionaryUtility.java
@@ -0,0 +1,113 @@
+/**
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
+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;
+
+
+public class DictionaryUtility {
+
+  /**
+   * Convert field and child fields that have a dictionary encoding to message format, so fields
+   * have the dictionary type
+   *
+   * NOTE: in the message format, fields have the dictionary type
+   *       in the memory format, they have the index type
+   */
+  public static Field toMessageFormat(Field field, DictionaryProvider provider, Set<Long> dictionaryIdsUsed) {
+    DictionaryEncoding encoding = field.getDictionary();
+    List<Field> children = field.getChildren();
+
+    if (encoding == null && children.isEmpty()) {
+      return field;
+    }
+
+    List<Field> updatedChildren = new ArrayList<>(children.size());
+    for (Field child: children) {
+      updatedChildren.add(toMessageFormat(child, provider, dictionaryIdsUsed));
+    }
+
+    ArrowType type;
+    if (encoding == null) {
+      type = field.getType();
+    } else {
+      long id = encoding.getId();
+      Dictionary dictionary = provider.lookup(id);
+      if (dictionary == null) {
+        throw new IllegalArgumentException("Could not find dictionary with ID " + id);
+      }
+      type = dictionary.getVectorType();
+
+      dictionaryIdsUsed.add(id);
+    }
+
+    return new Field(field.getName(), new FieldType(field.isNullable(), type, encoding, field.getMetadata()), updatedChildren);
+  }
+
+  /**
+   * Convert field and child fields that have a dictionary encoding to message format, so fields
+   * have the index type
+   */
+  public static Field toMemoryFormat(Field field, BufferAllocator allocator, Map<Long, Dictionary> dictionaries) {
+    DictionaryEncoding encoding = field.getDictionary();
+    List<Field> children = field.getChildren();
+
+    if (encoding == null && children.isEmpty()) {
+      return field;
+    }
+
+    List<Field> updatedChildren = new ArrayList<>(children.size());
+    for (Field child: children) {
+      updatedChildren.add(toMemoryFormat(child, allocator, dictionaries));
+    }
+
+    ArrowType type;
+    if (encoding == null) {
+      type = field.getType();
+    } else {
+      // re-type the field for in-memory format
+      type = encoding.getIndexType();
+      if (type == null) {
+        type = new ArrowType.Int(32, true);
+      }
+      // get existing or create dictionary vector
+      if (!dictionaries.containsKey(encoding.getId())) {
+        // create a new dictionary vector for the values
+        String dictName = "DICT" + encoding.getId();
+        Field dictionaryField = new Field(dictName, new FieldType(false, field.getType(), null, null), children);
+        FieldVector dictionaryVector = dictionaryField.createVector(allocator);
+        dictionaries.put(encoding.getId(), new Dictionary(dictionaryVector, encoding));
+      }
+    }
+
+    return new Field(field.getName(), new FieldType(field.isNullable(), type, encoding, field.getMetadata()), updatedChildren);
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
index 3035576..b70a63f 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/Validator.java
@@ -22,7 +22,10 @@ import java.util.List;
 
 import org.apache.arrow.vector.FieldVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
 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.Schema;
 
@@ -33,101 +36,143 @@ import com.google.common.base.Objects;
  */
 public class Validator {
 
-    /**
-     * Validate two arrow schemas are equal.
-     *
-     * @param schema1 the 1st shema to compare
-     * @param schema2 the 2nd shema to compare
-     * @throws IllegalArgumentException if they are different.
-     */
-    public static void compareSchemas(Schema schema1, Schema schema2) {
-        if (!schema2.equals(schema1)) {
-            throw new IllegalArgumentException("Different schemas:\n" + schema2 + "\n" + schema1);
-        }
-    }
-
-    /**
-     * Validate two arrow vectorSchemaRoot are equal.
-     *
-     * @param root1 the 1st shema to compare
-     * @param root2 the 2nd shema to compare
-     * @throws IllegalArgumentException if they are different.
-     */
-    public static void compareVectorSchemaRoot(VectorSchemaRoot root1, VectorSchemaRoot root2) {
-        compareSchemas(root2.getSchema(), root1.getSchema());
-        if (root1.getRowCount() != root2.getRowCount()) {
-            throw new IllegalArgumentException("Different row count:\n" + root1.getRowCount() + "\n" + root2.getRowCount());
-        }
-        List<FieldVector> arrowVectors = root1.getFieldVectors();
-        List<FieldVector> jsonVectors = root2.getFieldVectors();
-        if (arrowVectors.size() != jsonVectors.size()) {
-            throw new IllegalArgumentException("Different column count:\n" + arrowVectors.size() + "\n" + jsonVectors.size());
-        }
-        for (int i = 0; i < arrowVectors.size(); i++) {
-            Field field = root1.getSchema().getFields().get(i);
-            FieldVector arrowVector = arrowVectors.get(i);
-            FieldVector jsonVector = jsonVectors.get(i);
-            int valueCount = arrowVector.getAccessor().getValueCount();
-            if (valueCount != jsonVector.getAccessor().getValueCount()) {
-                throw new IllegalArgumentException("Different value count for field " + field + " : " + valueCount + " != " + jsonVector.getAccessor().getValueCount());
-            }
-            for (int j = 0; j < valueCount; j++) {
-                Object arrow = arrowVector.getAccessor().getObject(j);
-                Object json = jsonVector.getAccessor().getObject(j);
-                if (!equals(field.getType(), arrow, json)) {
-                    throw new IllegalArgumentException(
-                            "Different values in column:\n" + field + " at index " + j + ": " + arrow + " != " + json);
-                }
-            }
-        }
-    }
-
-    static boolean equals(ArrowType type, final Object o1, final Object o2) {
-        if (type instanceof ArrowType.FloatingPoint) {
-            ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) type;
-            switch (fpType.getPrecision()) {
-                case DOUBLE:
-                    return equalEnough((Double)o1, (Double)o2);
-                case SINGLE:
-                    return equalEnough((Float)o1, (Float)o2);
-                case HALF:
-                default:
-                    throw new UnsupportedOperationException("unsupported precision: " + fpType);
-            }
-        } else if (type instanceof ArrowType.Binary) {
-            return Arrays.equals((byte[]) o1, (byte[]) o2);
-        }
-
-        return Objects.equal(o1, o2);
-    }
-
-    static boolean equalEnough(Float f1, Float f2) {
-        if (f1 == null || f2 == null) {
-            return f1 == null && f2 == null;
-        }
-        if (f1.isNaN()) {
-            return f2.isNaN();
-        }
-        if (f1.isInfinite()) {
-            return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
-        }
-        float average = Math.abs((f1 + f2) / 2);
-        float differenceScaled = Math.abs(f1 - f2) / (average == 0.0f ? 1f : average);
-        return differenceScaled < 1.0E-6f;
-    }
-
-    static boolean equalEnough(Double f1, Double f2) {
-        if (f1 == null || f2 == null) {
-            return f1 == null && f2 == null;
-        }
-        if (f1.isNaN()) {
-            return f2.isNaN();
-        }
-        if (f1.isInfinite()) {
-            return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
-        }
-        double average = Math.abs((f1 + f2) / 2);
-        double differenceScaled = Math.abs(f1 - f2) / (average == 0.0d ? 1d : average);
-        return differenceScaled < 1.0E-12d;
+  /**
+   * Validate two arrow schemas are equal.
+   *
+   * @param schema1 the 1st schema to compare
+   * @param schema2 the 2nd schema to compare
+   * @throws IllegalArgumentException if they are different.
+   */
+  public static void compareSchemas(Schema schema1, Schema schema2) {
+    if (!schema2.equals(schema1)) {
+      throw new IllegalArgumentException("Different schemas:\n" + schema2 + "\n" + schema1);
     }
+  }
+
+  /**
+   * Validate two Dictionary encodings and dictionaries with id's from the encodings
+   */
+  public static void compareDictionaries(List<DictionaryEncoding> encodings1, List<DictionaryEncoding> encodings2, DictionaryProvider provider1, DictionaryProvider provider2) {
+
+    if (encodings1.size() != encodings2.size()) {
+      throw new IllegalArgumentException("Different dictionary encoding count:\n" + encodings1.size() + "\n" + encodings2.size());
+    }
+
+    for (int i = 0; i < encodings1.size(); i++) {
+      if (!encodings1.get(i).equals(encodings2.get(i))) {
+        throw new IllegalArgumentException("Different dictionary encodings:\n" + encodings1.get(i) + "\n" + encodings2.get(i));
+      }
+
+      long id = encodings1.get(i).getId();
+      Dictionary dict1 = provider1.lookup(id);
+      Dictionary dict2 = provider2.lookup(id);
+
+      if (dict1 == null || dict2 == null) {
+        throw new IllegalArgumentException("The DictionaryProvider did not contain the required dictionary with id: " + id +"\n" + dict1 + "\n" + dict2);
+      }
+
+      try {
+        compareFieldVectors(dict1.getVector(), dict2.getVector());
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgumentException("Different dictionaries:\n" + dict1 + "\n" + dict2, e);
+      }
+    }
+  }
+
+  /**
+   * Validate two arrow vectorSchemaRoot are equal.
+   *
+   * @param root1 the 1st schema to compare
+   * @param root2 the 2nd schema to compare
+   * @throws IllegalArgumentException if they are different.
+   */
+  public static void compareVectorSchemaRoot(VectorSchemaRoot root1, VectorSchemaRoot root2) {
+    compareSchemas(root2.getSchema(), root1.getSchema());
+    if (root1.getRowCount() != root2.getRowCount()) {
+      throw new IllegalArgumentException("Different row count:\n" + root1.getRowCount() + " != " + root2.getRowCount());
+    }
+    List<FieldVector> vectors1 = root1.getFieldVectors();
+    List<FieldVector> vectors2 = root2.getFieldVectors();
+    if (vectors1.size() != vectors2.size()) {
+      throw new IllegalArgumentException("Different column count:\n" + vectors1.toString() + "\n!=\n" + vectors2.toString());
+    }
+    for (int i = 0; i < vectors1.size(); i++) {
+      compareFieldVectors(vectors1.get(i), vectors2.get(i));
+    }
+  }
+
+  /**
+   * Validate two arrow FieldVectors are equal.
+   *
+   * @param vector1 the 1st VectorField to compare
+   * @param vector2 the 2nd VectorField to compare
+   * @throws IllegalArgumentException if they are different
+   */
+  public static void compareFieldVectors(FieldVector vector1, FieldVector vector2) {
+    Field field1 = vector1.getField();
+    if (!field1.equals(vector2.getField())) {
+      throw new IllegalArgumentException("Different Fields:\n" + field1 + "\n!=\n" + vector2.getField());
+    }
+    int valueCount = vector1.getAccessor().getValueCount();
+    if (valueCount != vector2.getAccessor().getValueCount()) {
+      throw new IllegalArgumentException("Different value count for field " + field1 + " : " + valueCount + " != " + vector2.getAccessor().getValueCount());
+    }
+    for (int j = 0; j < valueCount; j++) {
+      Object obj1 = vector1.getAccessor().getObject(j);
+      Object obj2 = vector2.getAccessor().getObject(j);
+      if (!equals(field1.getType(), obj1, obj2)) {
+        throw new IllegalArgumentException(
+            "Different values in column:\n" + field1 + " at index " + j + ": " + obj1 + " != " + obj2);
+      }
+    }
+  }
+
+  static boolean equals(ArrowType type, final Object o1, final Object o2) {
+    if (type instanceof ArrowType.FloatingPoint) {
+      ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) type;
+      switch (fpType.getPrecision()) {
+        case DOUBLE:
+          return equalEnough((Double)o1, (Double)o2);
+        case SINGLE:
+          return equalEnough((Float)o1, (Float)o2);
+        case HALF:
+        default:
+          throw new UnsupportedOperationException("unsupported precision: " + fpType);
+      }
+    } else if (type instanceof ArrowType.Binary) {
+      return Arrays.equals((byte[]) o1, (byte[]) o2);
+    }
+
+    return Objects.equal(o1, o2);
+  }
+
+  static boolean equalEnough(Float f1, Float f2) {
+    if (f1 == null || f2 == null) {
+      return f1 == null && f2 == null;
+    }
+    if (f1.isNaN()) {
+      return f2.isNaN();
+    }
+    if (f1.isInfinite()) {
+      return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
+    }
+    float average = Math.abs((f1 + f2) / 2);
+    float differenceScaled = Math.abs(f1 - f2) / (average == 0.0f ? 1f : average);
+    return differenceScaled < 1.0E-6f;
+  }
+
+  static boolean equalEnough(Double f1, Double f2) {
+    if (f1 == null || f2 == null) {
+      return f1 == null && f2 == null;
+    }
+    if (f1.isNaN()) {
+      return f2.isNaN();
+    }
+    if (f1.isInfinite()) {
+      return f2.isInfinite() && Math.signum(f1) == Math.signum(f2);
+    }
+    double average = Math.abs((f1 + f2) / 2);
+    double differenceScaled = Math.abs(f1 - f2) / (average == 0.0d ? 1d : average);
+    return differenceScaled < 1.0E-12d;
+  }
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java b/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
index 63027e6..3f717cb 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/BaseFileTest.java
@@ -17,18 +17,25 @@
  */
 package org.apache.arrow.vector.file;
 
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
 import java.util.List;
 
+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.NullableIntVector;
 import org.apache.arrow.vector.NullableTimeMilliVector;
+import org.apache.arrow.vector.NullableVarCharVector;
 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.impl.ComplexWriterImpl;
+import org.apache.arrow.vector.complex.impl.UnionListWriter;
 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;
@@ -39,8 +46,17 @@ import org.apache.arrow.vector.complex.writer.IntWriter;
 import org.apache.arrow.vector.complex.writer.TimeMilliWriter;
 import org.apache.arrow.vector.complex.writer.TimeStampMilliTZWriter;
 import org.apache.arrow.vector.complex.writer.TimeStampMilliWriter;
+import org.apache.arrow.vector.dictionary.Dictionary;
+import org.apache.arrow.vector.dictionary.DictionaryEncoder;
+import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.holders.NullableTimeStampMilliHolder;
+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.DateUtility;
+import org.apache.arrow.vector.util.DictionaryUtility;
+import org.apache.arrow.vector.util.Text;
 import org.joda.time.DateTimeZone;
 import org.joda.time.LocalDateTime;
 import org.junit.After;
@@ -51,6 +67,8 @@ import org.slf4j.LoggerFactory;
 
 import io.netty.buffer.ArrowBuf;
 
+import static org.apache.arrow.vector.TestUtils.newNullableVarCharVector;
+
 /**
  * Helps testing the file formats
  */
@@ -193,6 +211,226 @@ public class BaseFileTest {
     }
   }
 
+  protected VectorSchemaRoot writeFlatDictionaryData(BufferAllocator bufferAllocator, DictionaryProvider.MapDictionaryProvider provider) {
+
+    // Define dictionaries and add to provider
+    NullableVarCharVector dictionary1Vector = newNullableVarCharVector("D1", bufferAllocator);
+    dictionary1Vector.allocateNewSafe();
+    NullableVarCharVector.Mutator mutator = dictionary1Vector.getMutator();
+    mutator.set(0, "foo".getBytes(StandardCharsets.UTF_8));
+    mutator.set(1, "bar".getBytes(StandardCharsets.UTF_8));
+    mutator.set(2, "baz".getBytes(StandardCharsets.UTF_8));
+    mutator.setValueCount(3);
+
+    Dictionary dictionary1 = new Dictionary(dictionary1Vector, new DictionaryEncoding(1L, false, null));
+    provider.put(dictionary1);
+
+    NullableVarCharVector dictionary2Vector = newNullableVarCharVector("D2", bufferAllocator);
+    dictionary2Vector.allocateNewSafe();
+    mutator = dictionary2Vector.getMutator();
+    mutator.set(0, "micro".getBytes(StandardCharsets.UTF_8));
+    mutator.set(1, "small".getBytes(StandardCharsets.UTF_8));
+    mutator.set(2, "large".getBytes(StandardCharsets.UTF_8));
+    mutator.setValueCount(3);
+
+    Dictionary dictionary2 = new Dictionary(dictionary2Vector, new DictionaryEncoding(2L, false, null));
+    provider.put(dictionary2);
+
+    // Populate the vectors
+    NullableVarCharVector vector1A = newNullableVarCharVector("varcharA", bufferAllocator);
+    vector1A.allocateNewSafe();
+    mutator = vector1A.getMutator();
+    mutator.set(0, "foo".getBytes(StandardCharsets.UTF_8));
+    mutator.set(1, "bar".getBytes(StandardCharsets.UTF_8));
+    mutator.set(3, "baz".getBytes(StandardCharsets.UTF_8));
+    mutator.set(4, "bar".getBytes(StandardCharsets.UTF_8));
+    mutator.set(5, "baz".getBytes(StandardCharsets.UTF_8));
+    mutator.setValueCount(6);
+
+    FieldVector encodedVector1A = (FieldVector) DictionaryEncoder.encode(vector1A, dictionary1);
+    vector1A.close();  // Done with this vector after encoding
+
+    // Write this vector using indices instead of encoding
+    NullableIntVector encodedVector1B = new NullableIntVector("varcharB", bufferAllocator);
+    encodedVector1B.allocateNewSafe();
+    NullableIntVector.Mutator mutator1B = encodedVector1B.getMutator();
+    mutator1B.set(0, 2);  // "baz"
+    mutator1B.set(1, 1);  // "bar"
+    mutator1B.set(2, 2);  // "baz"
+    mutator1B.set(4, 1);  // "bar"
+    mutator1B.set(5, 0);  // "foo"
+    mutator1B.setValueCount(6);
+
+    NullableVarCharVector vector2 = newNullableVarCharVector("sizes", bufferAllocator);
+    vector2.allocateNewSafe();
+    mutator = vector2.getMutator();
+    mutator.set(1, "large".getBytes(StandardCharsets.UTF_8));
+    mutator.set(2, "small".getBytes(StandardCharsets.UTF_8));
+    mutator.set(3, "small".getBytes(StandardCharsets.UTF_8));
+    mutator.set(4, "large".getBytes(StandardCharsets.UTF_8));
+    mutator.setValueCount(6);
+
+    FieldVector encodedVector2 = (FieldVector) DictionaryEncoder.encode(vector2, dictionary2);
+    vector2.close();  // Done with this vector after encoding
+
+    List<Field> fields = ImmutableList.of(encodedVector1A.getField(), encodedVector1B.getField(), encodedVector2.getField());
+    List<FieldVector> vectors = ImmutableList.of(encodedVector1A, encodedVector1B, encodedVector2);
+
+    return new VectorSchemaRoot(fields, vectors, encodedVector1A.getAccessor().getValueCount());
+  }
+
+  protected void validateFlatDictionary(VectorSchemaRoot root, DictionaryProvider provider) {
+    FieldVector vector1A = root.getVector("varcharA");
+    Assert.assertNotNull(vector1A);
+
+    DictionaryEncoding encoding1A = vector1A.getField().getDictionary();
+    Assert.assertNotNull(encoding1A);
+    Assert.assertEquals(1L, encoding1A.getId());
+
+    FieldVector.Accessor accessor = vector1A.getAccessor();
+    Assert.assertEquals(6, accessor.getValueCount());
+    Assert.assertEquals(0, accessor.getObject(0));
+    Assert.assertEquals(1, accessor.getObject(1));
+    Assert.assertEquals(null, accessor.getObject(2));
+    Assert.assertEquals(2, accessor.getObject(3));
+    Assert.assertEquals(1, accessor.getObject(4));
+    Assert.assertEquals(2, accessor.getObject(5));
+
+    FieldVector vector1B = root.getVector("varcharB");
+    Assert.assertNotNull(vector1B);
+
+    DictionaryEncoding encoding1B = vector1A.getField().getDictionary();
+    Assert.assertNotNull(encoding1B);
+    Assert.assertTrue(encoding1A.equals(encoding1B));
+    Assert.assertEquals(1L, encoding1B.getId());
+
+    accessor = vector1B.getAccessor();
+    Assert.assertEquals(6, accessor.getValueCount());
+    Assert.assertEquals(2, accessor.getObject(0));
+    Assert.assertEquals(1, accessor.getObject(1));
+    Assert.assertEquals(2, accessor.getObject(2));
+    Assert.assertEquals(null, accessor.getObject(3));
+    Assert.assertEquals(1, accessor.getObject(4));
+    Assert.assertEquals(0, accessor.getObject(5));
+
+
+    FieldVector vector2 = root.getVector("sizes");
+    Assert.assertNotNull(vector2);
+
+    DictionaryEncoding encoding2 = vector2.getField().getDictionary();
+    Assert.assertNotNull(encoding2);
+    Assert.assertEquals(2L, encoding2.getId());
+
+    accessor = vector2.getAccessor();
+    Assert.assertEquals(6, accessor.getValueCount());
+    Assert.assertEquals(null, accessor.getObject(0));
+    Assert.assertEquals(2, accessor.getObject(1));
+    Assert.assertEquals(1, accessor.getObject(2));
+    Assert.assertEquals(1, accessor.getObject(3));
+    Assert.assertEquals(2, accessor.getObject(4));
+    Assert.assertEquals(null, accessor.getObject(5));
+
+    Dictionary dictionary1 = provider.lookup(1L);
+    Assert.assertNotNull(dictionary1);
+    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary1.getVector()).getAccessor();
+    Assert.assertEquals(3, dictionaryAccessor.getValueCount());
+    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
+    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
+    Assert.assertEquals(new Text("baz"), dictionaryAccessor.getObject(2));
+
+    Dictionary dictionary2 = provider.lookup(2L);
+    Assert.assertNotNull(dictionary2);
+    dictionaryAccessor = ((NullableVarCharVector) dictionary2.getVector()).getAccessor();
+    Assert.assertEquals(3, dictionaryAccessor.getValueCount());
+    Assert.assertEquals(new Text("micro"), dictionaryAccessor.getObject(0));
+    Assert.assertEquals(new Text("small"), dictionaryAccessor.getObject(1));
+    Assert.assertEquals(new Text("large"), dictionaryAccessor.getObject(2));
+  }
+
+  protected VectorSchemaRoot writeNestedDictionaryData(BufferAllocator bufferAllocator, DictionaryProvider.MapDictionaryProvider provider) {
+
+    // Define the dictionary and add to the provider
+    NullableVarCharVector dictionaryVector = newNullableVarCharVector("D2", bufferAllocator);
+    dictionaryVector.allocateNewSafe();
+    dictionaryVector.getMutator().set(0, "foo".getBytes(StandardCharsets.UTF_8));
+    dictionaryVector.getMutator().set(1, "bar".getBytes(StandardCharsets.UTF_8));
+    dictionaryVector.getMutator().setValueCount(2);
+
+    Dictionary dictionary = new Dictionary(dictionaryVector, new DictionaryEncoding(2L, false, null));
+    provider.put(dictionary);
+
+    // Write the vector data using dictionary indices
+    ListVector listVector = ListVector.empty("list", bufferAllocator);
+    DictionaryEncoding encoding = dictionary.getEncoding();
+    listVector.addOrGetVector(new FieldType(true, encoding.getIndexType(), encoding));
+    listVector.allocateNew();
+    UnionListWriter listWriter = new UnionListWriter(listVector);
+    listWriter.startList();
+    listWriter.writeInt(0);
+    listWriter.writeInt(1);
+    listWriter.endList();
+    listWriter.startList();
+    listWriter.writeInt(0);
+    listWriter.endList();
+    listWriter.startList();
+    listWriter.writeInt(1);
+    listWriter.endList();
+    listWriter.setValueCount(3);
+
+    List<Field> fields = ImmutableList.of(listVector.getField());
+    List<FieldVector> vectors = ImmutableList.<FieldVector>of(listVector);
+    return new VectorSchemaRoot(fields, vectors, 3);
+  }
+
+  protected void validateNestedDictionary(VectorSchemaRoot root, DictionaryProvider provider) {
+    FieldVector vector = root.getFieldVectors().get(0);
+    Assert.assertNotNull(vector);
+    Assert.assertNull(vector.getField().getDictionary());
+    Field nestedField = vector.getField().getChildren().get(0);
+
+    DictionaryEncoding encoding = nestedField.getDictionary();
+    Assert.assertNotNull(encoding);
+    Assert.assertEquals(2L, encoding.getId());
+    Assert.assertEquals(new ArrowType.Int(32, true), encoding.getIndexType());
+
+    FieldVector.Accessor accessor = vector.getAccessor();
+    Assert.assertEquals(3, accessor.getValueCount());
+    Assert.assertEquals(Arrays.asList(0, 1), accessor.getObject(0));
+    Assert.assertEquals(Arrays.asList(0), accessor.getObject(1));
+    Assert.assertEquals(Arrays.asList(1), accessor.getObject(2));
+
+    Dictionary dictionary = provider.lookup(2L);
+    Assert.assertNotNull(dictionary);
+    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary.getVector()).getAccessor();
+    Assert.assertEquals(2, dictionaryAccessor.getValueCount());
+    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
+    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
+  }
+
+  protected void validateNestedDictionary(ListVector vector, DictionaryProvider provider) {
+    Assert.assertNotNull(vector);
+    Assert.assertNull(vector.getField().getDictionary());
+    Field nestedField = vector.getField().getChildren().get(0);
+
+    DictionaryEncoding encoding = nestedField.getDictionary();
+    Assert.assertNotNull(encoding);
+    Assert.assertEquals(2L, encoding.getId());
+    Assert.assertEquals(new ArrowType.Int(32, true), encoding.getIndexType());
+
+    ListVector.Accessor accessor = vector.getAccessor();
+    Assert.assertEquals(3, accessor.getValueCount());
+    Assert.assertEquals(Arrays.asList(0, 1), accessor.getObject(0));
+    Assert.assertEquals(Arrays.asList(0), accessor.getObject(1));
+    Assert.assertEquals(Arrays.asList(1), accessor.getObject(2));
+
+    Dictionary dictionary = provider.lookup(2L);
+    Assert.assertNotNull(dictionary);
+    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary.getVector()).getAccessor();
+    Assert.assertEquals(2, dictionaryAccessor.getValueCount());
+    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
+    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
+  }
+
   protected void writeData(int count, MapVector parent) {
     ComplexWriter writer = new ComplexWriterImpl("root", parent);
     MapWriter rootWriter = writer.rootAsMap();

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
index 3353112..90fb5d2 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/TestArrowFile.java
@@ -17,8 +17,6 @@
  */
 package org.apache.arrow.vector.file;
 
-import static org.apache.arrow.vector.TestUtils.newNullableVarCharVector;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -26,9 +24,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -42,16 +38,10 @@ 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.NullableVarCharVector;
 import org.apache.arrow.vector.VectorSchemaRoot;
 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.NullableMapVector;
-import org.apache.arrow.vector.complex.impl.UnionListWriter;
-import org.apache.arrow.vector.dictionary.Dictionary;
-import org.apache.arrow.vector.dictionary.DictionaryEncoder;
-import org.apache.arrow.vector.dictionary.DictionaryProvider;
 import org.apache.arrow.vector.dictionary.DictionaryProvider.MapDictionaryProvider;
 import org.apache.arrow.vector.schema.ArrowBuffer;
 import org.apache.arrow.vector.schema.ArrowMessage;
@@ -64,11 +54,9 @@ import org.apache.arrow.vector.types.Types.MinorType;
 import org.apache.arrow.vector.types.pojo.ArrowType;
 import org.apache.arrow.vector.types.pojo.ArrowType.FixedSizeList;
 import org.apache.arrow.vector.types.pojo.ArrowType.Int;
-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.types.pojo.Schema;
-import org.apache.arrow.vector.util.Text;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -469,36 +457,12 @@ public class TestArrowFile extends BaseFileTest {
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
 
     // write
-    try (BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
-         NullableVarCharVector vector = newNullableVarCharVector("varchar", originalVectorAllocator);
-         NullableVarCharVector dictionaryVector = newNullableVarCharVector("dict", originalVectorAllocator)) {
-      vector.allocateNewSafe();
-      NullableVarCharVector.Mutator mutator = vector.getMutator();
-      mutator.set(0, "foo".getBytes(StandardCharsets.UTF_8));
-      mutator.set(1, "bar".getBytes(StandardCharsets.UTF_8));
-      mutator.set(3, "baz".getBytes(StandardCharsets.UTF_8));
-      mutator.set(4, "bar".getBytes(StandardCharsets.UTF_8));
-      mutator.set(5, "baz".getBytes(StandardCharsets.UTF_8));
-      mutator.setValueCount(6);
-
-      dictionaryVector.allocateNewSafe();
-      mutator = dictionaryVector.getMutator();
-      mutator.set(0, "foo".getBytes(StandardCharsets.UTF_8));
-      mutator.set(1, "bar".getBytes(StandardCharsets.UTF_8));
-      mutator.set(2, "baz".getBytes(StandardCharsets.UTF_8));
-      mutator.setValueCount(3);
-
-      Dictionary dictionary = new Dictionary(dictionaryVector, new DictionaryEncoding(1L, false, null));
-      MapDictionaryProvider provider = new MapDictionaryProvider();
-      provider.put(dictionary);
-
-      FieldVector encodedVector = (FieldVector) DictionaryEncoder.encode(vector, dictionary);
+    try (BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)) {
 
-      List<Field> fields = ImmutableList.of(encodedVector.getField());
-      List<FieldVector> vectors = ImmutableList.of(encodedVector);
-      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, 6);
+      MapDictionaryProvider provider = new MapDictionaryProvider();
 
-      try (FileOutputStream fileOutputStream = new FileOutputStream(file);
+      try (VectorSchemaRoot root = writeFlatDictionaryData(originalVectorAllocator, provider);
+           FileOutputStream fileOutputStream = new FileOutputStream(file);
            ArrowFileWriter fileWriter = new ArrowFileWriter(root, provider, fileOutputStream.getChannel());
            ArrowStreamWriter streamWriter = new ArrowStreamWriter(root, provider, stream)) {
         LOGGER.debug("writing schema: " + root.getSchema());
@@ -510,8 +474,10 @@ public class TestArrowFile extends BaseFileTest {
         streamWriter.end();
       }
 
-      dictionaryVector.close();
-      encodedVector.close();
+      // Need to close dictionary vectors
+      for (long id: provider.getDictionaryIds()) {
+        provider.lookup(id).getVector().close();
+      }
     }
 
     // read from file
@@ -522,7 +488,7 @@ public class TestArrowFile extends BaseFileTest {
       Schema schema = root.getSchema();
       LOGGER.debug("reading schema: " + schema);
       Assert.assertTrue(arrowReader.loadNextBatch());
-      validateFlatDictionary(root.getFieldVectors().get(0), arrowReader);
+      validateFlatDictionary(root, arrowReader);
     }
 
     // Read from stream
@@ -533,81 +499,31 @@ public class TestArrowFile extends BaseFileTest {
       Schema schema = root.getSchema();
       LOGGER.debug("reading schema: " + schema);
       Assert.assertTrue(arrowReader.loadNextBatch());
-      validateFlatDictionary(root.getFieldVectors().get(0), arrowReader);
+      validateFlatDictionary(root, arrowReader);
     }
   }
 
-  private void validateFlatDictionary(FieldVector vector, DictionaryProvider provider) {
-    Assert.assertNotNull(vector);
-
-    DictionaryEncoding encoding = vector.getField().getDictionary();
-    Assert.assertNotNull(encoding);
-    Assert.assertEquals(1L, encoding.getId());
-
-    FieldVector.Accessor accessor = vector.getAccessor();
-    Assert.assertEquals(6, accessor.getValueCount());
-    Assert.assertEquals(0, accessor.getObject(0));
-    Assert.assertEquals(1, accessor.getObject(1));
-    Assert.assertEquals(null, accessor.getObject(2));
-    Assert.assertEquals(2, accessor.getObject(3));
-    Assert.assertEquals(1, accessor.getObject(4));
-    Assert.assertEquals(2, accessor.getObject(5));
-
-    Dictionary dictionary = provider.lookup(1L);
-    Assert.assertNotNull(dictionary);
-    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary.getVector()).getAccessor();
-    Assert.assertEquals(3, dictionaryAccessor.getValueCount());
-    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
-    Assert.assertEquals(new Text("baz"), dictionaryAccessor.getObject(2));
-  }
-
   @Test
   public void testWriteReadNestedDictionary() throws IOException {
     File file = new File("target/mytest_dict_nested.arrow");
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
 
-    DictionaryEncoding encoding = new DictionaryEncoding(2L, false, null);
-
     // data being written:
     // [['foo', 'bar'], ['foo'], ['bar']] -> [[0, 1], [0], [1]]
 
     // write
-    try (NullableVarCharVector dictionaryVector = newNullableVarCharVector("dictionary", allocator);
-         ListVector listVector = ListVector.empty("list", allocator)) {
-
-      Dictionary dictionary = new Dictionary(dictionaryVector, encoding);
+    try (
+        BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)
+    ) {
       MapDictionaryProvider provider = new MapDictionaryProvider();
-      provider.put(dictionary);
-
-      dictionaryVector.allocateNew();
-      dictionaryVector.getMutator().set(0, "foo".getBytes(StandardCharsets.UTF_8));
-      dictionaryVector.getMutator().set(1, "bar".getBytes(StandardCharsets.UTF_8));
-      dictionaryVector.getMutator().setValueCount(2);
-
-      listVector.addOrGetVector(new FieldType(true, new Int(32, true), encoding));
-      listVector.allocateNew();
-      UnionListWriter listWriter = new UnionListWriter(listVector);
-      listWriter.startList();
-      listWriter.writeInt(0);
-      listWriter.writeInt(1);
-      listWriter.endList();
-      listWriter.startList();
-      listWriter.writeInt(0);
-      listWriter.endList();
-      listWriter.startList();
-      listWriter.writeInt(1);
-      listWriter.endList();
-      listWriter.setValueCount(3);
-
-      List<Field> fields = ImmutableList.of(listVector.getField());
-      List<FieldVector> vectors = ImmutableList.<FieldVector>of(listVector);
-      VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, 3);
-
-      try (
+
+      try (VectorSchemaRoot root = writeNestedDictionaryData(vectorAllocator, provider);
            FileOutputStream fileOutputStream = new FileOutputStream(file);
            ArrowFileWriter fileWriter = new ArrowFileWriter(root, provider, fileOutputStream.getChannel());
            ArrowStreamWriter streamWriter = new ArrowStreamWriter(root, provider, stream)) {
+
+        validateNestedDictionary(root, provider);
+
         LOGGER.debug("writing schema: " + root.getSchema());
         fileWriter.start();
         streamWriter.start();
@@ -616,6 +532,11 @@ public class TestArrowFile extends BaseFileTest {
         fileWriter.end();
         streamWriter.end();
       }
+
+      // Need to close dictionary vectors
+      for (long id: provider.getDictionaryIds()) {
+        provider.lookup(id).getVector().close();
+      }
     }
 
     // read from file
@@ -626,7 +547,7 @@ public class TestArrowFile extends BaseFileTest {
       Schema schema = root.getSchema();
       LOGGER.debug("reading schema: " + schema);
       Assert.assertTrue(arrowReader.loadNextBatch());
-      validateNestedDictionary((ListVector) root.getFieldVectors().get(0), arrowReader);
+      validateNestedDictionary(root, arrowReader);
     }
 
     // Read from stream
@@ -637,34 +558,10 @@ public class TestArrowFile extends BaseFileTest {
       Schema schema = root.getSchema();
       LOGGER.debug("reading schema: " + schema);
       Assert.assertTrue(arrowReader.loadNextBatch());
-      validateNestedDictionary((ListVector) root.getFieldVectors().get(0), arrowReader);
+      validateNestedDictionary(root, arrowReader);
     }
   }
 
-  private void validateNestedDictionary(ListVector vector, DictionaryProvider provider) {
-    Assert.assertNotNull(vector);
-    Assert.assertNull(vector.getField().getDictionary());
-    Field nestedField = vector.getField().getChildren().get(0);
-
-    DictionaryEncoding encoding = nestedField.getDictionary();
-    Assert.assertNotNull(encoding);
-    Assert.assertEquals(2L, encoding.getId());
-    Assert.assertEquals(new Int(32, true), encoding.getIndexType());
-
-    ListVector.Accessor accessor = vector.getAccessor();
-    Assert.assertEquals(3, accessor.getValueCount());
-    Assert.assertEquals(Arrays.asList(0, 1), accessor.getObject(0));
-    Assert.assertEquals(Arrays.asList(0), accessor.getObject(1));
-    Assert.assertEquals(Arrays.asList(1), accessor.getObject(2));
-
-    Dictionary dictionary = provider.lookup(2L);
-    Assert.assertNotNull(dictionary);
-    NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary.getVector()).getAccessor();
-    Assert.assertEquals(2, dictionaryAccessor.getValueCount());
-    Assert.assertEquals(new Text("foo"), dictionaryAccessor.getObject(0));
-    Assert.assertEquals(new Text("bar"), dictionaryAccessor.getObject(1));
-  }
-
   @Test
   public void testWriteReadFixedSizeList() throws IOException {
     File file = new File("target/mytest_fixed_list.arrow");

http://git-wip-us.apache.org/repos/asf/arrow/blob/7d86c28e/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java b/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
index 6c29cab..311cada 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/file/json/TestJSONFile.java
@@ -25,6 +25,8 @@ 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.dictionary.DictionaryProvider;
+import org.apache.arrow.vector.dictionary.DictionaryProvider.MapDictionaryProvider;
 import org.apache.arrow.vector.file.BaseFileTest;
 import org.apache.arrow.vector.types.pojo.Schema;
 import org.junit.Assert;
@@ -45,7 +47,7 @@ public class TestJSONFile extends BaseFileTest {
         BufferAllocator originalVectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE);
         MapVector parent = MapVector.empty("parent", originalVectorAllocator)) {
       writeComplexData(count, parent);
-      writeJSON(file, new VectorSchemaRoot(parent.getChild("root")));
+      writeJSON(file, new VectorSchemaRoot(parent.getChild("root")), null);
     }
 
     // read
@@ -74,13 +76,13 @@ public class TestJSONFile extends BaseFileTest {
       writeComplexData(count, parent);
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateComplexContent(root.getRowCount(), root);
-      writeJSON(file, root);
+      writeJSON(file, root, null);
     }
   }
 
-  public void writeJSON(File file, VectorSchemaRoot root) throws IOException {
+  public void writeJSON(File file, VectorSchemaRoot root, DictionaryProvider provider) throws IOException {
     JsonFileWriter writer = new JsonFileWriter(file, JsonFileWriter.config().pretty(true));
-    writer.start(root.getSchema());
+    writer.start(root.getSchema(), provider);
     writer.write(root);
     writer.close();
   }
@@ -101,7 +103,7 @@ public class TestJSONFile extends BaseFileTest {
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateUnionData(count, root);
 
-      writeJSON(file, root);
+      writeJSON(file, root, null);
     }
     // read
     try (
@@ -136,7 +138,7 @@ public class TestJSONFile extends BaseFileTest {
       VectorSchemaRoot root = new VectorSchemaRoot(parent.getChild("root"));
       validateDateTimeContent(count, root);
 
-      writeJSON(file, new VectorSchemaRoot(parent.getChild("root")));
+      writeJSON(file, new VectorSchemaRoot(parent.getChild("root")), null);
     }
 
     // read
@@ -156,6 +158,85 @@ public class TestJSONFile extends BaseFileTest {
   }
 
   @Test
+  public void testWriteReadDictionaryJSON() throws IOException {
+    File file = new File("target/mytest_dictionary.json");
+
+    // write
+    try (
+        BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)
+    ) {
+      MapDictionaryProvider provider = new MapDictionaryProvider();
+
+      try (VectorSchemaRoot root = writeFlatDictionaryData(vectorAllocator, provider)) {
+        printVectors(root.getFieldVectors());
+        validateFlatDictionary(root, provider);
+        writeJSON(file, root, provider);
+      }
+
+      // Need to close dictionary vectors
+      for (long id: provider.getDictionaryIds()) {
+        provider.lookup(id).getVector().close();
+      }
+    }
+
+    // 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();) {
+        validateFlatDictionary(root, reader);
+      }
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testWriteReadNestedDictionaryJSON() throws IOException {
+    File file = new File("target/mytest_dict_nested.json");
+
+    // data being written:
+    // [['foo', 'bar'], ['foo'], ['bar']] -> [[0, 1], [0], [1]]
+
+    // write
+    try (
+        BufferAllocator vectorAllocator = allocator.newChildAllocator("original vectors", 0, Integer.MAX_VALUE)
+    ) {
+      MapDictionaryProvider provider = new MapDictionaryProvider();
+
+      try (VectorSchemaRoot root = writeNestedDictionaryData(vectorAllocator, provider)) {
+        printVectors(root.getFieldVectors());
+        validateNestedDictionary(root, provider);
+        writeJSON(file, root, provider);
+      }
+
+      // Need to close dictionary vectors
+      for (long id: provider.getDictionaryIds()) {
+        provider.lookup(id).getVector().close();
+      }
+    }
+
+    // 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();) {
+        validateNestedDictionary(root, reader);
+      }
+      reader.close();
+    }
+  }
+
+  @Test
   public void testSetStructLength() throws IOException {
     File file = new File("../../integration/data/struct_example.json");
     try (