You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pr...@apache.org on 2020/05/06 01:31:56 UTC

[drill] branch master updated: DRILL-7717: Support Mongo extended types in V2 JSON loader

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ed5e543  DRILL-7717: Support Mongo extended types in V2 JSON loader
ed5e543 is described below

commit ed5e5438de071c0bdd1a7982f5d4ef029a95e10d
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Sun Apr 26 14:11:47 2020 -0700

    DRILL-7717: Support Mongo extended types in V2 JSON loader
    
    Adds support for Mongo extended types to the JSON loader.
    Refactors the JSON code to make such extensions easier.
    Added support for provided schema with extended types.
---
 .../java/org/apache/drill/categories/JsonTest.java |  24 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |   9 +-
 .../json/extended/BaseExtendedValueParser.java     | 107 ++++
 .../json/extended/ExtendedTypeFieldFactory.java    | 282 ++++++++++
 .../easy/json/extended/ExtendedTypeNames.java      |  51 ++
 .../easy/json/extended/MongoBinaryValueParser.java | 150 ++++++
 .../easy/json/extended/MongoDateValueParser.java   |  89 ++++
 .../json/extended/SimpleExtendedValueParser.java   |  48 ++
 .../store/easy/json/extended/package-info.java     | 142 +++++
 .../easy/json/loader/AbstractArrayListener.java    |  97 ----
 .../easy/json/loader/AbstractValueListener.java    |  81 ---
 .../store/easy/json/loader/BaseFieldFactory.java   | 261 ++++++++++
 .../easy/json/loader/EmptyArrayFieldParser.java    |  66 +++
 .../exec/store/easy/json/loader/FieldDefn.java     | 135 +++++
 .../exec/store/easy/json/loader/FieldFactory.java  |  50 ++
 .../easy/json/loader/InferredFieldFactory.java     | 245 +++++++++
 .../store/easy/json/loader/JsonLoaderImpl.java     |  80 ++-
 .../store/easy/json/loader/JsonLoaderOptions.java  |  37 ++
 .../exec/store/easy/json/loader/ListListener.java  |  82 ---
 .../store/easy/json/loader/NullFieldParser.java    |  56 ++
 .../easy/json/loader/ProvidedFieldFactory.java     | 176 +++++++
 .../json/loader/RepeatedListValueListener.java     | 196 -------
 .../store/easy/json/loader/ScalarListener.java     | 107 ----
 .../easy/json/loader/SimpleArrayListener.java      |  68 +++
 .../easy/json/loader/StructuredValueListener.java  | 143 ------
 .../exec/store/easy/json/loader/TupleListener.java | 525 -------------------
 .../exec/store/easy/json/loader/TupleParser.java   | 159 ++++++
 .../easy/json/loader/UnknownFieldListener.java     | 215 --------
 .../store/easy/json/loader/VariantListener.java    |  91 ----
 .../exec/store/easy/json/loader/VariantParser.java |  97 ++++
 .../easy/json/parser/AbstractElementParser.java    |  10 +-
 .../exec/store/easy/json/parser/ArrayListener.java |  10 -
 .../exec/store/easy/json/parser/ArrayParser.java   |  53 +-
 .../store/easy/json/parser/ArrayValueParser.java   |  66 +++
 .../store/easy/json/parser/DummyValueParser.java   |  21 +-
 .../exec/store/easy/json/parser/ElementParser.java |   2 -
 .../store/easy/json/parser/EmptyArrayParser.java   |  82 +++
 .../store/easy/json/parser/FieldParserFactory.java | 106 ++++
 .../store/easy/json/parser/FullValueParser.java    |  67 +++
 .../easy/json/parser/JsonStructureOptions.java     |   9 -
 .../easy/json/parser/JsonStructureParser.java      |  24 +-
 .../store/easy/json/parser/JsonValueParser.java    |  20 +-
 .../store/easy/json/parser/NullValueParser.java    |  56 ++
 .../store/easy/json/parser/ObjectListener.java     | 135 -----
 .../exec/store/easy/json/parser/ObjectParser.java  | 162 +++---
 ...ctElementParser.java => ObjectValueParser.java} |  38 +-
 .../exec/store/easy/json/parser/RootParser.java    |  18 +-
 .../store/easy/json/parser/ScalarValueParser.java  |  83 +++
 .../easy/json/parser/SimpleMessageParser.java      |  17 +-
 .../exec/store/easy/json/parser/TokenIterator.java |  18 +-
 .../exec/store/easy/json/parser/ValueDef.java      |  33 +-
 .../store/easy/json/parser/ValueDefFactory.java    |  21 +-
 .../exec/store/easy/json/parser/ValueListener.java | 113 +---
 .../exec/store/easy/json/parser/ValueParser.java   | 158 +-----
 .../store/easy/json/reader/BaseJsonReader.java     |   1 -
 .../json/{loader => values}/BigIntListener.java    |  44 +-
 .../BinaryValueListener.java}                      |  49 +-
 .../json/{loader => values}/BooleanListener.java   |  46 +-
 .../store/easy/json/values/DateValueListener.java  |  70 +++
 .../DecimalValueListener.java}                     |  56 +-
 .../json/{loader => values}/DoubleListener.java    |  46 +-
 .../easy/json/values/IntervalValueListener.java    |  57 +++
 .../store/easy/json/values/ScalarListener.java     |  79 +++
 .../json/values/StrictBigIntValueListener.java     |  61 +++
 .../json/values/StrictDoubleValueListener.java     |  62 +++
 .../easy/json/values/StrictIntValueListener.java   |  61 +++
 .../StrictStringValueListener.java}                |  48 +-
 .../store/easy/json/values/TimeValueListener.java  |  65 +++
 .../easy/json/values/TimestampValueListener.java   |  63 +++
 .../easy/json/values/UtcDateValueListener.java     |  73 +++
 .../json/values/UtcTimestampValueListener.java     |  72 +++
 .../json/{loader => values}/VarCharListener.java   |  51 +-
 .../store/easy/json/loader/BaseJsonLoaderTest.java |   8 +-
 .../exec/store/easy/json/loader/TestBasics.java    | 238 +++++++++
 .../store/easy/json/loader/TestExtendedArrays.java | 570 +++++++++++++++++++++
 .../store/easy/json/loader/TestExtendedTypes.java  | 562 ++++++++++++++++++++
 .../easy/json/loader/TestExtendedWithSchema.java   | 449 ++++++++++++++++
 .../store/easy/json/loader/TestMessageParser.java  | 248 +++++++++
 .../exec/store/easy/json/loader/TestObjects.java   |  18 +-
 .../store/easy/json/loader/TestRepeatedList.java   |  38 +-
 .../store/easy/json/loader/TestScalarArrays.java   |  18 +-
 .../exec/store/easy/json/loader/TestScalars.java   | 175 ++++++-
 .../exec/store/easy/json/loader/TestUnknowns.java  |  31 +-
 .../exec/store/easy/json/loader/TestVariant.java   |  12 +-
 .../store/easy/json/parser/BaseTestJsonParser.java | 350 -------------
 .../easy/json/parser/TestJsonParserArrays.java     | 401 ---------------
 .../easy/json/parser/TestJsonParserBasics.java     | 327 ------------
 .../easy/json/parser/TestJsonParserErrors.java     | 117 -----
 .../easy/json/parser/TestJsonParserMessage.java    | 193 -------
 .../easy/json/parser/TestJsonParserObjects.java    | 161 ------
 .../easy/json/parser/TestJsonParserUnknowns.java   | 294 -----------
 .../apache/drill/test/rowSet/RowSetUtilities.java  |  15 +
 .../drill/exec/expr/fn/impl/DateUtility.java       |   2 -
 .../drill/exec/record/metadata/MetadataUtils.java  |   7 +-
 94 files changed, 6162 insertions(+), 4267 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java b/common/src/test/java/org/apache/drill/categories/JsonTest.java
similarity index 56%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
copy to common/src/test/java/org/apache/drill/categories/JsonTest.java
index a99dcec..8539145 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
+++ b/common/src/test/java/org/apache/drill/categories/JsonTest.java
@@ -15,28 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.parser;
-
+package org.apache.drill.categories;
 
 /**
- * Abstract base class for all JSON element parsers.
+ * Tests for JSON-related tests: parser, writer, etc..
  */
-public abstract class AbstractElementParser implements ElementParser {
-  final JsonStructureParser structParser;
-  private final ElementParser parent;
-
-  public AbstractElementParser(ElementParser parent) {
-    this.parent = parent;
-    this.structParser = parent.structParser();
-  }
-
-  @Override
-  public ElementParser parent() { return parent; }
-
-  @Override
-  public JsonStructureParser structParser() { return structParser; }
+public interface JsonTest {
+  // Junit category marker
 
-  protected ErrorFactory errorFactory() {
-    return structParser.errorFactory();
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index a0d14a9..43396f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -460,11 +460,14 @@ public final class ExecConstants {
   public static final String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
   public static final BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE,
       new OptionDescription("Drill reads all data from the JSON files as VARCHAR. Prevents schema change errors."));
-  public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types",
+  public static final String JSON_EXTENDED_TYPES_KEY = "store.json.extended_types";
+  public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator(JSON_EXTENDED_TYPES_KEY,
       new OptionDescription("Turns on special JSON structures that Drill serializes for storing more type information than the four basic JSON types."));
-  public static final BooleanValidator JSON_WRITER_UGLIFY = new BooleanValidator("store.json.writer.uglify",
+  public static final String JSON_WRITER_UGLIFY_KEY = "store.json.writer.uglify";
+  public static final BooleanValidator JSON_WRITER_UGLIFY = new BooleanValidator(JSON_WRITER_UGLIFY_KEY,
       new OptionDescription("Enables Drill to return compact JSON output files; Drill does not separate records. Default is false. (Drill 1.4+)"));
-  public static final BooleanValidator JSON_WRITER_SKIPNULLFIELDS = new BooleanValidator("store.json.writer.skip_null_fields",
+  public static final String JSON_WRITER_SKIP_NULL_FIELDS_KEY = "store.json.writer.skip_null_fields";
+  public static final BooleanValidator JSON_WRITER_SKIPNULLFIELDS = new BooleanValidator(JSON_WRITER_SKIP_NULL_FIELDS_KEY,
       new OptionDescription("Enables Drill to skip extraneous NULL fields in JSON output files when executing the CTAS statement. Default is true. (Drill 1.6+)"));
   public static final String JSON_READER_SKIP_INVALID_RECORDS_FLAG = "store.json.reader.skip_invalid_records";
   public static final BooleanValidator JSON_SKIP_MALFORMED_RECORDS_VALIDATOR = new BooleanValidator(JSON_READER_SKIP_INVALID_RECORDS_FLAG,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/BaseExtendedValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/BaseExtendedValueParser.java
new file mode 100644
index 0000000..1bce101
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/BaseExtendedValueParser.java
@@ -0,0 +1,107 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+import org.apache.drill.exec.store.easy.json.values.ScalarListener;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public abstract class BaseExtendedValueParser extends ValueParser {
+
+  protected static final String SCALAR_HINT = "{\"%s\": scalar}";
+
+  public BaseExtendedValueParser(JsonStructureParser structParser, ScalarListener listener) {
+    super(structParser, listener);
+  }
+
+  protected abstract String typeName();
+
+  /**
+   * Parse a value in extended form:<pre><code>
+   * {"$type": value}</code</pre>.
+   * <p>
+   * Uses the given type name. Can parse an entire field,
+   * or a subfield, as in the V2 date format.
+   */
+  protected void parseExtended(TokenIterator tokenizer, String typeName) {
+
+    JsonToken token = tokenizer.requireNext();
+
+    // Null: assume the value is null
+    // (Extension to extended types)
+    if (token == JsonToken.VALUE_NULL) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Value is a scalar, assume "Relaxed format"
+    // (Extension to extended types: allow strings.)
+    if (token.isScalarValue()) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Must be an object
+    requireToken(token, JsonToken.START_OBJECT);
+
+    // Field name must be correct
+    requireField(tokenizer, typeName);
+
+    // Value must be a scalar
+    listener.onValue(requireScalar(tokenizer), tokenizer);
+
+    // Must be no other fields
+    requireToken(tokenizer, JsonToken.END_OBJECT);
+  }
+
+  protected void requireToken(TokenIterator tokenizer, JsonToken expected) {
+    requireToken(tokenizer.requireNext(), expected);
+  }
+
+  protected void requireToken(JsonToken token, JsonToken expected) {
+    if (token != expected) {
+      throw syntaxError();
+    }
+  }
+
+  protected JsonToken requireScalar(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    if (!token.isScalarValue()) {
+      throw syntaxError();
+    }
+    return token;
+  }
+
+  protected void requireField(TokenIterator tokenizer, String fieldName) {
+    requireToken(tokenizer, JsonToken.FIELD_NAME);
+    if (!tokenizer.textValue().equals(fieldName)) {
+      throw syntaxError();
+    }
+  }
+
+  protected RuntimeException syntaxError() {
+    return errorFactory().structureError(
+        String.format("Expected <%s> for extended type %s.",
+            formatHint(), typeName()));
+  }
+
+  protected abstract String formatHint();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeFieldFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeFieldFactory.java
new file mode 100644
index 0000000..b768ece
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeFieldFactory.java
@@ -0,0 +1,282 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.store.easy.json.loader.BaseFieldFactory;
+import org.apache.drill.exec.store.easy.json.loader.FieldDefn;
+import org.apache.drill.exec.store.easy.json.loader.FieldFactory;
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+import org.apache.drill.exec.store.easy.json.values.BinaryValueListener;
+import org.apache.drill.exec.store.easy.json.values.UtcDateValueListener;
+import org.apache.drill.exec.store.easy.json.values.DecimalValueListener;
+import org.apache.drill.exec.store.easy.json.values.IntervalValueListener;
+import org.apache.drill.exec.store.easy.json.values.StrictBigIntValueListener;
+import org.apache.drill.exec.store.easy.json.values.StrictDoubleValueListener;
+import org.apache.drill.exec.store.easy.json.values.StrictIntValueListener;
+import org.apache.drill.exec.store.easy.json.values.StrictStringValueListener;
+import org.apache.drill.exec.store.easy.json.values.TimeValueListener;
+import org.apache.drill.exec.store.easy.json.values.UtcTimestampValueListener;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public class ExtendedTypeFieldFactory extends BaseFieldFactory {
+
+  public ExtendedTypeFieldFactory(JsonLoaderImpl loader, FieldFactory child) {
+    super(loader, child);
+  }
+
+  @Override
+  public ElementParser fieldParser(FieldDefn fieldDefn) {
+    ElementParser parser = buildExtendedTypeParser(fieldDefn);
+    if (parser == null) {
+      return child.fieldParser(fieldDefn);
+    } else {
+      return parser;
+    }
+  }
+
+  private ElementParser buildExtendedTypeParser(FieldDefn fieldDefn) {
+
+    // Extended types are objects: { "$type": ... }
+    // Extended arrays are [ { "$type": ...
+    TokenIterator tokenizer = fieldDefn.tokenizer();
+    JsonToken token = tokenizer.requireNext();
+    ElementParser parser;
+    switch (token) {
+      case START_OBJECT:
+        parser = extendedTypeParserFor(fieldDefn, false);
+        break;
+      case START_ARRAY:
+        parser = arrayParserFor(fieldDefn);
+        break;
+      default:
+        parser = null;
+    }
+    tokenizer.unget(token);
+    return parser;
+  }
+
+  private ElementParser arrayParserFor(FieldDefn fieldDefn) {
+    TokenIterator tokenizer = fieldDefn.tokenizer();
+    JsonToken token = tokenizer.requireNext();
+    if (token != JsonToken.START_OBJECT) {
+      tokenizer.unget(token);
+      return null;
+    }
+
+    ValueParser element = extendedTypeParserFor(fieldDefn, true);
+    tokenizer.unget(token);
+    if (element == null) {
+      return null;
+    }
+
+    return scalarArrayParserFor(element);
+  }
+
+  private BaseExtendedValueParser extendedTypeParserFor(FieldDefn fieldDefn, boolean isArray) {
+    TokenIterator tokenizer = fieldDefn.tokenizer();
+    JsonToken token = tokenizer.peek();
+    if (token != JsonToken.FIELD_NAME) {
+      return null;
+    }
+
+    String key = tokenizer.textValue().trim();
+    if (!key.startsWith(ExtendedTypeNames.TYPE_PREFIX)) {
+      return null;
+    }
+    return parserFor(fieldDefn, key, isArray);
+  }
+
+  private BaseExtendedValueParser parserFor(FieldDefn fieldDefn, String key, boolean isArray) {
+    switch (key) {
+      case ExtendedTypeNames.LONG:
+        return numberLongParser(fieldDefn, isArray);
+      case ExtendedTypeNames.DECIMAL:
+        return numberDecimalParser(fieldDefn, isArray);
+      case ExtendedTypeNames.DOUBLE:
+        return numberDoubleParser(fieldDefn, isArray);
+      case ExtendedTypeNames.INT:
+        return numberIntParser(fieldDefn, isArray);
+      case ExtendedTypeNames.DATE:
+        return dateParser(fieldDefn, isArray);
+      case ExtendedTypeNames.BINARY:
+      case ExtendedTypeNames.BINARY_TYPE:
+        return binaryParser(fieldDefn, isArray);
+      case ExtendedTypeNames.OBJECT_ID:
+        return oidParser(fieldDefn, isArray);
+      case ExtendedTypeNames.DATE_DAY:
+        return dateDayParser(fieldDefn, isArray);
+      case ExtendedTypeNames.TIME:
+        return timeParser(fieldDefn, isArray);
+      case ExtendedTypeNames.INTERVAL:
+        return intervalParser(fieldDefn, isArray);
+      default:
+        return null;
+    }
+  }
+
+  /**
+   * Infer the extended parser from the provided field type. The user is required
+   * to pick field type consistent with this mapping from their data types. Cannot
+   * handle, say, a column which is an int in one row and long in another.
+   */
+  @Override
+  public ValueParser scalarParserFor(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    switch (colSchema.type()) {
+      case BIGINT:
+      case UINT8:
+        return numberLongParser(fieldDefn, colSchema);
+      case DATE:
+        return dateDayParser(fieldDefn, colSchema);
+      case FLOAT4:
+      case FLOAT8:
+        return numberDoubleParser(fieldDefn, colSchema);
+      case INT:
+      case SMALLINT:
+      case UINT2:
+      case UINT4:
+        return numberIntParser(fieldDefn, colSchema);
+      case INTERVAL:
+      case INTERVALDAY:
+      case INTERVALYEAR:
+        return intervalParser(fieldDefn, colSchema);
+      case TIME:
+        return timeParser(fieldDefn, colSchema);
+      case TIMESTAMP:
+        return dateParser(fieldDefn, colSchema);
+      case VARBINARY:
+        return binaryParser(fieldDefn, colSchema);
+      case VARDECIMAL:
+        return numberDecimalParser(fieldDefn, colSchema);
+      default:
+        return child.scalarParserFor(fieldDefn, colSchema);
+    }
+  }
+
+  private BaseExtendedValueParser numberLongParser(FieldDefn fieldDefn, boolean isArray) {
+    return numberLongParser(fieldDefn, fieldDefn.schemaFor(MinorType.BIGINT, isArray));
+  }
+
+  private BaseExtendedValueParser numberLongParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.LONG,
+        new StrictBigIntValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser numberDecimalParser(FieldDefn fieldDefn, boolean isArray) {
+    // No information about precision and scale, so guess (38, 10).
+    // TODO: maybe make a config option?
+    return numberDecimalParser(fieldDefn,
+        MetadataUtils.newDecimal(fieldDefn.key(), fieldDefn.mode(isArray), 38, 10));
+  }
+
+  private BaseExtendedValueParser numberDecimalParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.DECIMAL,
+        new DecimalValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser numberDoubleParser(FieldDefn fieldDefn, boolean isArray) {
+    return numberDoubleParser(fieldDefn, fieldDefn.schemaFor(MinorType.FLOAT8, isArray));
+  }
+
+  private BaseExtendedValueParser numberDoubleParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.DOUBLE,
+        new StrictDoubleValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser numberIntParser(FieldDefn fieldDefn, boolean isArray) {
+    return numberIntParser(fieldDefn, fieldDefn.schemaFor(MinorType.INT, isArray));
+  }
+
+  private BaseExtendedValueParser numberIntParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.INT,
+        new StrictIntValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser dateParser(FieldDefn fieldDefn, boolean isArray) {
+    return dateParser(fieldDefn, fieldDefn.schemaFor(MinorType.TIMESTAMP, isArray));
+  }
+
+  private BaseExtendedValueParser dateParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new MongoDateValueParser(fieldDefn.parser(),
+        new UtcTimestampValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser binaryParser(FieldDefn fieldDefn, boolean isArray) {
+    return binaryParser(fieldDefn, fieldDefn.schemaFor(MinorType.VARBINARY, isArray));
+  }
+
+  private BaseExtendedValueParser binaryParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new MongoBinaryValueParser(fieldDefn.parser(),
+        new BinaryValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser dateDayParser(FieldDefn fieldDefn, boolean isArray) {
+    return dateDayParser(fieldDefn, fieldDefn.schemaFor(MinorType.DATE, isArray));
+  }
+
+  private BaseExtendedValueParser dateDayParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.DATE_DAY,
+        new UtcDateValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser timeParser(FieldDefn fieldDefn, boolean isArray) {
+    return timeParser(fieldDefn, fieldDefn.schemaFor(MinorType.TIME, isArray));
+  }
+
+  private BaseExtendedValueParser timeParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(fieldDefn.parser(), ExtendedTypeNames.TIME,
+        new TimeValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser intervalParser(FieldDefn fieldDefn, boolean isArray) {
+    return intervalParser(fieldDefn, fieldDefn.schemaFor(MinorType.INTERVAL, isArray));
+  }
+
+  private BaseExtendedValueParser intervalParser(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.INTERVAL,
+        new IntervalValueListener(loader(),
+            fieldDefn.scalarWriterFor(colSchema)));
+  }
+
+  private BaseExtendedValueParser oidParser(FieldDefn fieldDefn, boolean isArray) {
+    return new SimpleExtendedValueParser(
+        fieldDefn.parser(), ExtendedTypeNames.OBJECT_ID,
+        new StrictStringValueListener(loader(),
+            fieldDefn.scalarWriterFor(MinorType.VARCHAR, isArray)));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeNames.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeNames.java
new file mode 100644
index 0000000..3e62af6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/ExtendedTypeNames.java
@@ -0,0 +1,51 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.exec.vector.complex.fn.ExtendedTypeName;
+
+/**
+ * Names of Mongo extended types. Includes both
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/">
+ * V1</a> and
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/">
+ * V2</a> names.
+ *
+ * @see org.apache.drill.exec.vector.complex.fn.ExtendedTypeName ExtendedTypeName
+ * for an older version of these names
+ */
+public interface ExtendedTypeNames {
+
+  // Supported Mongo types
+  String TYPE_PREFIX = "$";
+  String LONG = "$numberLong";
+  String DECIMAL = "$numberDecimal";
+  String DOUBLE = "$numberDouble";
+  String INT = "$numberInt";
+  String DATE = "$date";
+  String BINARY = "$binary";
+  String OBJECT_ID = "$oid";
+
+  // Drill extensions
+  String DATE_DAY = ExtendedTypeName.DATE;
+  String TIME = ExtendedTypeName.TIME;
+  String INTERVAL = ExtendedTypeName.INTERVAL;
+
+  // The V1 JSON reader allows binary of format {"$type": 1, $binary: "..."}
+  String BINARY_TYPE = ExtendedTypeName.TYPE;
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoBinaryValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoBinaryValueParser.java
new file mode 100644
index 0000000..c9eda82
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoBinaryValueParser.java
@@ -0,0 +1,150 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.values.ScalarListener;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parsers a binary. Ignores the subtype field.</pre>
+ */
+public class MongoBinaryValueParser extends BaseExtendedValueParser {
+
+  protected static final String BINARY_HINT =
+      "{\"$binary\": {base64: (\"<payload>\", subType: \"<t>\" }) | " +
+        "(\"<payload>\", \"$type\": \"<t>\") }";
+
+  public MongoBinaryValueParser(JsonStructureParser structParser, ScalarListener listener) {
+    super(structParser, listener);
+  }
+
+  @Override
+  protected String typeName() { return ExtendedTypeNames.BINARY; }
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+
+    // Null: assume the value is null
+    // (Extension to extended types)
+    JsonToken token = tokenizer.requireNext();
+    if (token == JsonToken.VALUE_NULL) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Value is a scalar, assume binary value as a string.
+    // This is a harmless extension to the standard.
+     if (token.isScalarValue()) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Must be an object
+    requireToken(token, JsonToken.START_OBJECT);
+
+    // { ^($binary | $type)
+    requireToken(tokenizer, JsonToken.FIELD_NAME);
+    String fieldName = tokenizer.textValue();
+    if (fieldName.equals(ExtendedTypeNames.BINARY_TYPE)) {
+      // { $type ^
+      parseV1Format(fieldName, tokenizer);
+    } else if (!fieldName.equals(ExtendedTypeNames.BINARY)) {
+      throw syntaxError();
+    } else if (tokenizer.peek() == JsonToken.START_OBJECT) {
+      // { $binary: ^{
+      parseV2Format(tokenizer);
+    } else {
+      // { $binary: ^value
+      parseV1Format(fieldName, tokenizer);
+    }
+  }
+
+  // Parse field: { ($binary | $type) ^ ...
+  private void parseV1Format(String fieldName, TokenIterator tokenizer) {
+    boolean sawData = false;
+    for (;;) {
+      // key: ^value
+      JsonToken token = requireScalar(tokenizer);
+      if (fieldName.equals(ExtendedTypeNames.BINARY)) {
+        if (sawData) {
+          throw syntaxError();
+        }
+        sawData = true;
+        listener.onValue(token, tokenizer);
+      }
+
+      // key: value ^(} | key ...)
+      token = tokenizer.requireNext();
+      if (token == JsonToken.END_OBJECT) {
+        break;
+      } if (token != JsonToken.FIELD_NAME) {
+        throw syntaxError();
+      }
+      fieldName = tokenizer.textValue();
+      switch (fieldName) {
+      case ExtendedTypeNames.BINARY:
+      case ExtendedTypeNames.BINARY_TYPE:
+        break;
+      default:
+        throw syntaxError();
+      }
+    }
+    if (!sawData) {
+      throw syntaxError();
+    }
+  }
+
+  // Parse field: { $binary: ^{ "base64": "<payload>", "subType": "<t>" } }
+  // With fields in either order
+  private void parseV2Format(TokenIterator tokenizer) {
+    boolean sawData = false;
+    requireToken(tokenizer, JsonToken.START_OBJECT);
+    for (;;) {
+      JsonToken token = tokenizer.requireNext();
+      if (token == JsonToken.END_OBJECT) {
+        break;
+      } else if (token != JsonToken.FIELD_NAME) {
+        throw syntaxError();
+      }
+      switch (tokenizer.textValue()) {
+        case "base64":
+          if (sawData) {
+            throw syntaxError();
+          }
+          sawData = true;
+          listener.onValue(requireScalar(tokenizer), tokenizer);
+          break;
+        case "subType":
+          requireScalar(tokenizer);
+          break;
+        default:
+          throw syntaxError();
+      }
+    }
+    // { $binary: { ... } ^}
+    requireToken(tokenizer, JsonToken.END_OBJECT);
+  }
+
+  @Override
+  protected String formatHint() {
+    return BINARY_HINT;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoDateValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoDateValueParser.java
new file mode 100644
index 0000000..e3e206a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/MongoDateValueParser.java
@@ -0,0 +1,89 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.values.ScalarListener;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parses a Mongo date in the
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/#date">V1</a> format:<pre><code>
+ * { "$date": "&lt;date>" }</code></pre> and in the
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Date">V2</a> formats:<pre><code>
+ * {"$date": {"$numberLong": "&lt;millis>"}
+ * {"$date": "&lt;ISO-8601 Date/Time Format>"}</code></pre>
+ */
+public class MongoDateValueParser extends BaseExtendedValueParser {
+
+  private static final String DATE_HINT = "^{\"$date\": scalar | " +
+    String.format(SCALAR_HINT, ExtendedTypeNames.DOUBLE) + "}";
+
+  public MongoDateValueParser(JsonStructureParser structParser, ScalarListener listener) {
+    super(structParser, listener);
+  }
+
+  @Override
+  protected String typeName() { return ExtendedTypeNames.DATE; }
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+
+    // Null: assume the value is null
+    // (Extension to extended types)
+    JsonToken token = tokenizer.requireNext();
+    if (token == JsonToken.VALUE_NULL) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Value is a scalar, assume "Relaxed format"
+    // (Extension to extended types: allow strings.)
+    if (token.isScalarValue()) {
+      listener.onValue(token, tokenizer);
+      return;
+    }
+
+    // Must be an object
+    requireToken(token, JsonToken.START_OBJECT);
+
+    // Field name must be correct
+    requireField(tokenizer, ExtendedTypeNames.DATE);
+
+    // If value is an object, assume V2 canonical format.
+    token = tokenizer.requireNext();
+    if (token == JsonToken.START_OBJECT) {
+      tokenizer.unget(token);
+      parseExtended(tokenizer, ExtendedTypeNames.LONG);
+    } else {
+      // Otherwise, Value must be a scalar
+      tokenizer.unget(token);
+      listener.onValue(requireScalar(tokenizer), tokenizer);
+    }
+
+    // Must be no other fields
+    requireToken(tokenizer, JsonToken.END_OBJECT);
+  }
+
+  @Override
+  protected String formatHint() {
+    return DATE_HINT;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/SimpleExtendedValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/SimpleExtendedValueParser.java
new file mode 100644
index 0000000..cde240d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/SimpleExtendedValueParser.java
@@ -0,0 +1,48 @@
+/*
+ * 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.drill.exec.store.easy.json.extended;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.values.ScalarListener;
+
+/**
+ * Parsers a Mongo extended type of the form:<pre><code>
+ * { $type: value }</code></pre>
+ */
+public class SimpleExtendedValueParser extends BaseExtendedValueParser {
+  private final String typeName;
+
+  public SimpleExtendedValueParser(JsonStructureParser structParser, String typeName, ScalarListener listener) {
+    super(structParser, listener);
+    this.typeName = typeName;
+  }
+
+  @Override
+  protected String typeName() { return typeName; }
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    parseExtended(tokenizer, typeName);
+  }
+
+  @Override
+  protected String formatHint() {
+    return String.format(SCALAR_HINT, typeName());
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/package-info.java
new file mode 100644
index 0000000..9cc9a9c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/extended/package-info.java
@@ -0,0 +1,142 @@
+/*
+ * 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.
+ */
+
+/**
+ * Provides parsing for Mongo extended types which are generally of the form
+ * <code>{ "$type": value }</code>. Supports both
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/">
+ * V1</a> and
+ * <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/">
+ * V2</a> names. Supports both the Canonical and Relaxed formats.
+ * <p>
+ * Does not support all types as some appear internal to Mongo. Supported
+ * types:
+ * <ul>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Array>
+ * Array</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Binary">
+ * Binary</a>, translated to a Drill {@code VARBINARY}. The data must be encoded in
+ * the <a href="https://fasterxml.github.io/jackson-core/javadoc/2.2.0/com/fasterxml/jackson/core/JsonParser.html#getBinaryValue()">
+ * default Jackson Base64 format.</a> The {@code subType} field, if present, is ignored.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Date">
+ * Date</a>, translated to a Drill {@code TIMESTAMP}. Drill's times are
+ * in the server local time. The UTC date in Mongo will be shifted to the local time
+ * zone on read.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/#numberdecimal">
+ * Decimal (V1)</a>, translated to a Drill {@code VARDECIMAL}.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Decimal128">
+ * Decimal128 (V2)</a>, translated to a Drill {@code VARDECIMAL}, but limited to the
+ * supported DECIMAL range.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Document">
+ * Document</a> which is translated to a Drill {@code MAP}. The map fields must be consistent
+ * across documents: same names and types. (This is a restriction of Maps in Drill's
+ * relational data model.) Field names cannot be the same as any of the extended type
+ * names.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Double">
+ * Double, translated to a Drill {@code FLOAT8}.</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Int64">
+ * Int64</a>, translated to a Drill {@code BIGINT}.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Int32">
+ * Int32</a>, translated to a Drill {@code INT}.</li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.ObjectId">
+ * Object ID</a>, translated to a Drill {@code VARCHAR}.</li>
+ * </ul>
+ * Unsupported types:
+ * <ul>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.MaxKey">
+ * MaxKey</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.MinKey">
+ * MinKey</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Regular-Expression">
+ * Regular Expression</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/#bson.data_ref">
+ * Data Ref (V1)</a></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json/#bson.Timestamp">
+ * Timestamp</a>. According to
+ * <a href="https://docs.mongodb.com/manual/reference/bson-types/#timestamps">this page</a>:
+ * <quote>The BSON timestamp type is for internal MongoDB use. For most cases, in application
+ * development, you will want to use the BSON date type.</quote></li>
+ * <li><a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/#bson.data_undefined">
+ * Undefined (V1)</a>, since Drill has no untyped {@code NULL} value.</li>
+ * </ul>
+ * <p>
+ * The unsupported types appear more for commands and queries rather than data. They do
+ * not represent a Drill type. If they appear in data, they will be translated to a
+ * Drill map.
+ * <p>
+ * Drill defines a few "extended extended" types:
+ * <ul>
+ * <li>Date ({@code $dateDay}) - a date-only field in the form {@code YYYY-MM-DD} which
+ * maps to a Drill {@code DATE} vector.</li>
+ * <li>Time ({@code $time}) - a time-only field in the form {@code HH:MM:SS.SSS} which
+ * maps to a Drill {@code TIME} vector.</li>
+ * <li>Interval ({@code $interval}) - a date/time interval in ISO format which maps
+ * to a Drill {@code INTERVAL} vector.</a>
+ * </ul>
+ * <p>
+ * Drill extends the extended types to allow null values in the usual way. Drill
+ * accepts normal "un-extended" JSON in the same file, but doing so can lead to ambiguities
+ * (see below.)
+ * <p>
+ * Once Drill defines a field as an extended type, parsing rules are tighter than
+ * for normal "non-extended" types. For example an extended double will not convert
+ * from a Boolean or float value.
+ *
+ * <h4>Provided Schema</h4>
+ *
+ * If used with a provided schema, then:
+ * <ul>
+ * <li>If the first field is in canonical format (with a type), then the extended
+ * type must agree with the provided type, or an error will occur.</li>
+ * <li>If the first field is in relaxed format, or is {@code null}, then the
+ * provided schema will force the given type as though the data were in
+ * canonical format.</li>
+ * </ul>
+ *
+ * <h4>Ambiguities</h4>
+ *
+ * Extended JSON is subject to the same ambiguities as normal JSON. If Drill sees a
+ * field in relaxed mode before extended mode, Drill will use its normal type inference
+ * rules. Thus, if the first field presents as {@code a: "30"}, Drill will infer the
+ * type as string, even if a later field presents as <code>a: { "numberInt": 30 }</code>.
+ * To avoid ambiguities, either use only the canonical format, or use a provided
+ * schema.
+ *
+ * <h4>Implementation</h4>
+ *
+ * Extended types disabled by default and must be enabled using the
+ * {@code store.json.extended_types} system/session option (
+ * {@link org.apache.drill.exec.ExecConstants#JSON_EXTENDED_TYPES_KEY}).
+ * <p>
+ * Extended types are implemented via a field factory. The field factory builds the
+ * structure needed each time the JSON structure parser sees a new field. For extended types,
+ * the field factory looks ahead to detect an extended type, specifically for the pattern
+ * <code>{ "$type":</code>. If the pattern is found, and the name is one of the supported
+ * type names, then the factory creates a parser to accept the enhanced type in either the
+ * canonical or relaxed forms.
+ * <p>
+ * Each field is represented by a Mongo-specific parser along with an associated value
+ * listener. The implementation does not reify the object structure; that structure is
+ * consumed by the field parser itself. The value listener receives value tokens as if
+ * the data were in relaxed format.
+ *
+ *
+ * @see org.apache.drill.exec.vector.complex.fn.VectorOutput.MapVectorOutput MapVectorOutput
+ * for an older implementation
+*/
+package org.apache.drill.exec.store.easy.json.extended;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractArrayListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractArrayListener.java
deleted file mode 100644
index 69d7bee..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractArrayListener.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ObjectValueListener;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-
-/**
- * Base class for scalar and object arrays. Represents the array
- * behavior of a field.
- */
-public abstract class AbstractArrayListener implements ArrayListener {
-
-  protected final JsonLoaderImpl loader;
-  protected final ColumnMetadata colSchema;
-  protected final ValueListener elementListener;
-
-  public AbstractArrayListener(JsonLoaderImpl loader, ColumnMetadata colSchema, ValueListener elementListener) {
-    this.loader = loader;
-    this.colSchema = colSchema;
-    this.elementListener = elementListener;
-  }
-
-  public ValueListener elementListener() { return elementListener; }
-
-  @Override
-  public void onStart() { }
-
-  @Override
-  public void onElementStart() { }
-
-  @Override
-  public void onElementEnd() { }
-
-  @Override
-  public void onEnd() { }
-
-  @Override
-  public ValueListener element(ValueDef valueDef) {
-    throw loader.typeConversionError(colSchema, valueDef);
-  }
-
-  protected UserException typeConversionError(String jsonType) {
-    return loader.typeConversionError(colSchema, jsonType);
-  }
-
-  public static class ScalarArrayListener extends AbstractArrayListener {
-
-    public ScalarArrayListener(JsonLoaderImpl loader, ColumnMetadata colSchema, ScalarListener valueListener) {
-      super(loader, colSchema, valueListener);
-    }
-
-    @Override
-    public ValueListener element(ValueDef valueDef) {
-      return elementListener;
-    }
-  }
-
-  public static class ObjectArrayListener extends AbstractArrayListener {
-    private final ArrayWriter arrayWriter;
-
-    public ObjectArrayListener(JsonLoaderImpl loader, ArrayWriter arrayWriter, ObjectValueListener valueListener) {
-      super(loader, arrayWriter.schema(), valueListener);
-      this.arrayWriter = arrayWriter;
-    }
-
-    @Override
-    public ValueListener element(ValueDef valueDef) {
-      return elementListener;
-    }
-
-    @Override
-    public void onElementEnd() {
-      arrayWriter.save();
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractValueListener.java
deleted file mode 100644
index b881ae2..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/AbstractValueListener.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-
-/**
- * Abstract base class for value (field or array element) listeners.
- */
-public abstract class AbstractValueListener implements ValueListener {
-
-  protected final JsonLoaderImpl loader;
-
-  public AbstractValueListener(JsonLoaderImpl loader) {
-    this.loader = loader;
-  }
-
-  @Override
-  public void bind(ValueHost host) { }
-
-  @Override
-  public void onBoolean(boolean value) {
-    throw typeConversionError("Boolean");
-  }
-
-  @Override
-  public void onInt(long value) {
-    throw typeConversionError("integer");
-  }
-
-  @Override
-  public void onFloat(double value) {
-    throw typeConversionError("float");
-  }
-
-  @Override
-  public void onString(String value) {
-    throw typeConversionError("string");
-  }
-
-  @Override
-  public void onEmbeddedObject(String value) {
-    throw typeConversionError("object");
-  }
-
-  @Override
-  public ObjectListener object() {
-    throw typeConversionError("object");
-  }
-
-  @Override
-  public ArrayListener array(ValueDef valueDef) {
-    throw loader.typeConversionError(schema(), valueDef);
-  }
-
-  protected UserException typeConversionError(String jsonType) {
-    return loader.typeConversionError(schema(), jsonType);
-  }
-
-  protected abstract ColumnMetadata schema();
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BaseFieldFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BaseFieldFactory.java
new file mode 100644
index 0000000..49185ce
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BaseFieldFactory.java
@@ -0,0 +1,261 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import java.util.function.Function;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.easy.json.loader.SimpleArrayListener.ListArrayListener;
+import org.apache.drill.exec.store.easy.json.loader.SimpleArrayListener.StructureArrayListener;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.FieldParserFactory;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+import org.apache.drill.exec.store.easy.json.values.BigIntListener;
+import org.apache.drill.exec.store.easy.json.values.BinaryValueListener;
+import org.apache.drill.exec.store.easy.json.values.BooleanListener;
+import org.apache.drill.exec.store.easy.json.values.DateValueListener;
+import org.apache.drill.exec.store.easy.json.values.DecimalValueListener;
+import org.apache.drill.exec.store.easy.json.values.DoubleListener;
+import org.apache.drill.exec.store.easy.json.values.IntervalValueListener;
+import org.apache.drill.exec.store.easy.json.values.ScalarListener;
+import org.apache.drill.exec.store.easy.json.values.StrictIntValueListener;
+import org.apache.drill.exec.store.easy.json.values.TimeValueListener;
+import org.apache.drill.exec.store.easy.json.values.TimestampValueListener;
+import org.apache.drill.exec.store.easy.json.values.VarCharListener;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.exec.vector.accessor.VariantWriter;
+
+/**
+ * Base field factor class which handles the common tasks for
+ * building column writers and JSON listeners.
+ */
+public abstract class BaseFieldFactory implements FieldFactory {
+
+  protected final JsonLoaderImpl loader;
+  protected final FieldFactory child;
+
+  public BaseFieldFactory(JsonLoaderImpl loader) {
+    this(loader, null);
+  }
+
+  public BaseFieldFactory(JsonLoaderImpl loader, FieldFactory child) {
+    this.loader = loader;
+    this.child = child;
+  }
+
+  protected FieldParserFactory parserFactory() {
+    return loader().parser().fieldFactory();
+  }
+
+  @Override
+  public ElementParser ignoredFieldParser() {
+    return parserFactory().ignoredFieldParser();
+  }
+
+  protected JsonLoaderImpl loader() { return loader; }
+
+  @Override
+  public ValueParser scalarParserFor(FieldDefn fieldDefn, ColumnMetadata colSchema) {
+    return scalarParserFor(fieldDefn.scalarWriterFor(colSchema));
+  }
+
+  public ValueParser scalarParserFor(ScalarWriter writer) {
+    return parserFactory().simpleValueParser(scalarListenerFor(writer));
+  }
+
+  protected ElementParser scalarArrayParserFor(ValueParser element) {
+    return parserFactory().scalarArrayValueParser(
+        new SimpleArrayListener(), element);
+  }
+
+  protected ElementParser scalarArrayParserFor(ArrayWriter writer) {
+    return scalarArrayParserFor(scalarParserFor(writer.scalar()));
+  }
+
+  /**
+   * Create a repeated list listener for a scalar value.
+   */
+  protected ElementParser multiDimScalarArrayFor(ObjectWriter writer, int dims) {
+    return buildOuterArrays(writer, dims,
+        innerWriter -> scalarArrayParserFor(innerWriter.array()));
+  }
+
+  /**
+   * Create a map column and its associated object value listener for the
+   * a JSON object value given the value's key.
+   */
+  public ElementParser objectParserFor(FieldDefn fieldDefn) {
+    return objectParserFor(fieldDefn, MetadataUtils.newMap(fieldDefn.key()), null);
+  }
+
+  /**
+   * Create a map column and its associated object value listener for the
+   * given key and optional provided schema.
+   */
+  protected ElementParser objectParserFor(FieldDefn fieldDefn,
+      ColumnMetadata colSchema, TupleMetadata providedSchema) {
+    return objectParserFor(
+            fieldDefn.fieldWriterFor(colSchema).tuple(),
+            providedSchema);
+  }
+
+  /**
+   * Create a map array column and its associated parsers and listeners
+   * for the given column schema and optional provided schema.
+   */
+  protected ElementParser objectArrayParserFor(
+      FieldDefn fieldDefn, ColumnMetadata colSchema, TupleMetadata providedSchema) {
+    return objectArrayParserFor(fieldDefn.fieldWriterFor(colSchema).array(), providedSchema);
+  }
+
+  protected ElementParser objectArrayParserFor(ArrayWriter arrayWriter, TupleMetadata providedSchema) {
+    return parserFactory().arrayValueParser(
+        new StructureArrayListener(arrayWriter),
+        objectParserFor(arrayWriter.tuple(), providedSchema));
+  }
+
+  protected ElementParser objectParserFor(TupleWriter writer, TupleMetadata providedSchema) {
+    return parserFactory().objectValueParser(
+        new TupleParser(loader, writer, providedSchema));
+  }
+
+  /**
+   * Create a repeated list listener for a Map.
+   */
+  public ElementParser multiDimObjectArrayFor(
+      ObjectWriter writer, int dims, TupleMetadata providedSchema) {
+    return buildOuterArrays(writer, dims,
+        innerWriter ->
+          objectArrayParserFor(innerWriter.array(), providedSchema));
+  }
+
+  /**
+   * Create a variant (UNION) column and its associated parser given
+   * a column schema.
+   */
+  protected ElementParser variantParserFor(VariantWriter writer) {
+    return new VariantParser(loader, writer);
+  }
+
+  /**
+   * Create a variant array (LIST) column and its associated parser given
+   * a column schema.
+   */
+  protected ElementParser variantArrayParserFor(ArrayWriter arrayWriter) {
+    return parserFactory().arrayValueParser(
+        new ListArrayListener(arrayWriter),
+        variantParserFor(arrayWriter.variant()));
+  }
+
+  /**
+   * Create a repeated list listener for a variant. Here, the inner
+   * array is provided by a List (which is a repeated Union.)
+   */
+  protected ElementParser multiDimVariantArrayParserFor(
+      ObjectWriter writer, int dims) {
+    return buildOuterArrays(writer, dims,
+        innerWriter -> variantArrayParserFor(innerWriter.array()));
+  }
+
+  /**
+   * Create layers of repeated list listeners around the type-specific
+   * array. If the JSON has three array levels, the outer two are repeated
+   * lists, the inner is type-specific: say an array of {@code BIGINT} or
+   * a map array.
+   */
+  public ElementParser buildOuterArrays(ObjectWriter writer, int dims,
+      Function<ObjectWriter, ElementParser> innerCreator) {
+    ObjectWriter writers[] = new ObjectWriter[dims];
+    writers[0] = writer;
+    for (int i = 1; i < dims; i++) {
+      writers[i] = writers[i-1].array().entry();
+    }
+    ElementParser prevElementParser = innerCreator.apply(writers[dims - 1]);
+    for (int i = dims - 2; i >= 0; i--) {
+      prevElementParser = parserFactory().arrayValueParser(
+          new StructureArrayListener(writers[i].array()), prevElementParser);
+    }
+    return prevElementParser;
+  }
+
+  /**
+   * Build up a repeated list column definition given a specification of the
+   * number of dimensions and the JSON type. Creation of the element type is
+   * via a closure that builds the needed schema.
+   */
+  protected ColumnMetadata repeatedListSchemaFor(String key, int dims,
+      ColumnMetadata innerArray) {
+    ColumnMetadata prev = innerArray;
+    for (int i = 1; i < dims; i++) {
+      prev = MetadataUtils.newRepeatedList(key, prev);
+    }
+    return prev;
+  }
+
+  public ScalarListener scalarListenerFor(ScalarWriter writer) {
+    switch (writer.schema().type()) {
+      case BIGINT:
+        return new BigIntListener(loader, writer);
+      case BIT:
+        return new BooleanListener(loader, writer);
+      case FLOAT4:
+      case FLOAT8:
+        return new DoubleListener(loader, writer);
+      case VARCHAR:
+        return new VarCharListener(loader, writer);
+      case INT:
+      case SMALLINT:
+        return new StrictIntValueListener(loader, writer);
+      case INTERVAL:
+      case INTERVALDAY:
+      case INTERVALYEAR:
+        return new IntervalValueListener(loader, writer);
+      case DATE:
+        return new DateValueListener(loader, writer);
+      case TIME:
+        return new TimeValueListener(loader, writer);
+      case TIMESTAMP:
+        return new TimestampValueListener(loader, writer);
+      case VARBINARY:
+        return new BinaryValueListener(loader, writer);
+      case VARDECIMAL:
+        return new DecimalValueListener(loader, writer);
+      default:
+        throw loader.buildError(
+            UserException.internalError(null)
+              .message("Unsupported JSON reader type: %s",
+                  writer.schema().type().name()));
+    }
+  }
+
+  @Override
+  public ElementParser forceNullResolution(FieldDefn fieldDefn) {
+    return child.forceArrayResolution(fieldDefn);
+  }
+
+  @Override
+  public ElementParser forceArrayResolution(FieldDefn fieldDefn) {
+    return child.forceArrayResolution(fieldDefn);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/EmptyArrayFieldParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/EmptyArrayFieldParser.java
new file mode 100644
index 0000000..fdc5791
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/EmptyArrayFieldParser.java
@@ -0,0 +1,66 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.NullTypeMarker;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.EmptyArrayParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Represents a run of empty arrays for which we have no type information.
+ * Resolves to an actual type when a non-empty array appears. Must resolve
+ * the array even if it contains nulls so we can count the null values.
+ */
+public class EmptyArrayFieldParser extends EmptyArrayParser implements NullTypeMarker {
+
+  private final TupleParser tupleParser;
+
+  public EmptyArrayFieldParser(TupleParser tupleParser, String key) {
+    super(tupleParser.structParser(), key);
+    this.tupleParser = tupleParser;
+    tupleParser.loader().addNullMarker(this);
+  }
+
+  @Override
+  public void forceResolution() {
+    tupleParser.loader().removeNullMarker(this);
+    tupleParser.forceEmptyArrayResolution(key);
+  }
+
+  /**
+   * The column type is now known from context. Create a new array
+   * column, writer and parser to replace this parser.
+   */
+  @Override
+  protected ElementParser resolve(TokenIterator tokenizer) {
+    tupleParser.loader().removeNullMarker(this);
+    if (tokenizer.peek() == JsonToken.START_ARRAY) {
+
+      // The value is [], [ foo, so resolve directly
+      return tupleParser.resolveField(key, tokenizer);
+    } else {
+      // The value is [], foo, so must artificially create the
+      // surrounding array to get a repeated type, which will
+      // then parse the item as one-item array.
+     return tupleParser.resolveArray(key, tokenizer);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldDefn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldDefn.java
new file mode 100644
index 0000000..e1d3238
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldDefn.java
@@ -0,0 +1,135 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.parser.ValueDef;
+import org.apache.drill.exec.store.easy.json.parser.ValueDefFactory;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+/**
+ * Describes a new field within an object. Allows the listener to control
+ * how to handle the field: as unprojected, parsed as a typed field, as
+ * text, as JSON, or as a custom parser.
+ */
+public class FieldDefn {
+
+  private final TupleParser tupleParser;
+  private final String key;
+  private final TokenIterator tokenizer;
+  private ValueDef valueDef;
+  private ColumnMetadata providedColumn;
+
+  public FieldDefn(TupleParser tupleParser, final String key, TokenIterator tokenizer) {
+    this(tupleParser, key, tokenizer, false);
+  }
+
+  public FieldDefn(TupleParser tupleParser, final String key,
+      TokenIterator tokenizer, boolean isArray) {
+    this.tupleParser = tupleParser;
+    this.key = key;
+    this.tokenizer = tokenizer;
+    if (isArray) {
+      valueDef = ValueDefFactory.lookAhead(tokenizer);
+      valueDef = new ValueDef(valueDef.type(), valueDef.dimensions() + 1);
+    }
+  }
+
+  /**
+   * Returns the field name.
+   */
+  public String key() { return key; }
+
+  public TupleParser tupleParser() { return tupleParser; }
+
+  /**
+   * Token stream which allows a custom parser to look ahead
+   * as needed. The caller must "unget" all tokens to leave the
+   * tokenizer at the present location. Note that the underlying
+   * Jackson parser will return text for the last token consumed,
+   * even if tokens are unwound using the token iterator, so do not
+   * look ahead past the first field name or value; on look ahead
+   * over "static" tokens such as object and array start characters.
+   */
+  public TokenIterator tokenizer() { return tokenizer; }
+
+  /**
+   * Returns the parent parser which is needed to construct standard
+   * parsers.
+   */
+  public JsonStructureParser parser() { return tupleParser.structParser(); }
+
+  /**
+   * Looks ahead to guess the field type based on JSON tokens.
+   * While this is helpful, it really only works if the JSON
+   * is structured like a list of tuples, if the initial value is not {@code null},
+   * and if initial arrays are not empty. The structure parser cannot see
+   * into the future beyond the first field value; the value listener for each
+   * field must handle "type-deferral" if needed to handle missing or null
+   * values. That is, type-consistency is a semantic task handled by the listener,
+   * not a syntax task handled by the parser.
+   */
+  public ValueDef lookahead() {
+    Preconditions.checkState(tokenizer != null);
+    if (valueDef == null) {
+      valueDef = ValueDefFactory.lookAhead(tokenizer);
+    }
+    return valueDef;
+  }
+
+  public TupleWriter writer() { return tupleParser.writer(); }
+
+  public ColumnMetadata providedColumn() {
+    if (providedColumn == null) {
+      TupleMetadata tupleSchema = tupleParser.providedSchema();
+      providedColumn = tupleSchema == null ? null : tupleSchema.metadata(key);
+    }
+    return providedColumn;
+  }
+
+  public ColumnMetadata schemaFor(MinorType type, boolean isArray) {
+    return MetadataUtils.newScalar(key, type, mode(isArray));
+  }
+
+  public DataMode mode(boolean isArray) {
+    return isArray ? DataMode.REPEATED : DataMode.OPTIONAL;
+  }
+
+  public ScalarWriter scalarWriterFor(MinorType type, boolean isArray) {
+    return scalarWriterFor(schemaFor(type, isArray));
+  }
+
+  public ScalarWriter scalarWriterFor(ColumnMetadata colSchema) {
+    ObjectWriter writer = fieldWriterFor(colSchema);
+    return colSchema.isArray() ? writer.array().scalar() : writer.scalar();
+  }
+
+  public ObjectWriter fieldWriterFor(ColumnMetadata colSchema) {
+    final int index = writer().addColumn(colSchema);
+    return writer().column(index);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldFactory.java
new file mode 100644
index 0000000..e9a4ab8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/FieldFactory.java
@@ -0,0 +1,50 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+
+/**
+ * Extensible mechanism to build fields for a JSON object (a Drill
+ * row or Map).
+ */
+public interface FieldFactory {
+
+  /**
+   * Create a parser for a field. The caller will add the field
+   * to the parent object.
+   * Called only for projected fields. May add a "deferred"
+   * undefined field if the value type is undefined. Such fields are added
+   * to the underlying row or map at a later time.
+   */
+  ElementParser fieldParser(FieldDefn fieldDefn);
+
+  ElementParser ignoredFieldParser();
+
+  ElementParser forceNullResolution(FieldDefn fieldDefn);
+  ElementParser forceArrayResolution(FieldDefn fieldDefn);
+
+  /**
+   * Internal method which allows a custom parser (such as for
+   * extended types) to provide the scalar parser for a provided
+   * schema.
+   */
+  ValueParser scalarParserFor(FieldDefn fieldDefn, ColumnMetadata colSchema);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/InferredFieldFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/InferredFieldFactory.java
new file mode 100644
index 0000000..2f224e5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/InferredFieldFactory.java
@@ -0,0 +1,245 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.ValueDef;
+import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
+import org.apache.drill.exec.store.easy.json.values.VarCharListener;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Create Drill field listeners based on the observed look-ahead
+ * tokens in JSON.
+ */
+public class InferredFieldFactory extends BaseFieldFactory {
+  protected static final Logger logger = LoggerFactory.getLogger(InferredFieldFactory.class);
+
+  public InferredFieldFactory(JsonLoaderImpl loader) {
+    super(loader);
+  }
+
+  /**
+   * Build a column and its listener based on a look-ahead hint.
+   */
+  @Override
+  public ElementParser fieldParser(FieldDefn fieldDefn) {
+    ValueDef valueDef = fieldDefn.lookahead();
+    if (valueDef.type().isUnknown()) {
+      return parserForUnknown(fieldDefn);
+    } else {
+      return resolveField(fieldDefn);
+    }
+  }
+
+  /**
+   * Create a listener when we don't have type information. For the case
+   * {@code null} appears before other values.
+   */
+  private ElementParser parserForUnknown(FieldDefn fieldDefn) {
+    ValueDef valueDef = fieldDefn.lookahead();
+    if (!valueDef.isArray()) {
+
+      // For the case null appears before other values.
+      return new NullFieldParser(fieldDefn.tupleParser(), fieldDefn.key());
+    } else if (valueDef.dimensions() > 1) {
+
+      // An unknown nested array: [[]], etc. Must guess a type.
+      return forceRepeatedListResolution(fieldDefn);
+    } else if (valueDef.type() == JsonType.NULL) {
+
+      // For the case of [null], must force resolution
+      return forceArrayResolution(fieldDefn);
+    } else {
+
+      // For the case [] appears before other values.
+      return new EmptyArrayFieldParser(fieldDefn.tupleParser(), fieldDefn.key());
+    }
+  }
+
+  private ElementParser forceRepeatedListResolution(FieldDefn fieldDefn) {
+    ColumnMetadata innerSchema = schemaForUnknown(fieldDefn, true);
+    int dims = fieldDefn.lookahead().dimensions();
+    ColumnMetadata fieldSchema = repeatedListSchemaFor(fieldDefn.key(), dims, innerSchema);
+    return buildOuterArrays(
+        fieldDefn.fieldWriterFor(fieldSchema), dims,
+        innerWriter -> scalarArrayParserFor(
+            unknownParserFor(innerWriter.array().scalar())));
+  }
+
+  @Override
+  public ElementParser forceNullResolution(FieldDefn fieldDefn) {
+    logger.warn("Ambiguous type! JSON field {}" +
+        " contains all nulls. Assuming JSON text.", fieldDefn.key());
+    return forceResolution(fieldDefn, false);
+  }
+
+  @Override
+  public ElementParser forceArrayResolution(FieldDefn fieldDefn) {
+    logger.warn("Ambiguous type! JSON field {}" +
+        " contains all empty arrays. Assuming array of JSON text.", fieldDefn.key());
+    return scalarArrayParserFor(forceResolution(fieldDefn, true));
+  }
+
+  private ValueParser forceResolution(FieldDefn fieldDefn, boolean isArray) {
+    return unknownParserFor(
+        fieldDefn.scalarWriterFor(
+            schemaForUnknown(fieldDefn, isArray)));
+  }
+
+  private ColumnMetadata schemaForUnknown(FieldDefn fieldDefn, boolean isArray) {
+    if (loader.options().unknownsAsJson) {
+      return fieldDefn.schemaFor(MinorType.VARCHAR, isArray);
+    } else {
+      return fieldDefn.schemaFor(loader.options().nullType, isArray);
+    }
+  }
+
+  private ValueParser unknownParserFor(ScalarWriter writer) {
+    if (loader.options().unknownsAsJson) {
+      return parserFactory().jsonTextParser(new VarCharListener(loader, writer));
+    } else {
+      return parserFactory().simpleValueParser(scalarListenerFor(writer));
+    }
+  }
+
+  private ElementParser resolveField(FieldDefn fieldDefn) {
+    ValueDef valueDef = fieldDefn.lookahead();
+    Preconditions.checkArgument(!valueDef.type().isUnknown());
+    if (!valueDef.isArray()) {
+      if (valueDef.type().isObject()) {
+        return objectParserFor(fieldDefn);
+      } else {
+        return scalarParserFor(fieldDefn, false);
+      }
+    } else if (valueDef.dimensions() == 1) {
+      if (valueDef.type().isObject()) {
+        return objectArrayParserFor(fieldDefn);
+      } else {
+        return scalarArrayParserFor(scalarParserFor(fieldDefn, true));
+      }
+    } else { // 2+ dimensions
+      if (valueDef.type().isObject()) {
+        return multiDimObjectArrayParserFor(fieldDefn);
+      } else {
+        return multiDimScalarArrayParserFor(fieldDefn);
+      }
+    }
+  }
+
+  public ValueParser scalarParserFor(FieldDefn fieldDefn, boolean isArray) {
+    if (loader.options().allTextMode) {
+      return parserFactory().textValueParser(
+          new VarCharListener(loader,
+              fieldDefn.scalarWriterFor(MinorType.VARCHAR, isArray)));
+    } else {
+      return scalarParserFor(fieldDefn,
+              fieldDefn.schemaFor(scalarTypeFor(fieldDefn), isArray));
+    }
+  }
+
+  /**
+   * Create a multi- (2+) dimensional scalar array from a JSON value description.
+   */
+  private ElementParser multiDimScalarArrayParserFor(FieldDefn fieldDefn) {
+    ColumnMetadata innerSchema = fieldDefn.schemaFor(scalarTypeFor(fieldDefn), true);
+    int dims = fieldDefn.lookahead().dimensions();
+    ColumnMetadata fieldSchema = repeatedListSchemaFor(fieldDefn.key(), dims, innerSchema);
+    return multiDimScalarArrayFor(
+        fieldDefn.fieldWriterFor(fieldSchema), dims);
+  }
+
+  /**
+   * Create a map array column and its associated object array listener
+   * for the given key.
+   */
+  public ElementParser objectArrayParserFor(FieldDefn fieldDefn) {
+    return objectArrayParserFor(fieldDefn, MetadataUtils.newMapArray(fieldDefn.key()), null);
+  }
+
+  /**
+   * Create a RepeatedList which contains (empty) Map objects using the provided
+   * schema. That is, create a multi-dimensional array of maps.
+   * The map fields are created on the fly, optionally using the provided schema.
+   */
+  private ElementParser multiDimObjectArrayParserFor(FieldDefn fieldDefn) {
+    ColumnMetadata innerSchema =  MetadataUtils.newMapArray(fieldDefn.key());
+    int dims = fieldDefn.lookahead().dimensions();
+    ColumnMetadata fieldSchema = repeatedListSchemaFor(fieldDefn.key(), dims, innerSchema);
+    return multiDimObjectArrayFor(fieldDefn.fieldWriterFor(fieldSchema), dims, null);
+  }
+
+  /**
+   * Create a RepeatedList which contains Unions. (Actually, this is an
+   * array of List objects internally.) The variant is variable, it makes no
+   * sense to specify a schema for the variant. Also, omitting the schema
+   * save a large amount of complexity that will likely never be needed.
+   */
+  @SuppressWarnings("unused")
+  private ElementParser repeatedListOfVariantListenerFor(FieldDefn fieldDefn) {
+    ColumnMetadata innerSchema =  MetadataUtils.newVariant(fieldDefn.key(), DataMode.REPEATED);
+    int dims = fieldDefn.lookahead().dimensions();
+    ColumnMetadata fieldSchema = repeatedListSchemaFor(fieldDefn.key(), dims, innerSchema);
+    return multiDimVariantArrayParserFor(fieldDefn.fieldWriterFor(fieldSchema), dims);
+  }
+
+  /**
+   * Convert the JSON type, obtained by looking ahead one token, to a Drill
+   * scalar type. Report an error if the JSON type does not map to a Drill
+   * type (which can occur in a context where we expect a scalar, but got
+   * an object or array.)
+   */
+  private MinorType scalarTypeFor(FieldDefn fieldDefn) {
+    MinorType colType = drillTypeFor(fieldDefn.lookahead().type());
+    if (colType == null) {
+      throw loader().unsupportedJsonTypeException(
+          fieldDefn.key(), fieldDefn.lookahead().type());
+    }
+    return colType;
+  }
+
+  public MinorType drillTypeFor(JsonType type) {
+    if (loader().options().allTextMode) {
+      return MinorType.VARCHAR;
+    }
+    switch (type) {
+    case BOOLEAN:
+      return MinorType.BIT;
+    case FLOAT:
+      return MinorType.FLOAT8;
+    case INTEGER:
+      if (loader().options().readNumbersAsDouble) {
+        return MinorType.FLOAT8;
+      } else {
+        return MinorType.BIGINT;
+      }
+    case STRING:
+      return MinorType.VARCHAR;
+    default:
+      throw new IllegalStateException(type.name());
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderImpl.java
index 69ddfc1..f1e934f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderImpl.java
@@ -31,11 +31,13 @@ import org.apache.drill.exec.physical.resultSet.RowSetLoader;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.store.easy.json.extended.ExtendedTypeFieldFactory;
 import org.apache.drill.exec.store.easy.json.parser.ErrorFactory;
 import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
 import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser.JsonStructureParserBuilder;
 import org.apache.drill.exec.store.easy.json.parser.MessageParser;
 import org.apache.drill.exec.store.easy.json.parser.MessageParser.MessageContextException;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator.RecoverableJsonException;
 import org.apache.drill.exec.store.easy.json.parser.ValueDef;
 import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
 import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
@@ -132,7 +134,7 @@ import com.fasterxml.jackson.core.JsonToken;
  * </ul>
  */
 public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
-  protected static final Logger logger = LoggerFactory.getLogger(JsonLoaderImpl.class);
+  private static final Logger logger = LoggerFactory.getLogger(JsonLoaderImpl.class);
 
   public static class JsonLoaderBuilder {
     private ResultSetLoader rsLoader;
@@ -208,8 +210,8 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
   private final ResultSetLoader rsLoader;
   private final JsonLoaderOptions options;
   private final CustomErrorContext errorContext;
-  private final TupleListener rowListener;
   private final JsonStructureParser parser;
+  private final FieldFactory fieldFactory;
   private boolean eof;
 
   /**
@@ -228,19 +230,37 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
     this.rsLoader = builder.rsLoader;
     this.options = builder.options;
     this.errorContext = builder. errorContext;
-    this.rowListener = new TupleListener(this, rsLoader.writer(), builder.providedSchema);
-    this.parser = new JsonStructureParserBuilder()
+    this.fieldFactory = buildFieldFactory(builder);
+    this.parser = buildParser(builder);
+  }
+
+  private JsonStructureParser buildParser(JsonLoaderBuilder builder) {
+    return new JsonStructureParserBuilder()
             .fromStream(builder.stream)
             .fromReader(builder.reader)
             .options(builder.options)
-            .rootListener(rowListener)
+            .parserFactory(parser ->
+                new TupleParser(parser, JsonLoaderImpl.this, rsLoader.writer(), builder.providedSchema))
             .errorFactory(this)
             .messageParser(builder.messageParser)
             .dataPath(builder.dataPath)
             .build();
   }
 
+  private FieldFactory buildFieldFactory(JsonLoaderBuilder builder) {
+    FieldFactory factory = new InferredFieldFactory(this);
+    if (options.enableExtendedTypes) {
+      factory = new ExtendedTypeFieldFactory(this, factory);
+    }
+    if (builder.providedSchema != null) {
+      factory = new ProvidedFieldFactory(this, factory);
+    }
+    return factory;
+  }
+
   public JsonLoaderOptions options() { return options; }
+  public JsonStructureParser parser() { return parser; }
+  public FieldFactory fieldFactory() { return fieldFactory; }
 
   @Override // JsonLoader
   public boolean readBatch() {
@@ -319,9 +339,13 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
 
   @Override // ErrorFactory
   public RuntimeException structureError(String msg) {
-    throw buildError(
-        UserException.dataReadError()
-          .message(msg));
+    if (options.skipMalformedRecords) {
+      throw new RecoverableJsonException();
+    } else {
+      throw buildError(
+          UserException.dataReadError()
+            .message(msg));
+    }
   }
 
   @Override // ErrorFactory
@@ -341,9 +365,13 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
 
   @Override // ErrorFactory
   public RuntimeException syntaxError(JsonToken token) {
-    throw buildError(
-        UserException.dataReadError()
-          .addContext("Syntax error on token", token.toString()));
+    if (options.skipMalformedRecords) {
+      throw new RecoverableJsonException();
+    } else {
+      throw buildError(
+          UserException.dataReadError()
+            .message("Syntax error on token", token.toString()));
+    }
   }
 
   @Override // ErrorFactory
@@ -354,7 +382,7 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
           .addContext("Recovery attempts", parser.recoverableErrorCount()));
   }
 
-  protected UserException typeConversionError(ColumnMetadata schema, ValueDef valueDef) {
+  public UserException typeConversionError(ColumnMetadata schema, ValueDef valueDef) {
     StringBuilder buf = new StringBuilder()
         .append(valueDef.type().name().toLowerCase());
     if (valueDef.isArray()) {
@@ -365,14 +393,14 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
     return typeConversionError(schema, buf.toString());
   }
 
-  protected UserException typeConversionError(ColumnMetadata schema, String tokenType) {
+  public UserException typeConversionError(ColumnMetadata schema, String tokenType) {
     return buildError(schema,
         UserException.dataReadError()
           .message("Type of JSON token is not compatible with its column")
           .addContext("JSON token type", tokenType));
   }
 
-  protected UserException dataConversionError(ColumnMetadata schema, String tokenType, String value) {
+  public UserException dataConversionError(ColumnMetadata schema, String tokenType, String value) {
     return buildError(schema,
         UserException.dataReadError()
           .message("Type of JSON token is not compatible with its column")
@@ -380,6 +408,12 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
           .addContext("JSON token", value));
   }
 
+  public UserException nullDisallowedError(ColumnMetadata schema) {
+    return buildError(schema,
+        UserException.dataReadError()
+          .message("JSON value \"null\" for a column that does not allow null values"));
+  }
+
   public UserException unsupportedType(ColumnMetadata schema) {
     return buildError(schema,
         UserException.validationError()
@@ -404,7 +438,7 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
           .addContext("On token", e.token.name()));
   }
 
-  protected UserException buildError(ColumnMetadata schema, UserException.Builder builder) {
+  public UserException buildError(ColumnMetadata schema, UserException.Builder builder) {
     return buildError(builder
         .addContext("Column", schema.name())
         .addContext("Column type", schema.typeString()));
@@ -412,12 +446,16 @@ public class JsonLoaderImpl implements JsonLoader, ErrorFactory {
 
   protected UserException buildError(UserException.Builder builder) {
     builder
-      .addContext(errorContext)
-      .addContext("Line", parser.lineNumber())
-      .addContext("Position", parser.columnNumber());
-    String token = parser.token();
-    if (token != null) {
-      builder.addContext("Near token", token);
+      .addContext(errorContext);
+    if (parser != null) {
+      // Parser is not set during bootstrap to find the start of data
+      builder
+        .addContext("Line", parser.lineNumber())
+        .addContext("Position", parser.columnNumber());
+      String token = parser.token();
+      if (token != null) {
+        builder.addContext("Near token", token);
+      }
     }
     return builder.build(logger);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderOptions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderOptions.java
index a8b221d..7e839fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderOptions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/JsonLoaderOptions.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store.easy.json.loader;
 
+import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.store.easy.json.parser.JsonStructureOptions;
@@ -28,6 +29,18 @@ import org.apache.drill.exec.store.easy.json.parser.JsonStructureOptions;
  */
 public class JsonLoaderOptions extends JsonStructureOptions {
 
+  /**
+   * JSON returns values as typed tokens. If {@code allTextMode} is
+   * set, the structure parser converts all scalars (except {@code null})
+   * to text and forwards the values to the listener as text.
+   * Implements Drill's "all-text mode" for JSON.
+   */
+  public boolean allTextMode;
+
+  /**
+   * Forces all numbers to be double, even if the first number that
+   * appears is an integer.
+   */
   public boolean readNumbersAsDouble;
   public boolean unionEnabled;
 
@@ -45,11 +58,35 @@ public class JsonLoaderOptions extends JsonStructureOptions {
    */
   public boolean classicArrayNulls;
 
+  /**
+   * If a field contains all nulls, all empty arrays, or the first
+   * non-empty array contains a null ({@code [ null, ... ]}, then the
+   * default action is to read the column as JSON: literally convert
+   * the values (of any JSON complexity) to JSON string of that structure.
+   * If {@code false}, then uses the default type.
+   */
+  public boolean unknownsAsJson = true;
+
+  /**
+   * Type to use for a field which contains all nulls, or all empty
+   * arrays. Also the default type for an array that starts with the
+   * {@code null} value, if {@link #unknownsAsJson} is {@code false}.
+   */
+  public MinorType nullType = MinorType.VARCHAR;
+
+  /**
+   * Enable support for Mongo-style extended types:<br>
+   * <code> { field: { "$type": value }, ... }<?code>
+   */
+  public boolean enableExtendedTypes;
+
   public JsonLoaderOptions() { }
 
   public JsonLoaderOptions(OptionSet options) {
     super(options);
+    this.allTextMode = options.getBoolean(ExecConstants.JSON_ALL_TEXT_MODE);
     this.readNumbersAsDouble = options.getBoolean(ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE);
     this.unionEnabled = options.getBoolean(ExecConstants.ENABLE_UNION_TYPE_KEY);
+    this.enableExtendedTypes = options.getBoolean(ExecConstants.JSON_EXTENDED_TYPES_KEY);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ListListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ListListener.java
deleted file mode 100644
index 05c65a4..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ListListener.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.ObjectWriter;
-
-/**
- * Listener for the List vector writer. A List in Drill is essentially
- * a repeated Union.
- */
-public class ListListener extends AbstractValueListener {
-
-  private final ObjectWriter listWriter;
-  private final ListArrayListener arrayListener;
-
-  public ListListener(JsonLoaderImpl loader, ObjectWriter listWriter) {
-    super(loader);
-    this.listWriter = listWriter;
-    arrayListener = new ListArrayListener(loader, listWriter.array());
-  }
-
-  @Override
-  public void onNull() { }
-
-  @Override
-  protected ColumnMetadata schema() {
-    return listWriter.schema();
-  }
-
-  @Override
-  public ArrayListener array(ValueDef valueDef) {
-    return arrayListener;
-  }
-
-  private static class ListArrayListener extends AbstractArrayListener {
-
-    private final ArrayWriter listWriter;
-
-    public ListArrayListener(JsonLoaderImpl loader, ArrayWriter listWriter) {
-      super(loader, listWriter.schema(),
-          new VariantListener(loader, listWriter.variant()));
-      this.listWriter = listWriter;
-    }
-
-    @Override
-    public void onElementStart() {
-      // For list, must say that the entry is non-null to
-      // record an empty list. {a: null} vs. {a: []}.
-      listWriter.setNull(false);
-    }
-
-    @Override
-    public ValueListener element(ValueDef valueDef) {
-      return elementListener;
-    }
-
-    @Override
-    public void onElementEnd() {
-      listWriter.save();
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/NullFieldParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/NullFieldParser.java
new file mode 100644
index 0000000..2006eaa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/NullFieldParser.java
@@ -0,0 +1,56 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.NullTypeMarker;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.NullValueParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+
+/**
+ * Parser for a field that contains only nulls. Waits, consuming nulls, until
+ * a non-null value appears, after which this parser is replaced by the
+ * "resolved" parser. The loader will force resolution at the end of the
+ * batch if no actual values are seen before then.
+ */
+public class NullFieldParser extends NullValueParser implements NullTypeMarker {
+
+  private final TupleParser tupleParser;
+
+  public NullFieldParser(TupleParser tupleParser, String key) {
+    super(tupleParser.structParser(), key);
+    this.tupleParser = tupleParser;
+    tupleParser.loader().addNullMarker(this);
+  }
+
+  @Override
+  public void forceResolution() {
+    tupleParser.loader().removeNullMarker(this);
+    tupleParser.forceNullResolution(key);
+  }
+
+  /**
+   * The column type is now known from context. Create a new scalar
+   * column, writer and parser to replace this parser.
+   */
+  @Override
+  protected ElementParser resolve(TokenIterator tokenizer) {
+    tupleParser.loader().removeNullMarker(this);
+    return tupleParser.resolveField(key, tokenizer);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ProvidedFieldFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ProvidedFieldFactory.java
new file mode 100644
index 0000000..f634608
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ProvidedFieldFactory.java
@@ -0,0 +1,176 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.FieldParserFactory;
+import org.apache.drill.exec.store.easy.json.parser.ValueParser;
+import org.apache.drill.exec.store.easy.json.values.VarCharListener;
+import org.apache.drill.exec.vector.accessor.ObjectWriter;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+
+/**
+ * Create a Drill field listener based on a provided schema. The schema
+ * takes precedence over the JSON syntax: the schema is expected to
+ * accurately describe what will occur for this field in the JSON
+ * input.
+ */
+public class ProvidedFieldFactory extends BaseFieldFactory {
+
+  public ProvidedFieldFactory(JsonLoaderImpl loader, FieldFactory child) {
+    super(loader, child);
+  }
+
+  /**
+   * Build a column and its listener based a provided schema.
+   * The user is responsible to ensure that the provided schema
+   * accurately reflects the structure of the JSON being parsed.
+   */
+  @Override
+  public ElementParser fieldParser(FieldDefn fieldDefn) {
+    if (fieldDefn.providedColumn() == null) {
+      return child.fieldParser(fieldDefn);
+    } else {
+      return parserFor(fieldDefn);
+    }
+  }
+
+  public ElementParser parserFor(FieldDefn fieldDefn) {
+    ColumnMetadata providedCol = fieldDefn.providedColumn();
+    switch (providedCol.structureType()) {
+      case PRIMITIVE:
+        return primitiveParserFor(fieldDefn);
+      case TUPLE:
+        return objectParserForSchema(fieldDefn);
+      case VARIANT:
+        return variantParserForSchema(fieldDefn);
+      case MULTI_ARRAY:
+        return multiDimArrayParserForSchema(fieldDefn);
+      default:
+        throw loader().unsupportedType(providedCol);
+    }
+  }
+
+  private ElementParser primitiveParserFor(FieldDefn fieldDefn) {
+    ColumnMetadata providedCol = fieldDefn.providedColumn();
+    if (providedCol.type() == MinorType.VARCHAR) {
+      return stringParserFor(fieldDefn);
+    } else {
+      return basicParserFor(fieldDefn);
+    }
+  }
+
+  private ElementParser basicParserFor(FieldDefn fieldDefn) {
+
+    // Delegate parser creation downward: will be done by the extended
+    // types factory, if present, else by inferred field factory.
+    ColumnMetadata colSchema = fieldDefn.providedColumn().copy();
+    ValueParser scalarParser = child.scalarParserFor(fieldDefn, colSchema);
+    if (colSchema.isArray()) {
+      return scalarArrayParserFor(scalarParser);
+    } else {
+      return scalarParser;
+    }
+  }
+
+  private ElementParser stringParserFor(FieldDefn fieldDefn) {
+    String mode = fieldDefn.providedColumn().property(JsonLoader.JSON_MODE);
+    if (mode == null) {
+      return basicParserFor(fieldDefn);
+    }
+    FieldParserFactory parserFactory = parserFactory();
+    switch (mode) {
+      case JsonLoader.JSON_TEXT_MODE:
+        return parserFactory.textValueParser(varCharListenerFor(fieldDefn));
+      case JsonLoader.JSON_LITERAL_MODE:
+        return parserFactory.jsonTextParser(varCharListenerFor(fieldDefn));
+      default:
+        return basicParserFor(fieldDefn);
+    }
+  }
+
+  private VarCharListener varCharListenerFor(FieldDefn fieldDefn) {
+    return new VarCharListener(loader,
+        fieldDefn.scalarWriterFor(fieldDefn.providedColumn().copy()));
+  }
+
+  private ElementParser objectParserForSchema(FieldDefn fieldDefn) {
+    ColumnMetadata providedCol = fieldDefn.providedColumn();
+
+    // Propagate the provided map schema into the object
+    // listener as a provided tuple schema.
+    ColumnMetadata colSchema = providedCol.cloneEmpty();
+    TupleMetadata providedSchema = providedCol.tupleSchema();
+    if (providedCol.isArray()) {
+      return objectArrayParserFor(fieldDefn, colSchema, providedSchema);
+    } else {
+      return objectParserFor(fieldDefn, colSchema, providedSchema);
+    }
+  }
+
+  /**
+   * Create a repeated list column and its multiple levels of inner structure
+   * from a provided schema. Repeated lists can nest to any number of levels to
+   * provide any number of dimensions. In general, if an array is <i>n</i>-dimensional,
+   * then there are <i>n</i>-1 repeated lists with some array type as the
+   * innermost dimension.
+   */
+  private ElementParser multiDimArrayParserForSchema(FieldDefn fieldDefn) {
+    // Parse the stack of repeated lists to count the "outer" dimensions and
+    // to locate the innermost array (the "list" which is "repeated").
+    int dims = 1; // For inner array
+    ColumnMetadata elementSchema = fieldDefn.providedColumn();
+    while (MetadataUtils.isRepeatedList(elementSchema)) {
+      dims++;
+      elementSchema = elementSchema.childSchema();
+      Preconditions.checkArgument(elementSchema != null);
+    }
+
+    ColumnMetadata colSchema = repeatedListSchemaFor(fieldDefn.key(), dims,
+        elementSchema.cloneEmpty());
+    ObjectWriter fieldWriter = fieldDefn.fieldWriterFor(colSchema);
+    switch (elementSchema.structureType()) {
+      case PRIMITIVE:
+        return multiDimScalarArrayFor(fieldWriter, dims);
+      case TUPLE:
+        return multiDimObjectArrayFor(fieldWriter,
+            dims, elementSchema.tupleSchema());
+      case VARIANT:
+        return multiDimVariantArrayParserFor(fieldWriter, dims);
+      default:
+        throw loader().unsupportedType(fieldDefn.providedColumn());
+    }
+  }
+
+  private ElementParser variantParserForSchema(FieldDefn fieldDefn) {
+    // A variant can contain multiple types. The schema does not
+    // declare the types; rather they are discovered by the reader.
+    // That is, there is no VARIANT<INT, DOUBLE>, there is just VARIANT.
+    ColumnMetadata colSchema = fieldDefn.providedColumn().cloneEmpty();
+    ObjectWriter fieldWriter = fieldDefn.fieldWriterFor(colSchema);
+    if (colSchema.isArray()) {
+      return variantArrayParserFor(fieldWriter.array());
+    } else {
+      return variantParserFor(fieldWriter.variant());
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/RepeatedListValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/RepeatedListValueListener.java
deleted file mode 100644
index ba1647d..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/RepeatedListValueListener.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import java.util.function.Function;
-
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ObjectArrayListener;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ScalarArrayListener;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ObjectValueListener;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.ObjectWriter;
-
-/**
- * Represents a JSON value that holds a RepeatedList (2D array) value.
- * The structure is:
- * <ul>
- * <li>Value - {@code RepeatedListValueListener}</li>
- * <li>Array - {@code RepeatedArrayListener}</li>
- * <li>Value - {@code RepeatedListElementListener} or
- * {@code ListListener}</li>
- * <li>Array - Depends on type</li>
- * <li>Value - Depends on type</li>
- * <li>Object - If a repeated list of maps</li>
- * </ul>
- */
-public class RepeatedListValueListener extends AbstractValueListener {
-
-  private final ObjectWriter repeatedListWriter;
-  private final RepeatedArrayListener outerArrayListener;
-
-  private RepeatedListValueListener(JsonLoaderImpl loader, ObjectWriter writer,
-      ValueListener elementListener) {
-    this(loader,  writer,
-        new RepeatedArrayListener(loader, writer.schema(),
-            writer.array(), elementListener));
-  }
-
-  private RepeatedListValueListener(JsonLoaderImpl loader, ObjectWriter writer,
-      RepeatedArrayListener outerArrayListener) {
-    super(loader);
-    this.repeatedListWriter = writer;
-    this.outerArrayListener = outerArrayListener;
-  }
-
-  /**
-   * Create a repeated list listener for a scalar value.
-   */
-  public static ValueListener multiDimScalarArrayFor(JsonLoaderImpl loader, ObjectWriter writer, int dims) {
-    return buildOuterArrays(loader, writer, dims,
-        innerWriter ->
-          new ScalarArrayListener(loader, innerWriter.schema(),
-              ScalarListener.listenerFor(loader, innerWriter))
-        );
-  }
-
-  /**
-   * Create a repeated list listener for a Map.
-   */
-  public static ValueListener multiDimObjectArrayFor(JsonLoaderImpl loader,
-      ObjectWriter writer, int dims, TupleMetadata providedSchema) {
-    return buildOuterArrays(loader, writer, dims,
-        innerWriter ->
-          new ObjectArrayListener(loader, innerWriter.array(),
-              new ObjectValueListener(loader, innerWriter.array().entry().schema(),
-                  new TupleListener(loader, innerWriter.array().tuple(), providedSchema))));
-  }
-
-  /**
-   * Create layers of repeated list listeners around the type-specific
-   * array. If the JSON has three array levels, the outer two are repeated
-   * lists, the inner is type-specific: say an array of {@code BIGINT} or
-   * a map array.
-   */
-  public static ValueListener buildOuterArrays(JsonLoaderImpl loader, ObjectWriter writer, int dims,
-      Function<ObjectWriter, ArrayListener> innerCreator) {
-    ColumnMetadata colSchema = writer.schema();
-    ObjectWriter writers[] = new ObjectWriter[dims];
-    writers[0] = writer;
-    for (int i = 1; i < dims; i++) {
-      writers[i] = writers[i-1].array().entry();
-    }
-    ArrayListener prevArrayListener = innerCreator.apply(writers[dims - 1]);
-    RepeatedArrayListener innerArrayListener = null;
-    for (int i = dims - 2; i >= 0; i--) {
-      innerArrayListener = new RepeatedArrayListener(loader, colSchema,
-          writers[i].array(),
-          new RepeatedListElementListener(loader, colSchema,
-              writers[i+1].array(), prevArrayListener));
-      prevArrayListener = innerArrayListener;
-    }
-    return new RepeatedListValueListener(loader, writer, innerArrayListener);
-  }
-
-  /**
-   * Create a repeated list listener for a variant. Here, the inner
-   * array is provided by a List (which is a repeated Union.)
-   */
-  public static ValueListener repeatedVariantListFor(JsonLoaderImpl loader,
-      ObjectWriter writer) {
-    return new RepeatedListValueListener(loader, writer,
-        new ListListener(loader, writer.array().entry()));
-  }
-
-  @Override
-  public ArrayListener array(ValueDef valueDef) {
-    return outerArrayListener;
-  }
-
-  @Override
-  public void onNull() { }
-
-  @Override
-  protected ColumnMetadata schema() {
-    return repeatedListWriter.schema();
-  }
-
-  /**
-   * Represents the outer array for a repeated (2D) list
-   */
-  private static class RepeatedArrayListener extends AbstractArrayListener {
-
-    private final ArrayWriter outerArrayWriter;
-
-    public RepeatedArrayListener(JsonLoaderImpl loader,
-        ColumnMetadata colMetadata, ArrayWriter outerArrayWriter,
-        ValueListener outerValue) {
-      super(loader, colMetadata, outerValue);
-      this.outerArrayWriter = outerArrayWriter;
-    }
-
-    @Override
-    public ValueListener element(ValueDef valueDef) {
-      return elementListener;
-    }
-
-    @Override
-    public void onElementEnd() {
-      outerArrayWriter.save();
-    }
-  }
-
-  /**
-   * Represents each item in the outer array of a RepeatedList. Such elements should
-   * only be arrays. However, Drill is forgiving if the value happens to be null, which
-   * is defined to be the same as an empty inner array.
-   */
-  private static class RepeatedListElementListener extends AbstractValueListener {
-
-    private final ColumnMetadata colMetadata;
-    private final ArrayListener innerArrayListener;
-    private final ArrayWriter innerArrayWriter;
-
-    public RepeatedListElementListener(JsonLoaderImpl loader, ColumnMetadata colMetadata,
-        ArrayWriter innerArrayWriter, ArrayListener innerArrayListener) {
-      super(loader);
-      this.colMetadata = colMetadata;
-      this.innerArrayListener = innerArrayListener;
-      this.innerArrayWriter = innerArrayWriter;
-    }
-
-    @Override
-    public ArrayListener array(ValueDef valueDef) {
-      return innerArrayListener;
-    }
-
-    @Override
-    public void onNull() {
-      innerArrayWriter.save();
-    }
-
-    @Override
-    protected ColumnMetadata schema() {
-      return colMetadata;
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ScalarListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ScalarListener.java
deleted file mode 100644
index 9c7381a..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/ScalarListener.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.vector.accessor.ObjectType;
-import org.apache.drill.exec.vector.accessor.ObjectWriter;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
-import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
-
-/**
- * Base class for scalar field listeners
- */
-public abstract class ScalarListener extends AbstractValueListener {
-
-  protected final ScalarWriter writer;
-  protected final boolean isArray;
-
-  public ScalarListener(JsonLoaderImpl loader, ScalarWriter writer) {
-    super(loader);
-    this.writer = writer;
-    ColumnMetadata colSchema = writer.schema();
-    isArray = colSchema.isArray();
-  }
-
-  public static ScalarListener listenerFor(JsonLoaderImpl loader, ObjectWriter colWriter) {
-    ScalarWriter writer = colWriter.type() == ObjectType.ARRAY ?
-        colWriter.array().scalar() : colWriter.scalar();
-    switch (writer.schema().type()) {
-      case BIGINT:
-        return new BigIntListener(loader, writer);
-      case BIT:
-        return new BooleanListener(loader, writer);
-      case FLOAT8:
-        return new DoubleListener(loader, writer);
-      case VARCHAR:
-        return new VarCharListener(loader, writer);
-      case DATE:
-      case FLOAT4:
-      case INT:
-      case INTERVAL:
-      case INTERVALDAY:
-      case INTERVALYEAR:
-      case SMALLINT:
-      case TIME:
-      case TIMESTAMP:
-      case VARBINARY:
-      case VARDECIMAL:
-        // TODO: Implement conversions for above
-      default:
-        throw loader.buildError(
-            UserException.internalError(null)
-              .message("Unsupported JSON reader type: %s",
-                  writer.schema().type().name()));
-    }
-  }
-
-  @Override
-  public ColumnMetadata schema() { return writer.schema(); }
-
-  @Override
-  public void onNull() {
-    setNull();
-  }
-
-  protected void setNull() {
-    try {
-      if (isArray) {
-        setArrayNull();
-      } else {
-        writer.setNull();
-      }
-    } catch (UnsupportedConversionError e) {
-      throw loader.buildError(schema(),
-          UserException.dataReadError()
-            .message("Null value encountered in JSON input where Drill does not allow nulls."));
-    }
-  }
-
-  protected abstract void setArrayNull();
-
-  @Override
-  public ArrayListener array(ValueDef valueDef) {
-    if (isArray) {
-      valueDef = new ValueDef(valueDef.type(), valueDef.dimensions() + 1);
-    }
-    throw loader.typeConversionError(schema(), valueDef);
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/SimpleArrayListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/SimpleArrayListener.java
new file mode 100644
index 0000000..35b0fc2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/SimpleArrayListener.java
@@ -0,0 +1,68 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
+import org.apache.drill.exec.vector.accessor.ArrayWriter;
+
+/**
+ * Base class for scalar and object arrays. Represents the array
+ * behavior of a field.
+ */
+public class SimpleArrayListener implements ArrayListener {
+
+  @Override
+  public void onStart() { }
+
+  @Override
+  public void onElementStart() { }
+
+  @Override
+  public void onElementEnd() { }
+
+  @Override
+  public void onEnd() { }
+
+  public static class StructureArrayListener extends SimpleArrayListener {
+
+    protected final ArrayWriter arrayWriter;
+
+    public StructureArrayListener(ArrayWriter arrayWriter) {
+      this.arrayWriter = arrayWriter;
+    }
+
+    @Override
+    public void onElementEnd() {
+      arrayWriter.save();
+    }
+  }
+
+  public static class ListArrayListener extends StructureArrayListener {
+
+    public ListArrayListener(ArrayWriter listWriter) {
+      super(listWriter);
+    }
+
+    @Override
+    public void onElementStart() {
+      // For list, must say that the entry is non-null to
+      // record an empty list. {a: null} vs. {a: []}.
+      arrayWriter.setNull(false);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/StructuredValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/StructuredValueListener.java
deleted file mode 100644
index a0835a8..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/StructuredValueListener.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ObjectArrayListener;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ScalarArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-
-/**
- * Base class for structured value listeners: arrays and objects.
- * Contains the concrete implementations as nested static classes.
- */
-public abstract class StructuredValueListener extends AbstractValueListener {
-
-  private final ColumnMetadata colSchema;
-
-  public StructuredValueListener(JsonLoaderImpl loader, ColumnMetadata colSchema) {
-    super(loader);
-    this.colSchema = colSchema;
-  }
-
-  @Override
-  public ColumnMetadata schema() { return colSchema; }
-
-  // Ignore array nulls: {a: null} is the same as omitting
-  // array column a: an array of zero elements
-  @Override
-  public void onNull() { }
-
-  /**
-   * Abstract base class for array values which hold a nested array
-   * listener.
-   */
-  public static abstract class ArrayValueListener extends StructuredValueListener {
-
-    protected final AbstractArrayListener arrayListener;
-
-    public ArrayValueListener(JsonLoaderImpl loader, ColumnMetadata colSchema, AbstractArrayListener arrayListener) {
-      super(loader, colSchema);
-      this.arrayListener = arrayListener;
-    }
-
-    public AbstractArrayListener arrayListener() { return arrayListener; }
-
-    public ValueListener elementListener() { return arrayListener.elementListener(); }
-  }
-
-  /**
-   * Value listener for a scalar array (Drill repeated primitive).
-   * Maps null values for the entire array to an empty array.
-   * Maps a scalar to an array with a single value.
-   */
-  public static class ScalarArrayValueListener extends ArrayValueListener {
-
-    public ScalarArrayValueListener(JsonLoaderImpl loader, ColumnMetadata colSchema, ScalarArrayListener arrayListener) {
-      super(loader, colSchema, arrayListener);
-    }
-
-    @Override
-    public ArrayListener array(ValueDef valueDef) {
-      Preconditions.checkArgument(valueDef.dimensions() == 1);
-      return arrayListener;
-    }
-
-    @Override
-    public void onBoolean(boolean value) {
-      elementListener().onBoolean(value);
-    }
-
-    @Override
-    public void onInt(long value) {
-      elementListener().onInt(value);
-    }
-
-    @Override
-    public void onFloat(double value) {
-      elementListener().onFloat(value);
-    }
-
-    @Override
-    public void onString(String value) {
-      elementListener().onString(value);
-    }
-  }
-
-  /**
-   * Value listener for object (MAP) values.
-   */
-  public static class ObjectValueListener extends StructuredValueListener {
-
-    private final ObjectListener tupleListener;
-
-    public ObjectValueListener(JsonLoaderImpl loader, ColumnMetadata colSchema, ObjectListener tupleListener) {
-      super(loader, colSchema);
-      this.tupleListener = tupleListener;
-    }
-
-    @Override
-    public ObjectListener object() {
-      return tupleListener;
-    }
-  }
-
-  /**
-   * Value listener for object array (repeated MAP) values.
-   */
-  public static class ObjectArrayValueListener extends ArrayValueListener {
-
-    public ObjectArrayValueListener(JsonLoaderImpl loader,
-        ColumnMetadata colSchema, ObjectArrayListener arrayListener) {
-      super(loader, colSchema, arrayListener);
-     }
-
-    @Override
-    public ArrayListener array(ValueDef valueDef) {
-      Preconditions.checkArgument(valueDef.dimensions() == 1);
-      // Called with a provided schema where the initial array
-      // value is empty.
-      Preconditions.checkArgument(!valueDef.type().isScalar());
-      return arrayListener;
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleListener.java
deleted file mode 100644
index 493bea9..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleListener.java
+++ /dev/null
@@ -1,525 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.record.metadata.MetadataUtils;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ObjectArrayListener;
-import org.apache.drill.exec.store.easy.json.loader.AbstractArrayListener.ScalarArrayListener;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ArrayValueListener;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ObjectArrayValueListener;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ObjectValueListener;
-import org.apache.drill.exec.store.easy.json.loader.StructuredValueListener.ScalarArrayValueListener;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.apache.drill.exec.vector.accessor.ArrayWriter;
-import org.apache.drill.exec.vector.accessor.ObjectWriter;
-import org.apache.drill.exec.vector.accessor.TupleWriter;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-
-/**
- * Accepts { name : value ... }
- * <p>
- * The structure parser maintains a map of known fields. Each time a
- * field is parsed, looks up the field in the map. If not found, the parser
- * looks ahead to find a value token, if any, and calls this class to add
- * a new column. This class creates a column writer based either on the
- * type provided in a provided schema, or inferred from the JSON token.
- * <p>
- * As it turns out, most of the semantic action occurs at the tuple level:
- * that is where fields are defined, types inferred, and projection is
- * computed.
- *
- * <h4>Nulls</h4>
- *
- * Much code here deals with null types, especially leading nulls, leading
- * empty arrays, and so on. The object parser creates a parser for each
- * value; a parser which "does the right thing" based on the data type.
- * For example, for a Boolean, the parser recognizes {@code true},
- * {@code false} and {@code null}.
- * <p>
- * But what happens if the first value for a field is {@code null}? We
- * don't know what kind of parser to create because we don't have a schema.
- * Instead, we have to create a temporary placeholder parser that will consume
- * nulls, waiting for a real type to show itself. Once that type appears, the
- * null parser can replace itself with the correct form. Each vector's
- * "fill empties" logic will back-fill the newly created vector with nulls
- * for prior rows.
- * <p>
- * Two null parsers are needed: one when we see an empty list, and one for
- * when we only see {@code null}. The one for {@code null{@code  must morph into
- * the one for empty lists if we see:<br>
- * {@code {a: null} {a: [ ]  }}<br>
- * <p>
- * If we get all the way through the batch, but have still not seen a type,
- * then we have to guess. A prototype type system can tell us, otherwise we
- * guess {@code VARCHAR}. ({@code VARCHAR} is the right choice for all-text
- * mode, it is as good a guess as any for other cases.)
- *
- * <h4>Projection List Hints</h4>
- *
- * To help, we consult the projection list, if any, for a column. If the
- * projection is of the form {@code a[0]}, we know the column had better
- * be an array. Similarly, if the projection list has {@code b.c}, then
- * {@code b} had better be an object.
- *
- * <h4>Array Handling</h4>
- *
- * The code here handles arrays in two ways. JSON normally uses the
- * {@code LIST} type. But, that can be expensive if lists are
- * well-behaved. So, the code here also implements arrays using the
- * classic {@code REPEATED} types. The repeated type option is disabled
- * by default. It can be enabled, for efficiency, if Drill ever supports
- * a JSON schema. If an array is well-behaved, mark that column as able
- * to use a repeated type.
- *
- * <h4>Ambiguous Types</h4>
- *
- * JSON nulls are untyped. A run of nulls does not tell us what type will
- * eventually appear. The best solution is to provide a schema. Without a
- * schema, the code is forgiving: defers selection of the column type until
- * the first non-null value (or, forces a type at the end of the batch.)
- * <p>
- * For scalars the pattern is: <code>{a: null} {a: "foo"}</code>. Type
- * selection happens on the value {@code "foo"}.
- * <p>
- * For arrays, the pattern is: <code>{a: []} {a: ["foo"]}</code>. Type
- * selection happens on the first array element. Note that type selection
- * must happen on the first element, even if tha element is null (which,
- * as we just said, ambiguous.)
- * <p>
- * If we are forced to pick a type (because we hit the end of a batch, or
- * we see {@code [null]}, then we pick {@code VARCHAR} as we allow any
- * scalar to be converted to {@code VARCHAR}. This helps for a single-file
- * query, but not if multiple fragments each make their own (inconsistent)
- * decisions. Only a schema provides a consistent answer.
- */
-public class TupleListener implements ObjectListener {
-
-  protected final JsonLoaderImpl loader;
-  protected final TupleWriter tupleWriter;
-  private final TupleMetadata providedSchema;
-
-  public TupleListener(JsonLoaderImpl loader, TupleWriter tupleWriter, TupleMetadata providedSchema) {
-    this.loader = loader;
-    this.tupleWriter = tupleWriter;
-    this.providedSchema = providedSchema;
-  }
-
-  public JsonLoaderImpl loader() { return loader; }
-
-  @Override
-  public void onStart() { }
-
-  @Override
-  public void onEnd() { }
-
-  @Override
-  public FieldType fieldType(String key) {
-    if (!tupleWriter.isProjected(key)) {
-      return FieldType.IGNORE;
-    }
-    ColumnMetadata providedCol = providedColumn(key);
-    if (providedCol == null) {
-      return FieldType.TYPED;
-    }
-    String mode = providedCol.property(JsonLoader.JSON_MODE);
-    if (mode == null) {
-      return FieldType.TYPED;
-    }
-    switch (mode) {
-      case JsonLoader.JSON_TEXT_MODE:
-        return FieldType.TEXT;
-      case JsonLoader.JSON_LITERAL_MODE:
-        return FieldType.JSON;
-      default:
-        return FieldType.TYPED;
-    }
-  }
-
-  /**
-   * Add a field not seen before. If a schema is provided, use the provided
-   * column schema to define the column. Else, build the column based on the
-   * look-ahead hints provided by the structure parser.
-   */
-  @Override
-  public ValueListener addField(String key, ValueDef valueDef) {
-    ColumnMetadata providedCol = providedColumn(key);
-    if (providedCol != null) {
-      return listenerForSchema(providedCol);
-    } else {
-      return listenerForValue(key, valueDef);
-    }
-  }
-
-  public ColumnMetadata providedColumn(String key) {
-    return providedSchema == null ? null : providedSchema.metadata(key);
-  }
-
-  /**
-   * Build a column and its listener based a provided schema.
-   * The user is responsible to ensure that the provided schema
-   * accurately reflects the structure of the JSON being parsed.
-   */
-  private ValueListener listenerForSchema(ColumnMetadata providedCol) {
-    switch (providedCol.structureType()) {
-
-      case PRIMITIVE: {
-        ColumnMetadata colSchema = providedCol.copy();
-        if (providedCol.isArray()) {
-          return scalarArrayListenerFor(colSchema);
-        } else {
-          return scalarListenerFor(colSchema);
-        }
-      }
-
-      case TUPLE: {
-        // Propagate the provided map schema into the object
-        // listener as a provided tuple schema.
-        ColumnMetadata colSchema = providedCol.cloneEmpty();
-        TupleMetadata providedSchema = providedCol.tupleSchema();
-        if (providedCol.isArray()) {
-          return objectArrayListenerFor(colSchema, providedSchema);
-        } else {
-          return objectListenerFor(colSchema, providedSchema);
-        }
-      }
-
-      case VARIANT: {
-        // A variant can contain multiple types. The schema does not
-        // declare the types; rather they are discovered by the reader.
-        // That is, there is no VARIANT<INT, DOUBLE>, there is just VARIANT.
-        ColumnMetadata colSchema = providedCol.cloneEmpty();
-        if (providedCol.isArray()) {
-          return variantArrayListenerFor(colSchema);
-        } else {
-          return variantListenerFor(colSchema);
-        }
-      }
-
-      case MULTI_ARRAY:
-        return multiDimArrayListenerForSchema(providedCol);
-
-      default:
-        throw loader.unsupportedType(providedCol);
-    }
-  }
-
-  /**
-   * Build a column and its listener based on a look-ahead hint.
-   */
-  protected ValueListener listenerForValue(String key, ValueDef valueDef) {
-    if (!valueDef.isArray()) {
-      if (valueDef.type().isUnknown()) {
-        return unknownListenerFor(key);
-      } else if (valueDef.type().isObject()) {
-        return objectListenerForValue(key);
-      } else {
-        return scalarListenerForValue(key, valueDef.type());
-      }
-    } else if (valueDef.dimensions() == 1) {
-      if (valueDef.type().isUnknown()) {
-        return unknownArrayListenerFor(key, valueDef);
-      } else if (valueDef.type().isObject()) {
-        return objectArrayListenerForValue(key);
-      } else {
-        return scalarArrayListenerForValue(key, valueDef.type());
-      }
-    } else {
-      if (valueDef.type().isUnknown()) {
-        return unknownArrayListenerFor(key, valueDef);
-      } else if (valueDef.type().isObject()) {
-        return multiDimObjectArrayListenerForValue(key, valueDef);
-      } else {
-        return multiDimScalarArrayListenerForValue(key, valueDef);
-      }
-    }
-  }
-
-  /**
-   * Create a scalar column and listener given the definition of a JSON
-   * scalar value.
-   */
-  public ScalarListener scalarListenerForValue(String key, JsonType jsonType) {
-    return scalarListenerFor(MetadataUtils.newScalar(key,
-        Types.optional(scalarTypeFor(key, jsonType))));
-  }
-
-  /**
-   * Create a scalar column and listener given the column schema.
-   */
-  public ScalarListener scalarListenerFor(ColumnMetadata colSchema) {
-    return ScalarListener.listenerFor(loader, addFieldWriter(colSchema));
-  }
-
-  /**
-   * Create a scalar array column and listener given the definition of a JSON
-   * array of scalars.
-   */
-  public ArrayValueListener scalarArrayListenerForValue(String key, JsonType jsonType) {
-    return scalarArrayListenerFor(MetadataUtils.newScalar(key,
-        Types.repeated(scalarTypeFor(key, jsonType))));
-  }
-
-  /**
-   * Create a multi- (2+) dimensional scalar array from a JSON value description.
-   */
-  private ValueListener multiDimScalarArrayListenerForValue(String key, ValueDef valueDef) {
-    return multiDimScalarArrayListenerFor(
-        repeatedListSchemaFor(key, valueDef.dimensions(),
-            MetadataUtils.newScalar(key, scalarTypeFor(key, valueDef.type()), DataMode.REPEATED)),
-        valueDef.dimensions());
-  }
-
-  /**
-   * Create a multi- (2+) dimensional scalar array from a column schema and dimension
-   * count hint.
-   */
-  private ValueListener multiDimScalarArrayListenerFor(ColumnMetadata colSchema, int dims) {
-    return RepeatedListValueListener.multiDimScalarArrayFor(loader,
-        addFieldWriter(colSchema), dims);
-  }
-
-  /**
-   * Create a scalar array column and array listener for the given column
-   * schema.
-   */
-  public ArrayValueListener scalarArrayListenerFor(ColumnMetadata colSchema) {
-    return new ScalarArrayValueListener(loader, colSchema,
-        new ScalarArrayListener(loader, colSchema,
-            scalarListenerFor(colSchema)));
-  }
-
-  /**
-   * Create a map column and its associated object value listener for the
-   * a JSON object value given the value's key.
-   */
-  public ObjectValueListener objectListenerForValue(String key) {
-    ColumnMetadata colSchema = MetadataUtils.newMap(key);
-    return objectListenerFor(colSchema, colSchema.tupleSchema());
-  }
-
-  /**
-   * Create a map column and its associated object value listener for the
-   * given key and optional provided schema.
-   */
-  public ObjectValueListener objectListenerFor(ColumnMetadata colSchema, TupleMetadata providedSchema) {
-    return new ObjectValueListener(loader, colSchema,
-        new TupleListener(loader, addFieldWriter(colSchema).tuple(),
-            providedSchema));
-  }
-
-  /**
-   * Create a map array column and its associated object array listener
-   * for the given key.
-   */
-  public ArrayValueListener objectArrayListenerForValue(String key) {
-    ColumnMetadata colSchema = MetadataUtils.newMapArray(key);
-    return objectArrayListenerFor(colSchema, colSchema.tupleSchema());
-  }
-
-  /**
-   * Create a map array column and its associated object array listener
-   * for the given column schema and optional provided schema.
-   */
-  public ArrayValueListener objectArrayListenerFor(
-      ColumnMetadata colSchema, TupleMetadata providedSchema) {
-    ArrayWriter arrayWriter = addFieldWriter(colSchema).array();
-    return new ObjectArrayValueListener(loader, colSchema,
-        new ObjectArrayListener(loader, arrayWriter,
-            new ObjectValueListener(loader, colSchema,
-                new TupleListener(loader, arrayWriter.tuple(), providedSchema))));
-  }
-
-  /**
-   * Create a RepeatedList which contains (empty) Map objects using the provided
-   * schema. That is, create a multi-dimensional array of maps.
-   * The map fields are created on the fly, optionally using the provided schema.
-   */
-  private ValueListener multiDimObjectArrayListenerForValue(String key, ValueDef valueDef) {
-    return multiDimObjectArrayListenerFor(
-        repeatedListSchemaFor(key, valueDef.dimensions(),
-            MetadataUtils.newMapArray(key)),
-        valueDef.dimensions(), null);
-  }
-
-  /**
-   * Create a multi- (2+) dimensional scalar array from a column schema, dimension
-   * count hint, and optional provided schema.
-   */
-  private ValueListener multiDimObjectArrayListenerFor(ColumnMetadata colSchema,
-      int dims, TupleMetadata providedSchema) {
-    return RepeatedListValueListener.multiDimObjectArrayFor(loader,
-        addFieldWriter(colSchema), dims, providedSchema);
-  }
-
-  /**
-   * Create a variant (UNION) column and its associated listener given
-   * a column schema.
-   */
-  private ValueListener variantListenerFor(ColumnMetadata colSchema) {
-    return new VariantListener(loader, addFieldWriter(colSchema).variant());
-  }
-
-  /**
-   * Create a variant array (LIST) column and its associated listener given
-   * a column schema.
-   */
-  private ValueListener variantArrayListenerFor(ColumnMetadata colSchema) {
-    return new ListListener(loader, addFieldWriter(colSchema));
-  }
-
-  /**
-   * Create a RepeatedList which contains Unions. (Actually, this is an
-   * array of List objects internally.) The variant is variable, it makes no
-   * sense to specify a schema for the variant. Also, omitting the schema
-   * save a large amount of complexity that will likely never be needed.
-   */
-  @SuppressWarnings("unused")
-  private ValueListener repeatedListOfVariantListenerFor(String key, ValueDef valueDef) {
-    return multiDimVariantArrayListenerFor(
-        MetadataUtils.newVariant(key, DataMode.REPEATED),
-        valueDef.dimensions());
-  }
-
-  /**
-   * Create a multi- (2+) dimensional variant array from a column schema and dimension
-   * count hint. This is actually an (n-1) dimensional array of lists, where a LISt
-   * is a repeated UNION.
-   */
-  private ValueListener multiDimVariantArrayListenerFor(ColumnMetadata colSchema, int dims) {
-    return RepeatedListValueListener.repeatedVariantListFor(loader,
-        addFieldWriter(colSchema));
-  }
-
-  /**
-   * Create a repeated list column and its multiple levels of inner structure
-   * from a provided schema. Repeated lists can nest to any number of levels to
-   * provide any number of dimensions. In general, if an array is <i>n</i>-dimensional,
-   * then there are <i>n</i>-1 repeated lists with some array type as the
-   * innermost dimension.
-   */
-  private ValueListener multiDimArrayListenerForSchema(ColumnMetadata providedSchema) {
-    // Parse the stack of repeated lists to count the "outer" dimensions and
-    // to locate the innermost array (the "list" which is "repeated").
-    int dims = 1; // For inner array
-    ColumnMetadata elementSchema = providedSchema;
-    while (MetadataUtils.isRepeatedList(elementSchema)) {
-      dims++;
-      elementSchema = elementSchema.childSchema();
-      Preconditions.checkArgument(elementSchema != null);
-    }
-
-    ColumnMetadata colSchema = repeatedListSchemaFor(providedSchema.name(), dims,
-        elementSchema.cloneEmpty());
-    switch (elementSchema.structureType()) {
-
-      case PRIMITIVE:
-        return multiDimScalarArrayListenerFor(colSchema, dims);
-
-      case TUPLE:
-        return multiDimObjectArrayListenerFor(colSchema,
-            dims, elementSchema.tupleSchema());
-
-      case VARIANT:
-        return multiDimVariantArrayListenerFor(colSchema, dims);
-
-      default:
-        throw loader.unsupportedType(providedSchema);
-    }
-  }
-
-  /**
-   * Create a listener when we don't have type information. For the case
-   * {@code null} appears before other values.
-   */
-  private ValueListener unknownListenerFor(String key) {
-    return new UnknownFieldListener(this, key);
-  }
-
-  /**
-   * Create a listener when we don't have type information. For the case
-   * {@code []} appears before other values.
-   */
-  private ValueListener unknownArrayListenerFor(String key, ValueDef valueDef) {
-    UnknownFieldListener fieldListener = new UnknownFieldListener(this, key);
-    fieldListener.array(valueDef);
-    return fieldListener;
-  }
-
-  private ObjectWriter addFieldWriter(ColumnMetadata colSchema) {
-    int index = tupleWriter.addColumn(colSchema);
-    return tupleWriter.column(index);
-  }
-
-  /**
-   * Convert the JSON type, obtained by looking ahead one token, to a Drill
-   * scalar type. Report an error if the JSON type does not map to a Drill
-   * type (which can occur in a context where we expect a scalar, but got
-   * an object or array.)
-   */
-  private MinorType scalarTypeFor(String key, JsonType jsonType) {
-    MinorType colType = drillTypeFor(jsonType);
-    if (colType == null) {
-      throw loader.unsupportedJsonTypeException(key, jsonType);
-    }
-    return colType;
-  }
-
-  public MinorType drillTypeFor(JsonType type) {
-    if (loader.options().allTextMode) {
-      return MinorType.VARCHAR;
-    }
-    switch (type) {
-    case BOOLEAN:
-      return MinorType.BIT;
-    case FLOAT:
-      return MinorType.FLOAT8;
-    case INTEGER:
-      if (loader.options().readNumbersAsDouble) {
-        return MinorType.FLOAT8;
-      } else {
-        return MinorType.BIGINT;
-      }
-    case STRING:
-      return MinorType.VARCHAR;
-    default:
-      return null;
-    }
-  }
-
-  /**
-   * Build up a repeated list column definition given a specification of the
-   * number of dimensions and the JSON type. Creation of the element type is
-   * via a closure that builds the needed schema.
-   */
-  private ColumnMetadata repeatedListSchemaFor(String key, int dims,
-      ColumnMetadata innerArray) {
-    ColumnMetadata prev = innerArray;
-    for (int i = 1; i < dims; i++) {
-      prev = MetadataUtils.newRepeatedList(key, prev);
-    }
-    return prev;
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleParser.java
new file mode 100644
index 0000000..af492c6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/TupleParser.java
@@ -0,0 +1,159 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.easy.json.parser.ElementParser;
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.ObjectParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.TupleWriter;
+
+/**
+ * Accepts { name : value ... }
+ * <p>
+ * The structure parser maintains a map of known fields. Each time a
+ * field is parsed, looks up the field in the map. If not found, the parser
+ * looks ahead to find a value token, if any, and calls this class to add
+ * a new column. This class creates a column writer based either on the
+ * type provided in a provided schema, or inferred from the JSON token.
+ * <p>
+ * As it turns out, most of the semantic action occurs at the tuple level:
+ * that is where fields are defined, types inferred, and projection is
+ * computed.
+ *
+ * <h4>Nulls</h4>
+ *
+ * Much code here deals with null types, especially leading nulls, leading
+ * empty arrays, and so on. The object parser creates a parser for each
+ * value; a parser which "does the right thing" based on the data type.
+ * For example, for a Boolean, the parser recognizes {@code true},
+ * {@code false} and {@code null}.
+ * <p>
+ * But what happens if the first value for a field is {@code null}? We
+ * don't know what kind of parser to create because we don't have a schema.
+ * Instead, we have to create a temporary placeholder parser that will consume
+ * nulls, waiting for a real type to show itself. Once that type appears, the
+ * null parser can replace itself with the correct form. Each vector's
+ * "fill empties" logic will back-fill the newly created vector with nulls
+ * for prior rows.
+ * <p>
+ * Two null parsers are needed: one when we see an empty list, and one for
+ * when we only see {@code null}. The one for {@code null{@code  must morph into
+ * the one for empty lists if we see:<br>
+ * {@code {a: null} {a: [ ]  }}<br>
+ * <p>
+ * If we get all the way through the batch, but have still not seen a type,
+ * then we have to guess. A prototype type system can tell us, otherwise we
+ * guess {@code VARCHAR}. ({@code VARCHAR} is the right choice for all-text
+ * mode, it is as good a guess as any for other cases.)
+ *
+ * <h4>Projection List Hints</h4>
+ *
+ * To help, we consult the projection list, if any, for a column. If the
+ * projection is of the form {@code a[0]}, we know the column had better
+ * be an array. Similarly, if the projection list has {@code b.c}, then
+ * {@code b} had better be an object.
+ *
+ * <h4>Array Handling</h4>
+ *
+ * The code here handles arrays in two ways. JSON normally uses the
+ * {@code LIST} type. But, that can be expensive if lists are
+ * well-behaved. So, the code here also implements arrays using the
+ * classic {@code REPEATED} types. The repeated type option is disabled
+ * by default. It can be enabled, for efficiency, if Drill ever supports
+ * a JSON schema. If an array is well-behaved, mark that column as able
+ * to use a repeated type.
+ *
+ * <h4>Ambiguous Types</h4>
+ *
+ * JSON nulls are untyped. A run of nulls does not tell us what type will
+ * eventually appear. The best solution is to provide a schema. Without a
+ * schema, the code is forgiving: defers selection of the column type until
+ * the first non-null value (or, forces a type at the end of the batch.)
+ * <p>
+ * For scalars the pattern is: <code>{a: null} {a: "foo"}</code>. Type
+ * selection happens on the value {@code "foo"}.
+ * <p>
+ * For arrays, the pattern is: <code>{a: []} {a: ["foo"]}</code>. Type
+ * selection happens on the first array element. Note that type selection
+ * must happen on the first element, even if tha element is null (which,
+ * as we just said, ambiguous.)
+ * <p>
+ * If we are forced to pick a type (because we hit the end of a batch, or
+ * we see {@code [null]}, then we pick {@code VARCHAR} as we allow any
+ * scalar to be converted to {@code VARCHAR}. This helps for a single-file
+ * query, but not if multiple fragments each make their own (inconsistent)
+ * decisions. Only a schema provides a consistent answer.
+ */
+public class TupleParser extends ObjectParser {
+
+  private final JsonLoaderImpl loader;
+  private final TupleWriter tupleWriter;
+  private final TupleMetadata providedSchema;
+
+  // Bootstrap case: struct parser not yet set on the JSON loader
+  public TupleParser(JsonStructureParser structParser, JsonLoaderImpl loader,
+      TupleWriter tupleWriter, TupleMetadata providedSchema) {
+    super(structParser);
+    this.loader = loader;
+    this.tupleWriter = tupleWriter;
+    this.providedSchema = providedSchema;
+  }
+
+  public TupleParser(JsonLoaderImpl loader, TupleWriter tupleWriter, TupleMetadata providedSchema) {
+    this(loader.parser(), loader, tupleWriter, providedSchema);
+  }
+
+  public JsonLoaderImpl loader() { return loader; }
+  public TupleWriter writer() { return tupleWriter; }
+  protected TupleMetadata providedSchema() { return providedSchema; }
+  protected FieldFactory fieldFactory() { return loader.fieldFactory(); }
+
+  @Override
+  public ElementParser onField(String key, TokenIterator tokenizer) {
+    if (!tupleWriter.isProjected(key)) {
+      return fieldFactory().ignoredFieldParser();
+    } else {
+      return fieldParserFor(key, tokenizer);
+    }
+  }
+
+  private ElementParser fieldParserFor(String key, TokenIterator tokenizer) {
+    return fieldFactory().fieldParser(new FieldDefn(this, key, tokenizer));
+  }
+
+  public ElementParser resolveField(String key, TokenIterator tokenizer) {
+    return replaceFieldParser(key, fieldParserFor(key, tokenizer));
+  }
+
+  public ElementParser resolveArray(String key, TokenIterator tokenizer) {
+    return replaceFieldParser(key,
+        fieldFactory().fieldParser(new FieldDefn(this, key, tokenizer, true)));
+  }
+
+  public void forceNullResolution(String key) {
+    replaceFieldParser(key,
+        fieldFactory().forceNullResolution(new FieldDefn(this, key, null)));
+  }
+
+  public void forceEmptyArrayResolution(String key) {
+    replaceFieldParser(key,
+        fieldFactory().forceArrayResolution(new FieldDefn(this, key, null)));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/UnknownFieldListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/UnknownFieldListener.java
deleted file mode 100644
index 530342d..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/UnknownFieldListener.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.NullTypeMarker;
-import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Represents a rather odd state: we have seen a value of one or more
- * {@code null}s or empty arrays ({@code []}), but we have not yet seen a
- * value that would give us a type. This listener
- * acts as a placeholder; waiting to see the type, at which point it replaces
- * itself with the actual typed listener. If a batch completes with only nulls
- * for this field, then the field becomes a {@code VARCHAR} field. Drill's "fill
- * empties" logic will back-fill nulls. All values in
- * subsequent batches will be read in "text mode" for that one field in
- * order to avoid a schema change.
- * <p>
- * Note what this listener does <i>not</i> do: it does not create a nullable
- * int field per Drill's normal (if less than ideal) semantics. First, JSON
- * <b>never</b> produces an int field, so nullable int is less than ideal.
- * Second, nullable int has no basis in reality and so is a poor choice
- * on that basis.
- * <p>
- * Note that we <i>cannot</i> use this class for an array that
- * contains nulls: {@code [null]}. The null is a value that must be
- * stored, so we must guess the type as we have no good way to count
- * array entries except via vectors.
- */
-public class UnknownFieldListener extends AbstractValueListener implements NullTypeMarker {
-  protected static final Logger logger = LoggerFactory.getLogger(UnknownFieldListener.class);
-
-  protected final TupleListener parentTuple;
-  protected final String key;
-  protected ValueHost host;
-  private UnknownArrayListener unknownArray;
-
-  public UnknownFieldListener(TupleListener parentTuple, String key) {
-    super(parentTuple.loader());
-    this.parentTuple = parentTuple;
-    this.key = key;
-    loader.addNullMarker(this);
-  }
-
-  @Override
-  public void bind(ValueHost host) {
-    this.host = host;
-  }
-
-  @Override
-  public void onNull() {
-    if (unknownArray != null) {
-      // An array, must resolve to some type.
-      resolveScalar(JsonType.NULL).onNull();
-    }
-    // Else ignore: still don't know what this is
-  }
-
-  @Override
-  public void onBoolean(boolean value) {
-    resolveScalar(JsonType.BOOLEAN).onBoolean(value);
-  }
-
-  @Override
-  public void onInt(long value) {
-    resolveScalar(JsonType.INTEGER).onInt(value);
-  }
-
-  @Override
-  public void onFloat(double value) {
-    resolveScalar(JsonType.FLOAT).onFloat(value);
-  }
-
-  @Override
-  public void onString(String value) {
-    resolveScalar(JsonType.STRING).onString(value);
-  }
-
-  @Override
-  public void onEmbeddedObject(String value) {
-    resolveScalar(JsonType.EMBEDDED_OBJECT).onEmbeddedObject(value);
-  }
-
-  @Override
-  public ObjectListener object() {
-    return resolveTo(parentTuple.objectListenerForValue(key)).object();
-  }
-
-  /**
-   * The column type is now known from context. Create a new, scalar
-   * column, writer and listener to replace ourself: this is the last
-   * call that this listener will receive.
-   */
-  protected ValueListener resolveScalar(JsonType type) {
-    if (unknownArray == null) {
-      return resolveTo(parentTuple.scalarListenerForValue(key, type));
-    } else {
-
-      // Saw {a: []}, {a: 10}. Since we infer that 10 is a
-      // single-element array, resolve to an array, then send
-      // the value to the element.
-      return unknownArray.element(new ValueDef(type, 0));
-    }
-  }
-
-  @Override
-  protected ColumnMetadata schema() {
-    throw new IllegalStateException("Unknown column has no schema");
-  }
-
-  @Override
-  public ArrayListener array(ValueDef valueDef) {
-    if (valueDef.dimensions() > 1) {
-
-      // if 2D+ array, then we know enough to choose a Repeated list
-      return resolveToArray(valueDef).array(valueDef);
-    }
-    if (unknownArray == null) {
-      unknownArray = new UnknownArrayListener(this);
-    }
-    return unknownArray;
-  }
-
-  protected ValueListener resolveTo(ValueListener newListener) {
-    host.bindListener(newListener);
-    loader.removeNullMarker(this);
-    return newListener;
-  }
-
-  @Override
-  public void forceResolution() {
-    if (unknownArray == null) {
-      logger.warn("Ambiguous type! JSON field {}" +
-          " contains all nulls. Assuming VARCHAR.", key);
-      resolveTo(parentTuple.scalarListenerForValue(key, JsonType.STRING));
-    } else {
-      logger.warn("Ambiguous type! JSON array field {}" +
-          " contains all empty arrays. Assuming repeated VARCHAR.", key);
-      resolveTo(parentTuple.scalarArrayListenerForValue(key, JsonType.STRING));
-    }
-  }
-
-  public ValueListener resolveToArray(ValueDef valueDef) {
-    if (valueDef.type().isUnknown()) {
-      logger.warn("Ambiguous type! JSON array field {}" +
-          " starts with null element. Assuming repeated VARCHAR.", key);
-      valueDef = new ValueDef(JsonType.STRING, valueDef.dimensions());
-    }
-    return resolveTo(parentTuple.listenerForValue(key, valueDef));
-  }
-
-  /**
-   * An unknown array within the unknown field. Represents an
-   * empty array: {@code []}. Resolves to a specific type upon
-   * presentation of the first element. If that element is
-   * {@code null}, must still choose a type to record nulls.
-   * <p>
-   * This array listener holds no element since none has been
-   * created yet; we use this only while we see empty arrays.
-   */
-  public static class UnknownArrayListener implements ArrayListener {
-
-    private final UnknownFieldListener parent;
-
-    public UnknownArrayListener(UnknownFieldListener parent) {
-      this.parent = parent;
-    }
-
-    @Override
-    public void onStart() { }
-
-    @Override
-    public void onElementStart() { }
-
-    @Override
-    public void onElementEnd() { }
-
-    @Override
-    public void onEnd() { }
-
-    /**
-     * Saw the first actual element. Swap out the field listener
-     * for a real array, then return the new element listener.
-     */
-    @Override
-    public ValueListener element(ValueDef valueDef) {
-      ValueDef arrayDef = new ValueDef(valueDef.type(), valueDef.dimensions() + 1);
-      return parent.resolveToArray(arrayDef)
-          .array(arrayDef)
-          .element(valueDef);
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantListener.java
deleted file mode 100644
index dda731c..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantListener.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.loader;
-
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
-import org.apache.drill.exec.vector.accessor.VariantWriter;
-
-/**
- * Listener for a UNION type column which maps each JSON type to
- * the matching Drill type within the UNION. Used only if a column
- * is declared as UNION in the provided schema. This implementation
- * does not have a way to convert a non-UNION column into a UNION
- * during the scan. The reason is simple: the scan is obligated to
- * return a consistent schema. Converting a column between types,
- * especially after returning the first batch, will lead to an
- * inconsistent schema and to downstream schema change failures.
- */
-public class VariantListener extends AbstractValueListener {
-
-  private final VariantWriter writer;
-
-  public VariantListener(JsonLoaderImpl loader, VariantWriter writer) {
-    super(loader);
-    this.writer = writer;
-  }
-
-  @Override
-  public void onNull() { }
-
-  @Override
-  public void onBoolean(boolean value) {
-    writer.scalar(MinorType.BIT).setBoolean(value);
-  }
-
-  @Override
-  public void onInt(long value) {
-    writer.scalar(MinorType.BIGINT).setLong(value);
-  }
-
-  @Override
-  public void onFloat(double value) {
-    writer.scalar(MinorType.FLOAT8).setDouble(value);
-  }
-
-  @Override
-  public void onString(String value) {
-    writer.scalar(MinorType.VARCHAR).setString(value);
-  }
-
-  @Override
-  protected ColumnMetadata schema() {
-    return writer.schema();
-  }
-
-  @Override
-  public ObjectListener object() {
-    return new VariantTupleListener(loader, writer);
-  }
-
-  private static class VariantTupleListener extends TupleListener {
-
-    private final VariantWriter writer;
-
-    public VariantTupleListener(JsonLoaderImpl loader, VariantWriter writer) {
-      super(loader, writer.tuple(), null);
-      this.writer = writer;
-    }
-
-    @Override
-    public void onStart() {
-      writer.setType(MinorType.MAP);
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantParser.java
new file mode 100644
index 0000000..ba5225e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VariantParser.java
@@ -0,0 +1,97 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.store.easy.json.parser.ArrayParser;
+import org.apache.drill.exec.store.easy.json.parser.FullValueParser;
+import org.apache.drill.exec.store.easy.json.parser.ObjectParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.VariantWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parser which accepts all JSON values and converts them to actions on a
+ * UNION vector writer. Scalar values are written to the writer directly.
+ * Object and array values create extra layers of parser and listener.
+ */
+public class VariantParser extends FullValueParser {
+
+  private final JsonLoaderImpl loader;
+  private final VariantWriter writer;
+
+  public VariantParser(JsonLoaderImpl loader, VariantWriter writer) {
+    super(loader.parser());
+    this.loader = loader;
+    this.writer = writer;
+  }
+
+  @Override
+  protected void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        writer.setNull();
+        break;
+      case VALUE_TRUE:
+        writer.scalar(MinorType.BIT).setBoolean(true);
+        break;
+      case VALUE_FALSE:
+        writer.scalar(MinorType.BIT).setBoolean(false);
+        break;
+      case VALUE_NUMBER_INT:
+        writer.scalar(MinorType.BIGINT).setLong(tokenizer.longValue());
+        break;
+      case VALUE_NUMBER_FLOAT:
+        writer.scalar(MinorType.FLOAT8).setDouble(tokenizer.doubleValue());
+        break;
+      case VALUE_STRING:
+        writer.scalar(MinorType.VARCHAR).setString(tokenizer.stringValue());
+        break;
+      default:
+        // Won't get here: the Jackson parser catches errors.
+        throw tokenizer.invalidValue(token);
+    }
+  }
+
+  @Override
+  protected ObjectParser buildObjectParser(TokenIterator tokenizer) {
+    return new VariantObjectParser(loader, writer);
+  }
+
+  @Override
+  protected ArrayParser buildArrayParser(TokenIterator tokenizer) {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  private static class VariantObjectParser extends TupleParser {
+
+    private final VariantWriter writer;
+
+    public VariantObjectParser(JsonLoaderImpl loader, VariantWriter writer) {
+      super(loader, writer.tuple(), null);
+      this.writer = writer;
+    }
+
+    @Override
+    public void onStart() {
+      writer.setType(MinorType.MAP);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
index a99dcec..38900b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
@@ -23,17 +23,11 @@ package org.apache.drill.exec.store.easy.json.parser;
  */
 public abstract class AbstractElementParser implements ElementParser {
   final JsonStructureParser structParser;
-  private final ElementParser parent;
 
-  public AbstractElementParser(ElementParser parent) {
-    this.parent = parent;
-    this.structParser = parent.structParser();
+  public AbstractElementParser(JsonStructureParser structParser) {
+    this.structParser = structParser;
   }
 
-  @Override
-  public ElementParser parent() { return parent; }
-
-  @Override
   public JsonStructureParser structParser() { return structParser; }
 
   protected ErrorFactory errorFactory() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayListener.java
index 06a67aa..2afd4ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayListener.java
@@ -82,16 +82,6 @@ package org.apache.drill.exec.store.easy.json.parser;
 public interface ArrayListener {
 
   /**
-   * Provide an element listener for the first non-empty value
-   * seen for the array.
-   *
-   * @param valueDef description of the element (without the array
-   * dimensions)
-   * @return a listener to consume values of the array element
-   */
-  ValueListener element(ValueDef valueDef);
-
-  /**
    * Called at the entrance to each level (dimension) of an array.
    * That is, called when the structure parser accepts the {@code [}
    * token.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayParser.java
index d6f5912..44041ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayParser.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.store.easy.json.parser;
 
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener.FieldType;
-
 import com.fasterxml.jackson.core.JsonToken;
 
 /**
@@ -38,15 +36,19 @@ import com.fasterxml.jackson.core.JsonToken;
  */
 public class ArrayParser extends AbstractElementParser {
 
-  private ValueParser elementParser;
-  private ArrayListener arrayListener;
+  private final ArrayListener arrayListener;
+  private final ElementParser elementParser;
 
-  public ArrayParser(ValueParser parent, ArrayListener arrayListener) {
-    super(parent);
+  public ArrayParser(JsonStructureParser structParser, ArrayListener arrayListener, ElementParser elementParser) {
+    super(structParser);
     this.arrayListener = arrayListener;
+    this.elementParser = elementParser;
   }
 
-  public ValueParser elementParser() { return elementParser; }
+  public ElementParser elementParser() { return elementParser; }
+
+  @SuppressWarnings("unchecked")
+  public <T extends ArrayListener> T listener() { return (T) arrayListener; }
 
   /**
    * Parses <code>[ ^ ((value)(, (value)* )? ]</code>
@@ -69,45 +71,8 @@ public class ArrayParser extends AbstractElementParser {
   }
 
   private void parseElement(TokenIterator tokenizer) {
-    if (elementParser == null) {
-      detectElement(tokenizer);
-    }
     arrayListener.onElementStart();
     elementParser.parse(tokenizer);
     arrayListener.onElementEnd();
   }
-
-  private void detectElement(TokenIterator tokenizer) {
-    addElement(ValueDefFactory.lookAhead(tokenizer));
-  }
-
-  public void addElement(ValueDef valueDef) {
-    bindElement(arrayListener.element(valueDef));
-  }
-
-  public void bindElement(ValueListener elementListener) {
-    elementParser = new ValueParser(this, "[]", FieldType.TYPED);
-    elementParser.bindListener(elementListener);
-  }
-
-  public void bindListener(ArrayListener newListener) {
-    arrayListener = newListener;
-    if (elementParser != null) {
-      elementParser.bindListener(arrayListener.element(ValueDef.UNKNOWN));
-    }
-  }
-
-  /**
-   * Expand the structure of this array given a description of the
-   * look-ahead value. Skip if this is a 1D array of unknown type.
-   * If 2D or greater, then we must create the child array of one
-   * less dimension.
-    */
-  public void expandStructure(ValueDef valueDef) {
-    if (valueDef.dimensions() > 1 || !valueDef.type().isUnknown()) {
-      ValueDef elementDef = new ValueDef(valueDef.type(), valueDef.dimensions() - 1);
-      addElement(elementDef);
-      elementParser.expandStructure(elementDef);
-    }
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayValueParser.java
new file mode 100644
index 0000000..bc4dcec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayValueParser.java
@@ -0,0 +1,66 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public class ArrayValueParser extends AbstractElementParser {
+
+  protected final ArrayParser arrayParser;
+
+  public ArrayValueParser(ArrayParser arrayParser) {
+    super(arrayParser.structParser());
+    this.arrayParser = arrayParser;
+  }
+
+  /**
+   * Parses <code>true | false | null | integer | float | string|
+   *              embedded-object | [ ... ]</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    if (token == JsonToken.START_ARRAY) {
+      // Position: [ ^
+      arrayParser.parse(tokenizer);
+    } else if (token == JsonToken.VALUE_NULL) {
+      // Treat as if the field was not present: ignore
+    } else if (token.isScalarValue()) {
+      tokenizer.unget(token);
+      parseValue(tokenizer);
+    } else {
+      throw errorFactory().structureError("JSON array expected");
+    }
+  }
+
+  protected void parseValue(TokenIterator tokenizer) {
+    throw errorFactory().structureError("JSON array expected");
+  }
+
+  public static class LenientArrayValueParser extends ArrayValueParser {
+
+    public LenientArrayValueParser(ArrayParser arrayParser) {
+      super(arrayParser);
+    }
+
+    @Override
+    protected void parseValue(TokenIterator tokenizer) {
+      arrayParser.elementParser().parse(tokenizer);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/DummyValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/DummyValueParser.java
index d705f00..59aecf9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/DummyValueParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/DummyValueParser.java
@@ -23,11 +23,11 @@ import com.fasterxml.jackson.core.JsonToken;
  * Parse and ignore an unprojected value. The parsing just "free wheels", we
  * care only about matching brackets, but not about other details.
  */
-class DummyValueParser extends AbstractElementParser {
+public class DummyValueParser implements ElementParser {
 
-  public DummyValueParser(ElementParser parent) {
-    super(parent);
-  }
+  public static final ElementParser INSTANCE = new DummyValueParser();
+
+  private DummyValueParser() { }
 
   @Override
   public void parse(TokenIterator tokenizer) {
@@ -38,21 +38,12 @@ class DummyValueParser extends AbstractElementParser {
         parseTail(tokenizer);
         break;
 
-      case VALUE_NULL:
-      case VALUE_EMBEDDED_OBJECT:
-      case VALUE_FALSE:
-      case VALUE_TRUE:
-      case VALUE_NUMBER_FLOAT:
-      case VALUE_NUMBER_INT:
-      case VALUE_STRING:
-        break;
-
       default:
-        throw errorFactory().syntaxError(token);
+        break;
     }
   }
 
-  public void parseTail(TokenIterator tokenizer) {
+  private void parseTail(TokenIterator tokenizer) {
 
     // Parse (field: value)* }
     while (true) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ElementParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ElementParser.java
index b21b034..9801989 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ElementParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ElementParser.java
@@ -33,7 +33,5 @@ package org.apache.drill.exec.store.easy.json.parser;
  * parsed.
  */
 public interface ElementParser {
-  ElementParser parent();
-  JsonStructureParser structParser();
   void parse(TokenIterator tokenizer);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/EmptyArrayParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/EmptyArrayParser.java
new file mode 100644
index 0000000..4556406
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/EmptyArrayParser.java
@@ -0,0 +1,82 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Represents an empty array: the case where the parser has seen only
+ * {@code []}, but no array elements which would indicate the type.
+ * Resolves to a specific type upon
+ * presentation of the first element. If that element is
+ * {@code null}, we must still choose a type to record nulls.
+ * <p>
+ * This array listener holds no element since none has been
+ * created yet; we use this only while we see empty arrays.
+ */
+public abstract class EmptyArrayParser extends AbstractElementParser {
+
+  protected final String key;
+
+  public EmptyArrayParser(JsonStructureParser structParser, String key) {
+    super(structParser);
+    this.key = key;
+  }
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token1 = tokenizer.requireNext();
+
+    // Ignore null: treat as an empty array.
+    if (token1 == JsonToken.VALUE_NULL) {
+      return;
+    }
+
+    // Assume an scalar is a one-item array.
+    if (token1 != JsonToken.START_ARRAY) {
+      tokenizer.unget(token1);
+      resolve(tokenizer).parse(tokenizer);
+
+      // This parser never called again
+      return;
+    }
+
+    // Ignore an empty array, resolve a non-empty array.
+    // Must be done even if the array value is null so elements
+    // can be counted.
+    JsonToken token2 = tokenizer.requireNext();
+    if (token2 != JsonToken.END_ARRAY) {
+
+      // Saw the first actual element. Swap out this parser for a
+      // real field parser, then let that parser parse from here.
+      // The real parser is for the entire field, so we unget the
+      // opening bracket as well as the element.
+      tokenizer.unget(token2);
+      tokenizer.unget(token1);
+      resolve(tokenizer).parse(tokenizer);
+
+      // This parser never called again
+    }
+  }
+
+  /**
+   * Replace this parser with a new parser based on the current
+   * parse context.
+   */
+  protected abstract ElementParser resolve(TokenIterator tokenizer);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FieldParserFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FieldParserFactory.java
new file mode 100644
index 0000000..3951db7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FieldParserFactory.java
@@ -0,0 +1,106 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import java.util.function.Function;
+
+import org.apache.drill.exec.store.easy.json.parser.ArrayValueParser.LenientArrayValueParser;
+import org.apache.drill.exec.store.easy.json.parser.ScalarValueParser.SimpleValueParser;
+import org.apache.drill.exec.store.easy.json.parser.ScalarValueParser.TextValueParser;
+import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
+
+
+/**
+ * Creates a field parser given a field description and an optional field
+ * listener.
+ * <p>
+ * Parse position: <code>{ ... field : ^ ?</code> for a newly-seen field.
+ * Constructs a value parser and its listeners by looking ahead
+ * some number of tokens to "sniff" the type of the value. For
+ * example:
+ * <ul>
+ * <li>{@code foo: <value>} - Field value</li>
+ * <li>{@code foo: [ <value> ]} - 1D array value</li>
+ * <li>{@code foo: [ [<value> ] ]} - 2D array value</li>
+ * <li>Etc.</li>
+ * </ul>
+ * <p>
+ * There are two cases in which no type estimation is possible:
+ * <ul>
+ * <li>The value is {@code null}, indicated by
+ * {@link JsonType#NULL}.</code>
+ * <li>The value is an array, and the array is empty, indicated
+ * by {@link JsonType#EMPTY}.</li>
+ * </ul>
+ * {@link ValueDefFactory} handles syntactic type inference. The associated
+ * listener enforces semantic rules. For example, if a schema is
+ * available, and we know that field "x" must be an Integer, but
+ * this class reports that it is an object, then the listener should
+ * raise an exception.
+ * <p>
+ * Also, the parser cannot enforce type consistency. This method
+ * looks only at the first appearance of a value: a sample size of
+ * one. JSON allows anything.
+ * The listener must enforce semantic rules that say whether a different
+ * type is allowed for later values.
+ */
+public class FieldParserFactory {
+
+  private final JsonStructureParser structParser;
+  private final Function<JsonStructureParser, ObjectParser> parserFactory;
+
+  public FieldParserFactory(JsonStructureParser structParser,
+      Function<JsonStructureParser, ObjectParser> parserFactory) {
+    this.structParser = structParser;
+    this.parserFactory = parserFactory;
+  }
+
+  public ObjectParser rootParser() {
+    return parserFactory.apply(structParser);
+  }
+
+  public ElementParser ignoredFieldParser() {
+    return DummyValueParser.INSTANCE;
+  }
+
+  public ValueParser jsonTextParser(ValueListener fieldListener) {
+    return new JsonValueParser(structParser, fieldListener);
+  }
+
+  public ValueParser simpleValueParser(ValueListener fieldListener) {
+    return new SimpleValueParser(structParser, fieldListener);
+  }
+
+  public ValueParser textValueParser(ValueListener fieldListener) {
+    return new TextValueParser(structParser, fieldListener);
+  }
+
+  public ElementParser scalarArrayValueParser(ArrayListener arrayListener, ElementParser elementParser) {
+    return new LenientArrayValueParser(
+        new ArrayParser(structParser, arrayListener, elementParser));
+  }
+
+  public ElementParser arrayValueParser(ArrayListener arrayListener, ElementParser elementParser) {
+    return new ArrayValueParser(
+        new ArrayParser(structParser, arrayListener, elementParser));
+  }
+
+  public ElementParser objectValueParser(ObjectParser objParser) {
+    return new ObjectValueParser(objParser);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FullValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FullValueParser.java
new file mode 100644
index 0000000..bc1c577
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/FullValueParser.java
@@ -0,0 +1,67 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public abstract class FullValueParser extends AbstractElementParser {
+
+  private ObjectParser objectParser;
+  private ArrayParser arrayParser;
+
+  public FullValueParser(JsonStructureParser structParser) {
+    super(structParser);
+  }
+
+  /**
+   * Parses <code>true | false | null | integer | float | string|
+   *              embedded-object | { ... } | [ ... ]</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    switch (token) {
+    case START_OBJECT:
+      // Position: { ^
+      if (objectParser == null) {
+        // No object parser yet. May be that the value was null,
+        // or may be that it changed types.
+        objectParser = buildObjectParser(tokenizer);
+      }
+      objectParser.parse(tokenizer);
+      break;
+
+    case START_ARRAY:
+      // Position: [ ^
+      if (arrayParser == null) {
+        // No array parser yet. May be that the value was null,
+        // or may be that it changed types.
+        arrayParser = buildArrayParser(tokenizer);
+      }
+      arrayParser.parse(tokenizer);
+      break;
+
+    default:
+      onValue(token, tokenizer);
+    }
+  }
+
+  protected abstract void onValue(JsonToken token, TokenIterator tokenizer);
+  protected abstract ObjectParser buildObjectParser(TokenIterator tokenizer);
+  protected abstract ArrayParser buildArrayParser(TokenIterator tokenizer);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureOptions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureOptions.java
index 6e072d2..f4a3277 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureOptions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureOptions.java
@@ -28,14 +28,6 @@ import org.apache.drill.exec.server.options.OptionSet;
 public class JsonStructureOptions {
 
   /**
-   * JSON returns values as typed tokens. If {@code allTextMode} is
-   * set, the structure parser converts all scalars (except {@code null})
-   * to text and forwards the values to the listener as text.
-   * Implements Drill's "all-text mode" for JSON.
-   */
-  public boolean allTextMode;
-
-  /**
    * Allow Infinity and NaN for float values.
    */
 
@@ -60,7 +52,6 @@ public class JsonStructureOptions {
   public JsonStructureOptions() { }
 
   public JsonStructureOptions(OptionSet options) {
-    this.allTextMode = options.getBoolean(ExecConstants.JSON_ALL_TEXT_MODE);
     this.allowNanInf = options.getBoolean(ExecConstants.JSON_READER_NAN_INF_NUMBERS);
     this.skipMalformedRecords = options.getBoolean(ExecConstants.JSON_READER_SKIP_INVALID_RECORDS_FLAG);
     this.enableEscapeAnyChar = options.getBoolean(ExecConstants.JSON_READER_ESCAPE_ANY_CHAR);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
index 4d46a46..ded872c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.easy.json.parser;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
+import java.util.function.Function;
 
 import org.apache.drill.exec.store.easy.json.parser.MessageParser.MessageContextException;
 import org.apache.drill.exec.store.easy.json.parser.RootParser.EmbeddedArrayParser;
@@ -34,6 +35,7 @@ import org.slf4j.LoggerFactory;
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.core.json.JsonReadFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
 /**
@@ -75,7 +77,7 @@ public class JsonStructureParser {
     private InputStream stream;
     private Reader reader;
     private JsonStructureOptions options;
-    private ObjectListener rootListener;
+    private Function<JsonStructureParser, ObjectParser> parserFactory;
     private ErrorFactory errorFactory;
     private String dataPath;
     private MessageParser messageParser;
@@ -85,8 +87,9 @@ public class JsonStructureParser {
       return this;
     }
 
-    public JsonStructureParserBuilder rootListener(ObjectListener rootListener) {
-      this.rootListener = rootListener;
+    public JsonStructureParserBuilder parserFactory(
+        Function<JsonStructureParser, ObjectParser> parserFactory) {
+      this.parserFactory = parserFactory;
       return this;
     }
 
@@ -129,10 +132,10 @@ public class JsonStructureParser {
 
   private final JsonParser parser;
   private final JsonStructureOptions options;
-  private final ObjectListener rootListener;
   private final ErrorFactory errorFactory;
   private final TokenIterator tokenizer;
   private final RootParser rootState;
+  private final FieldParserFactory fieldFactory;
   private int errorRecoveryCount;
 
   /**
@@ -148,14 +151,15 @@ public class JsonStructureParser {
    */
   private JsonStructureParser(JsonStructureParserBuilder builder) {
     this.options = Preconditions.checkNotNull(builder.options);
-    this.rootListener = Preconditions.checkNotNull(builder.rootListener);
     this.errorFactory = Preconditions.checkNotNull(builder.errorFactory);
     try {
       ObjectMapper mapper = new ObjectMapper()
           .configure(JsonParser.Feature.ALLOW_COMMENTS, true)
           .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true)
-          .configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, options.allowNanInf)
-          .configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER, options.enableEscapeAnyChar);
+          .configure(JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS.mappedFeature(),
+              options.allowNanInf)
+          .configure(JsonReadFeature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER.mappedFeature(),
+              options.enableEscapeAnyChar);
 
       if (builder.stream != null) {
         parser = mapper.getFactory().createParser(builder.stream);
@@ -168,6 +172,8 @@ public class JsonStructureParser {
       throw errorFactory().ioException(e);
     }
     tokenizer = new TokenIterator(parser, options, errorFactory());
+    fieldFactory = new FieldParserFactory(this,
+        Preconditions.checkNotNull(builder.parserFactory));
 
     // Parse to the start of the data object(s), and create a root
     // state to parse objects and watch for the end of data.
@@ -181,7 +187,7 @@ public class JsonStructureParser {
 
   public JsonStructureOptions options() { return options; }
   public ErrorFactory errorFactory() { return errorFactory; }
-  public ObjectListener rootListener() { return rootListener; }
+  public FieldParserFactory fieldFactory() { return fieldFactory; }
 
   private RootParser makeRootState() {
     JsonToken token = tokenizer.next();
@@ -245,7 +251,7 @@ public class JsonStructureParser {
         return rootState.parseRoot(tokenizer);
       } catch (RecoverableJsonException e) {
         if (! recover()) {
-          return false;
+          throw errorFactory().structureError("Unrecoverable error - " + e.getMessage());
         }
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonValueParser.java
index 8819e2c..a8622bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonValueParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonValueParser.java
@@ -24,15 +24,13 @@ import com.fasterxml.jackson.core.JsonToken;
  * complexity) into a JSON string. That is, converts the parsed
  * JSON tokens back into the original JSON text.
  */
-public class JsonValueParser extends AbstractElementParser {
+public class JsonValueParser extends ValueParser {
 
-  private final ValueListener listener;
   private final StringBuilder json = new StringBuilder();
 
-  protected JsonValueParser(ElementParser parent, String key,
+  protected JsonValueParser(JsonStructureParser structParser,
       ValueListener listener) {
-    super(parent);
-     this.listener = listener;
+    super(structParser, listener);
   }
 
   @Override
@@ -40,7 +38,7 @@ public class JsonValueParser extends AbstractElementParser {
     JsonToken token = tokenizer.requireNext();
     json.setLength(0);
     parseValue(tokenizer, token);
-    listener.onString(json.toString());
+    listener.onText(json.toString());
     json.setLength(0);
   }
 
@@ -48,12 +46,12 @@ public class JsonValueParser extends AbstractElementParser {
     String textValue = tokenizer.textValue();
     switch (token) {
       case START_ARRAY:
-        json.append(textValue);
+        json.append(token.asString());
         parseArrayTail(tokenizer);
         break;
 
       case START_OBJECT:
-        json.append(textValue);
+        json.append(token.asString());
         parseObjectTail(tokenizer);
         break;
 
@@ -72,12 +70,11 @@ public class JsonValueParser extends AbstractElementParser {
   public void parseArrayTail(TokenIterator tokenizer) {
 
     // Accept value* ]
-
     boolean first = true;
     while (true) {
       JsonToken token = tokenizer.requireNext();
       if (token == JsonToken.END_ARRAY) {
-        json.append(tokenizer.textValue());
+        json.append(token.asString());
         return;
       }
       if (! first) {
@@ -91,12 +88,11 @@ public class JsonValueParser extends AbstractElementParser {
   public void parseObjectTail(TokenIterator tokenizer) {
 
     // Accept (field: value)* }
-
     boolean first = true;
     while (true) {
       JsonToken token = tokenizer.requireNext();
       if (token == JsonToken.END_OBJECT) {
-        json.append(tokenizer.textValue());
+        json.append(token.asString());
         return;
       }
       if (! first) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/NullValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/NullValueParser.java
new file mode 100644
index 0000000..384cc68
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/NullValueParser.java
@@ -0,0 +1,56 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parses nulls. On the first non-null token, replaces itself with
+ * a "resolved" parser to handle the actual structure.
+ */
+public abstract class NullValueParser extends AbstractElementParser {
+
+  protected final String key;
+
+  public NullValueParser(JsonStructureParser structParser, String key) {
+    super(structParser);
+    this.key = key;
+  }
+
+  /**
+   * Parses nulls. On the first non-null
+   * Parses <code>true | false | null | integer | float | string|
+   *              embedded-object | { ... } | [ ... ]</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    if (token != JsonToken.VALUE_NULL) {
+      tokenizer.unget(token);
+      resolve(tokenizer).parse(tokenizer);
+
+      // This parser never called again
+    }
+  }
+
+  /**
+   * Replace this parser with a new parser based on the current
+   * parse context.
+   */
+  protected abstract ElementParser resolve(TokenIterator tokenizer);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java
deleted file mode 100644
index 40b8617..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json.parser;
-
-import org.apache.calcite.model.JsonType;
-
-/**
- * Represents events on a object value. The object value may be a top-level
- * field or may be the element of an array. The listener gets an event when
- * an object is started and ended, as well as when a new field is discovered.
- * First, the parser asks if the field should be projected. If not, the
- * parser will create a dummy parser to "free-wheel" over whatever values the
- * field contains. (This is one way to avoid structure errors in a JSON file:
- * just ignore them.) Otherwise, the parser will look ahead to guess the
- * field type and will call one of the "add" methods, each of which should
- * return a value listener for the field itself.
- * <p>
- * The structure parser looks ahead some number of tokens to infer the value
- * of the field. While this is helpful, it really only works if the JSON
- * is structured like a list of tuples, if the initial value is not {@code null},
- * and if initial arrays are not empty. The structure parser cannot see
- * into the future beyond the first field value; the value listener for each
- * field must handle "type-deferal" if needed to handle missing or null
- * values. That is, type-consistency is a semantic task handled by the listener,
- * not a syntax task handled by the parser.
- *
- * <h4>Fields</h4>
- *
- * The structure of an object is:
- * <ul>
- * <li>{@code ObjectListener} which represents the object (tuple) as a whole.
- * Each field, indexed by name, is represented as a</li>
- * <li>{@code ValueListener} which represents the value "slot". That value
- * can be scalar, or can be structured, in which case the value listener
- * contains either a</li>
- * <li>{@code ArrayListener} for an array, or a</li>
- * <li>{@code ObjectListener} for a nested object (tuple).</li>
- * </ul>
- */
-public interface ObjectListener {
-
-  enum FieldType {
-
-    /**
-     * The field is unprojected, ignore its content. No value listener
-     * is created.
-     */
-    IGNORE,
-
-    /**
-     * Parse the JSON object according to its type.
-     */
-    TYPED,
-
-    /**
-     * The field is to be treated as "all-text". Used when the parser-level
-     * setting for {@code allTextMode} is {@code false}; allows per-field
-     * overrides to, perhaps, ride over inconsistent scalar types for a
-     * single field. The listener will receive only strings.
-     */
-    TEXT,
-
-    /**
-     * Parse the value, and all its children, as JSON.
-     * That is, converts the parsed JSON back into a
-     * JSON string. The listener will receive only strings.
-     */
-    JSON
-  }
-
-  /**
-   * Called at the start of a set of values for an object. That is, called
-   * when the structure parser accepts the <code>{</code> token.
-   */
-  void onStart();
-
-  /**
-   * Called by the structure parser when it first sees a new field for
-   * and object to determine how to parse the field.
-   * If not projected, the structure parser will not
-   * ask for a value listener and will insert a "dummy" parser that will
-   * free-wheel over any value of that field. As a result, unprojected
-   * fields can not cause type errors: they are invisible as long as
-   * they are syntactically valid.
-   * <p>
-   * The {@link FieldType#JSON} type says to parse the entire field, and
-   * its children, as a JSON string. The parser will ask for a value
-   * listener to accept the JSON text.
-   *
-   * @param key the object field name
-   * @return how the field should be parsed
-   */
-  FieldType fieldType(String key);
-
-  /**
-   * The structure parser has just encountered a new field for this
-   * object. The {@link #fieldType(String)} indicated that the field is
-   * to be projected. This method performs any setup needed to handle the
-   * field, then returns a value listener to receive events for the
-   * field value. The value listener may be asked to create additional
-   * structure, such as arrays or nested objects.
-   *
-   * @param key the field name
-   * @param valueDef a description of the field as inferred by looking
-   * ahead some number of tokens in the input JSON. Provides both a data
-   * type and array depth (dimensions.) If the type is
-   * {@link JsonType#NONE EMPTY}, then the field is an empty array.
-   * If the type is {@link JsonType#NULL NULL}, then the value is null. In these
-   * cases, the listener can replace itself when an actual value appears
-   * later
-   * @return a listener to receive events for the newly-created field
-   */
-  ValueListener addField(String key, ValueDef valueDef);
-
-  /**
-   * Called at the end of a set of values for an object. That is, called
-   * when the structure parser accepts the <code>}</code> token.
-   */
-  void onEnd();
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectParser.java
index ce331a3..6ddec1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectParser.java
@@ -20,14 +20,19 @@ package org.apache.drill.exec.store.easy.json.parser;
 import java.util.Map;
 
 import org.apache.drill.common.map.CaseInsensitiveMap;
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener.FieldType;
-import org.apache.drill.exec.store.easy.json.parser.ValueDef.JsonType;
+import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.core.JsonToken;
 
 /**
  * Parses a JSON object: <code>{ name : value ... }</code>
  * <p>
+ * The object value may the root object (the row), a top-level
+ * field or may be the element of an array. The event methods are called when
+ * an object is started and ended, as well as when a new field is discovered.
+ * <p>
  * Creates a map of known fields. Each time a field is parsed,
  * looks up the field in the map. If not found, the value is "sniffed"
  * to determine its type, and a matching parser and listener created.
@@ -50,6 +55,19 @@ import com.fasterxml.jackson.core.JsonToken;
  * The listener should provide a clear error if a particular token is not
  * valid for a given listener.
  *
+ * <h4>Fields</h4>
+ *
+ * The structure of an object is:
+ * <ul>
+ * <li>{@code ObjectListener} which represents the object (tuple) as a whole.
+ * Each field, indexed by name, is represented as a</li>
+ * <li>{@code ValueListener} which represents the value "slot". That value
+ * can be scalar, or can be structured, in which case the value listener
+ * contains either a</li>
+ * <li>{@code ArrayListener} for an array, or a</li>
+ * <li>{@code ObjectListener} for a nested object (tuple).</li>
+ * </ul>
+ *
  * <h4>Nulls</h4>
  *
  * Null values are handled at the semantic, not syntax level. If the
@@ -82,23 +100,81 @@ import com.fasterxml.jackson.core.JsonToken;
  * the array is multi-dimensional, there will be multiple array/value
  * parser pairs: one for each dimension.
  */
-public class ObjectParser extends AbstractElementParser {
-  private final ObjectListener listener;
+public abstract class ObjectParser extends AbstractElementParser {
+  protected static final Logger logger = LoggerFactory.getLogger(ObjectParser.class);
+
   private final Map<String, ElementParser> members = CaseInsensitiveMap.newHashMap();
 
-  public ObjectParser(ElementParser parent, ObjectListener listener) {
-    super(parent);
-    this.listener = listener;
+  public ObjectParser(JsonStructureParser structParser) {
+    super(structParser);
   }
 
-  public ObjectListener listener() { return listener; }
+  @VisibleForTesting
+  public ElementParser fieldParser(String key) {
+    return members.get(key);
+  }
+
+  /**
+   * Called at the start of a set of values for an object. That is, called
+   * when the structure parser accepts the <code>{</code> token.
+   */
+  protected void onStart() { }
+
+  /**
+   * The structure parser has just encountered a new field for this
+   * object. This method returns a parser for the field, along with
+   * an optional listener to handle events within the field. The field typically
+   * uses a value parser create by the {@link FieldParserFactory} class.
+   * However, special cases (such as Mongo extended types) can create a
+   * custom parser.
+   * <p>
+   * If the field is not projected, the method should return a dummy parser
+   * from {@link FieldParserFactory#ignoredFieldParser()}.
+   * The dummy parser will "free-wheel" over whatever values the
+   * field contains. (This is one way to avoid structure errors in a JSON file:
+   * just ignore them.) Otherwise, the parser will look ahead to guess the
+   * field type and will call one of the "add" methods, each of which should
+   * return a value listener for the field itself.
+   * <p>
+   * A normal field will respond to the structure of the JSON file as it
+   * appears. The associated value listener receives events for the
+   * field value. The value listener may be asked to create additional
+   * structure, such as arrays or nested objects.
+   * <p>
+   * Parse position: <code>{ ... field : ^ ?</code> for a newly-seen field.
+   * Constructs a value parser and its listeners by looking ahead
+   * some number of tokens to "sniff" the type of the value. For
+   * example:
+   * <ul>
+   * <li>{@code foo: <value>} - Field value</li>
+   * <li>{@code foo: [ <value> ]} - 1D array value</li>
+   * <li>{@code foo: [ [<value> ] ]} - 2D array value</li>
+   * <li>Etc.</li>
+   * </ul>
+   * <p>
+   * There are two cases in which no type estimation is possible:
+   * <ul>
+   * <li>{@code foo: null}</li>
+   * <li>{@code foo: []}</li>
+   * </ul>
+   *
+   * @param field description of the field, including the field name
+   * @return a parser for the newly-created field
+   */
+  protected abstract ElementParser onField(String key, TokenIterator tokenizer);
+
+  /**
+   * Called at the end of a set of values for an object. That is, called
+   * when the structure parser accepts the <code>}</code> token.
+   */
+  protected void onEnd() { }
 
   /**
    * Parses <code>{ ^ ... }</code>
    */
   @Override
   public void parse(TokenIterator tokenizer) {
-    listener.onStart();
+    onStart();
 
     // Parse (field: value)* }
     top: while (true) {
@@ -122,7 +198,7 @@ public class ObjectParser extends AbstractElementParser {
           throw errorFactory().syntaxError(token);
       }
     }
-    listener.onEnd();
+    onEnd();
   }
 
   /**
@@ -139,7 +215,7 @@ public class ObjectParser extends AbstractElementParser {
       // New key; sniff the value to determine the parser to use
       // (which also tell us the kind of column to create in Drill.)
       // Position: key: ^
-      fieldParser = detectValueParser(tokenizer, key);
+      fieldParser = detectValueParser(key, tokenizer);
       members.put(key, fieldParser);
     }
     // Parse the field value.
@@ -155,66 +231,22 @@ public class ObjectParser extends AbstractElementParser {
    * @param key name of the field
    * @return parser for the field
    */
-  private ElementParser detectValueParser(TokenIterator tokenizer, final String key) {
+  private ElementParser detectValueParser(String key, TokenIterator tokenizer) {
     if (key.isEmpty()) {
       throw errorFactory().structureError(
           "Drill does not allow empty keys in JSON key/value pairs");
     }
-    FieldType type = listener.fieldType(key);
-    switch (type) {
-      case IGNORE:
-        return new DummyValueParser(this);
-      case JSON:
-        return new JsonValueParser(this, key,
-            listener.addField(key, new ValueDef(JsonType.STRING, 0)));
-      default:
-        return createFieldParser(key, type, tokenizer);
+    ElementParser fieldParser = onField(key, tokenizer);
+    if (fieldParser == null) {
+      logger.warn("No JSON element parser returned for field {}, assuming unprojected", key);
+      return DummyValueParser.INSTANCE;
+    } else {
+      return fieldParser;
     }
   }
 
-  /**
-   * Parse position: <code>{ ... field : ^ ?</code> for a newly-seen field.
-   * Constructs a value parser and its listeners by looking ahead
-   * some number of tokens to "sniff" the type of the value. For
-   * example:
-   * <ul>
-   * <li>{@code foo: <value>} - Field value</li>
-   * <li>{@code foo: [ <value> ]} - 1D array value</li>
-   * <li>{@code foo: [ [<value> ] ]} - 2D array value</li>
-   * <li>Etc.</li>
-   * </ul>
-   * <p>
-   * There are two cases in which no type estimation is possible:
-   * <ul>
-   * <li>The value is {@code null}, indicated by
-   * {@link JsonType#NULL}.</code>
-   * <li>The value is an array, and the array is empty, indicated
-   * by {@link JsonType#EMPTY}.</li>
-   * </ul>
-   * {@link ValueDefFactory} handles syntactic type inference. The associated
-   * listener enforces semantic rules. For example, if a schema is
-   * available, and we know that field "x" must be an Integer, but
-   * this class reports that it is an object, then the listener should
-   * raise an exception.
-   * <p>
-   * Also, the parser cannot enforce type consistency. This method
-   * looks only at the first appearance of a value: a sample size of
-   * one. JSON allows anything.
-   * The listener must enforce semantic rules that say whether a different
-   * type is allowed for later values.
-   *
-   * @param key the name of the field
-   * @param type the kind of field parser to create
-   * @param tokenizer the token parser
-   * @return the value parser for the element, which may contain additional
-   * structure for objects or arrays
-   */
-  public ElementParser createFieldParser(String key, FieldType type,
-      TokenIterator tokenizer) {
-    ValueParser fp = new ValueParser(this, key, type);
-    ValueDef valueDef = ValueDefFactory.lookAhead(tokenizer);
-    fp.bindListener(listener.addField(key, valueDef));
-    fp.expandStructure(valueDef);
-    return fp;
+  public ElementParser replaceFieldParser(String key, ElementParser fieldParser) {
+    members.put(key, fieldParser);
+    return fieldParser;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectValueParser.java
similarity index 55%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectValueParser.java
index a99dcec..975e0be 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectValueParser.java
@@ -17,26 +17,32 @@
  */
 package org.apache.drill.exec.store.easy.json.parser;
 
+import com.fasterxml.jackson.core.JsonToken;
 
-/**
- * Abstract base class for all JSON element parsers.
- */
-public abstract class AbstractElementParser implements ElementParser {
-  final JsonStructureParser structParser;
-  private final ElementParser parent;
+public class ObjectValueParser extends AbstractElementParser {
 
-  public AbstractElementParser(ElementParser parent) {
-    this.parent = parent;
-    this.structParser = parent.structParser();
-  }
+  private final ObjectParser objectParser;
 
-  @Override
-  public ElementParser parent() { return parent; }
+  public ObjectValueParser(ObjectParser objectParser) {
+    super(objectParser.structParser());
+    this.objectParser = objectParser;
+  }
 
+  /**
+   * Parses <code>{ ... }</code>
+   */
   @Override
-  public JsonStructureParser structParser() { return structParser; }
-
-  protected ErrorFactory errorFactory() {
-    return structParser.errorFactory();
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    switch (token) {
+      case START_OBJECT:
+        objectParser.parse(tokenizer);
+        break;
+      case VALUE_NULL:
+        // Silently ignore, treat as a missing field
+        break;
+      default:
+        throw errorFactory().structureError("JSON object expected");
+    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/RootParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/RootParser.java
index 1252c3e..36ac2f8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/RootParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/RootParser.java
@@ -33,7 +33,7 @@ import com.fasterxml.jackson.core.JsonToken;
  * requires newline separators between objects, this parser allows
  * any amount of whitespace, including none.
  */
-public abstract class RootParser implements ElementParser {
+public abstract class RootParser {
   protected static final Logger logger = LoggerFactory.getLogger(RootParser.class);
 
   private final JsonStructureParser structParser;
@@ -41,7 +41,7 @@ public abstract class RootParser implements ElementParser {
 
   public RootParser(JsonStructureParser structParser) {
     this.structParser = structParser;
-    this.rootObject = new ObjectParser(this, structParser.rootListener());
+    this.rootObject = structParser.fieldFactory().rootParser();
   }
 
   /**
@@ -54,14 +54,6 @@ public abstract class RootParser implements ElementParser {
    */
   public abstract boolean parseRoot(TokenIterator tokenizer);
 
-  // Generic parsing not allowed at the root since the root must
-  // report EOF. Use parseRoot() instead.
-  @Override
-  public void parse(TokenIterator tokenizer) {
-    throw new UnsupportedOperationException(
-        "Call parseRoot() at the root level to check for EOF.");
-  }
-
   /**
    * Parse one data object. This is the "root" object which may contain
    * nested objects. Called when the outer parser detects a start
@@ -94,12 +86,6 @@ public abstract class RootParser implements ElementParser {
     return structParser.errorFactory();
   }
 
-  @Override
-  public ElementParser parent() { return null; }
-
-  @Override
-  public JsonStructureParser structParser() { return structParser; }
-
   /**
    * Parser for a <a href="http://jsonlines.org/">jsonlines</a>-style
    * data set which consists of a series of objects. EOF from the parser
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ScalarValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ScalarValueParser.java
new file mode 100644
index 0000000..66d7e0f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ScalarValueParser.java
@@ -0,0 +1,83 @@
+/*
+ * 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.drill.exec.store.easy.json.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public abstract class ScalarValueParser extends ValueParser {
+
+  public ScalarValueParser(JsonStructureParser structParser, ValueListener listener) {
+    super(structParser, listener);
+  }
+
+  /**
+   * Parses <code>true | false | null | integer | float | string|
+   *              embedded-object</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    if (token.isScalarValue()) {
+      parseValue(tokenizer, token);
+    } else {
+      throw errorFactory().structureError("Structure value found where scalar expected");
+    }
+  }
+
+  protected abstract void parseValue(TokenIterator tokenizer, JsonToken token);
+
+  /**
+   * Parses <code>true | false | null | integer | float | string |<br>
+   *              embedded-object</code><br>
+   * and simply passes the value token on to the listener.
+   */
+  public static class SimpleValueParser extends ScalarValueParser {
+
+    public SimpleValueParser(JsonStructureParser structParser, ValueListener listener) {
+      super(structParser, listener);
+    }
+
+    @Override
+    public void parseValue(TokenIterator tokenizer, JsonToken token) {
+      listener.onValue(token, tokenizer);
+    }
+  }
+
+  /**
+   * Parses <code>true | false | null | integer | float | string |<br>
+   *              embedded-object</code>
+   * <p>
+   * Forwards the result as a string.
+   */
+  public static class TextValueParser extends ScalarValueParser {
+
+    public TextValueParser(JsonStructureParser structParser, ValueListener listener) {
+      super(structParser, listener);
+    }
+
+    @Override
+    public void parseValue(TokenIterator tokenizer, JsonToken token) {
+      if (token == JsonToken.VALUE_NULL) {
+        listener.onValue(token, tokenizer);
+      } else {
+        listener.onText(
+            token == JsonToken.VALUE_NULL ? null : tokenizer.textValue());
+      }
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/SimpleMessageParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/SimpleMessageParser.java
index b93f29e..f100a8f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/SimpleMessageParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/SimpleMessageParser.java
@@ -130,22 +130,7 @@ public class SimpleMessageParser implements MessageParser {
   }
 
   private void skipElement(TokenIterator tokenizer) {
-    int level = 0;
-    do {
-      JsonToken token = tokenizer.requireNext();
-      switch (token) {
-        case START_OBJECT:
-        case START_ARRAY:
-          level++;
-          break;
-        case END_OBJECT:
-        case END_ARRAY:
-          level--;
-          break;
-        default:
-          break;
-      }
-    } while (level > 0);
+    DummyValueParser.INSTANCE.parse(tokenizer);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/TokenIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/TokenIterator.java
index 5fbcc25..a7d6a34 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/TokenIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/TokenIterator.java
@@ -34,7 +34,7 @@ public class TokenIterator {
    * error is detected within a record. Allows for recovery.
    */
   @SuppressWarnings("serial")
-  class RecoverableJsonException extends RuntimeException {
+  public static class RecoverableJsonException extends RuntimeException {
   }
 
   private final JsonParser parser;
@@ -170,4 +170,20 @@ public class TokenIterator {
       throw errorFactory.typeError(e);
     }
   }
+
+  public byte[] binaryValue() {
+    try {
+      return parser.getBinaryValue();
+    } catch (JsonParseException e) {
+      throw errorFactory.syntaxError(e);
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    } catch (UnsupportedConversionError e) {
+      throw errorFactory.typeError(e);
+    }
+  }
+
+  public RuntimeException invalidValue(JsonToken token) {
+    return errorFactory.structureError("Unexpected JSON value: " + token.name());
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java
index 8896129..f95675a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDef.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.store.easy.json.parser;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
  * Description of a JSON value as inferred from looking ahead in
  * the JSON stream. Includes a type (which can be empty for an empty
@@ -59,11 +61,15 @@ public class ValueDef {
   }
 
   public static final ValueDef UNKNOWN_ARRAY = new ValueDef(JsonType.UNKNOWN, 1);
-  public static final ValueDef UNKNOWN = new ValueDef(JsonType.UNKNOWN, 0);
+  public static final ValueDef UNKNOWN = new ValueDef(JsonType.UNKNOWN);
 
   private final int arrayDims;
   private final JsonType type;
 
+  public ValueDef(JsonType type) {
+    this(type, 0);
+  }
+
   public ValueDef(JsonType type, int dims) {
     this.type = type;
     this.arrayDims = dims;
@@ -82,4 +88,29 @@ public class ValueDef {
     }
     return buf.toString();
   }
+
+  public static JsonType jsonTypeFor(JsonToken token) {
+    switch (token) {
+
+      case VALUE_NULL:
+        return JsonType.NULL;
+
+      case VALUE_FALSE:
+      case VALUE_TRUE:
+        return JsonType.BOOLEAN;
+
+      case VALUE_NUMBER_INT:
+        return JsonType.INTEGER;
+
+      case VALUE_NUMBER_FLOAT:
+        return JsonType.FLOAT;
+
+      case VALUE_STRING:
+      case VALUE_EMBEDDED_OBJECT:
+        return JsonType.STRING;
+
+      default:
+        throw new IllegalStateException("Not a scalar type: " + token.name());
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDefFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDefFactory.java
index 9013dd2..699061c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDefFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueDefFactory.java
@@ -72,27 +72,8 @@ public class ValueDefFactory {
         jsonType = JsonType.NULL;
         break;
 
-      case VALUE_FALSE:
-      case VALUE_TRUE:
-        jsonType = JsonType.BOOLEAN;
-        break;
-
-      case VALUE_NUMBER_INT:
-        jsonType = JsonType.INTEGER;
-        break;
-
-      case VALUE_NUMBER_FLOAT:
-        jsonType = JsonType.FLOAT;
-        break;
-
-      case VALUE_STRING:
-        jsonType = JsonType.STRING;
-        break;
-
       default:
-        // Won't get here: the Jackson parser catches
-        // errors.
-        throw tokenizer.errorFactory().syntaxError(token);
+        jsonType = ValueDef.jsonTypeFor(token);
     }
     tokenizer.unget(token);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueListener.java
index 94587d6..491d2c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueListener.java
@@ -17,8 +17,10 @@
  */
 package org.apache.drill.exec.store.easy.json.parser;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
- * Represents a JSON object, either a direct object field, or level
+ * Represents a JSON scalar value, either a direct object field, or level
  * within an array. That is:
  * <ul>
  * <li>{@code foo: <value>} - Field value</li>
@@ -27,115 +29,22 @@ package org.apache.drill.exec.store.easy.json.parser;
  * <li><code>foo: { ... }</code> - object</li>
  * <li><code>foo: [+ { ... } ]</code> - object array</li>
  * </ul>
- * <p>
- * A value listener appears at each level of an array. The top
- * and inner dimensions will provide an array listener, the bottom
- * level (outermost dimension) will see the value events.
- * <p>
- * A field value can be a scalar, an array or an object.
- * The structured types return a child listener specialized for that
- * type. The parser asks for the structured listener only once, when
- * building the parse tree for that structure. The scalar value
- * methods are called each time a value is parsed. Note that, for
- * any given row, it may be that no method is called if the field
- * does not appear in that record.
- * <p>
- * Object and array listeners are given contextual information when
- * adding fields or elements. JSON allows any value to appear in any
- * context. So, as the parse proceeds, the
- * parser may come across a structure different than the initial hint.
- * For example, the initial value might be null, and the later value
- * might be an array. The initial value might be an integer, but the
- * later value could be an object. It
- * is up to the listener implementation to decide whether to support
- * such structures. The implementation should log a warning, or throw
- * an exception, if it does not support a particular event.
- * <p>
- * JSON is flexible. It could be that the first value seen for an element
- * is {@code null} (or a scalar) and so the parser calls a scalar
- * method on the value listener. Perhaps the next value is an object or
- * an array. The parser focuses only on syntax: the JSON is what it is.
- * The parser simply asks the listener for an array or object listener
- * (then caches the resulting listener). The value listener is responsible
- * for semantics: deciding if it is valid to mix types in a field.
  */
 public interface ValueListener {
 
   /**
-   * Allows the object listener to revise the listener for a field,
-   * such as when a field starts null and resolves to some concrete
-   * type.
-   */
-  interface ValueHost {
-    void bindListener(ValueListener listener);
-  }
-
-  void bind(ValueHost host);
-
-  /**
-   * Called on parsing a {@code null} value for the field. Called whether
-   * the field is parsed as all-text or as typed values.
-   */
-  void onNull();
-
-  /**
-   * Called for the JSON {@code true} or {@code false} values when parsing
-   * the field as a typed field.
-   *
-   * @param value the Boolean value of the parsed token
-   */
-  void onBoolean(boolean value);
-
-  /**
-   * Called for JSON integer values when parsing the field as a typed
-   * field.
-   *
-   * @param value the integer value of the parsed token
-   */
-  void onInt(long value);
-
-  /**
-   * Called for JSON float values when parsing the field as a typed
-   * field.
-   *
-   * @param value the float value of the parsed token
-   */
-  void onFloat(double value);
-
-  /**
-   * Called for JSON string values when parsing the field as a typed
-   * field, and for all non-null scalar values when parsed in
-   * all-text mode
-   *
-   * @param value the string value of the parsed token
-   */
-  void onString(String value);
-
-  /**
-   * Called for embedded object values when parsing the field as a typed
-   * field.
-   * <p>
-   * Note: This method is for completeness with the entire set of JSON
-   * value tokens. It is not currently supported in Drill.
-   *
-   * @param value the string value of the parsed token
-   */
-  void onEmbeddedObject(String value);
-
-  /**
-   * The parser has encountered a object value for the field for the first
-   * time. That is: <code>foo: {</code>.
+   * Called for a JSON scalar token.
    *
-   * @return an object listener for the object
+   * @param token the scalar token
+   * @param tokenizer provides access to the value of the token
    */
-  ObjectListener object();
+  void onValue(JsonToken token, TokenIterator tokenizer);
 
   /**
-   * The parser has encountered a array value for the first time.
+   * Called when a parser converts a JSON structure to text rather
+   * than delivering the token directly.
    *
-   * @param valueDef description of the array dimensions (if
-   * a multi-dimensional array) and type (if known)
-   * @return an array listener for the array
+   * @param value the string value of the parsed token or structure
    */
-  ArrayListener array(ValueDef valueDef);
+  void onText(String value);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueParser.java
index 1cd0986..25d0abe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueParser.java
@@ -17,11 +17,6 @@
  */
 package org.apache.drill.exec.store.easy.json.parser;
 
-import org.apache.drill.exec.store.easy.json.parser.ObjectListener.FieldType;
-import org.apache.drill.exec.store.easy.json.parser.ValueListener.ValueHost;
-
-import com.fasterxml.jackson.core.JsonToken;
-
 /**
  * Parses a JSON value. JSON allows any value type to appear anywhere a
  * value is allowed; this parser reflects that rule. The associated listener
@@ -39,157 +34,12 @@ import com.fasterxml.jackson.core.JsonToken;
  * Listeners can enforce one type only, or can be more flexible and
  * allow multiple types.
  */
-public class ValueParser extends AbstractElementParser implements ValueHost {
-
-  private interface ValueHandler {
-    void accept(TokenIterator tokenizer, JsonToken token);
-  }
-
-  /**
-   * Parses <code>true | false | null | integer | float | string |<br>
-   *              embedded-object</code>
-   * <p>
-   * Forwards the result as a typed value.
-   */
-  public class TypedValueHandler implements ValueHandler {
-
-    @Override
-    public void accept(TokenIterator tokenizer, JsonToken token) {
-      switch (token) {
-        case VALUE_TRUE:
-          listener.onBoolean(true);
-          break;
-        case VALUE_FALSE:
-          listener.onBoolean(false);
-          break;
-        case VALUE_NUMBER_INT:
-          listener.onInt(tokenizer.longValue());
-          break;
-        case VALUE_NUMBER_FLOAT:
-          listener.onFloat(tokenizer.doubleValue());
-          break;
-        case VALUE_STRING:
-          listener.onString(tokenizer.stringValue());
-          break;
-        case VALUE_EMBEDDED_OBJECT:
-          listener.onEmbeddedObject(tokenizer.stringValue());
-        default:
-          // Won't get here: the Jackson parser catches
-          // errors.
-          throw errorFactory().syntaxError(token);
-      }
-    }
-  }
+public abstract class ValueParser extends AbstractElementParser {
 
-  /**
-   * Parses <code>true | false | null | integer | float | string |<br>
-   *              embedded-object</code>
-   * <p>
-   * Forwards the result as a string.
-   */
-  public class TextValueHandler implements ValueHandler {
-
-    @Override
-    public void accept(TokenIterator tokenizer, JsonToken token) {
-      switch (token) {
-        case VALUE_EMBEDDED_OBJECT:
-        case VALUE_FALSE:
-        case VALUE_TRUE:
-        case VALUE_NUMBER_FLOAT:
-        case VALUE_NUMBER_INT:
-        case VALUE_STRING:
-          listener.onString(tokenizer.textValue());
-          break;
-
-        default:
-          // Won't get here: the Jackson parser catches
-          // errors.
-          throw errorFactory().syntaxError(token);
-      }
-    }
-  }
-
-  private final String key;
-  protected final ValueHandler valueHandler;
-  private ValueListener listener;
-  private ObjectParser objectParser;
-  private ArrayParser arrayParser;
-
-  public ValueParser(ElementParser parent, String key, FieldType type) {
-    super(parent);
-    this.key = key;
-    if (type == FieldType.TEXT || structParser().options().allTextMode) {
-      valueHandler = new TextValueHandler();
-    } else {
-      valueHandler = new TypedValueHandler();
-    }
-  }
+  protected ValueListener listener;
 
-  @Override
-  public void bindListener(ValueListener listener) {
+  public ValueParser(JsonStructureParser structParser, ValueListener listener) {
+    super(structParser);
     this.listener = listener;
-    listener.bind(this);
-    if (arrayParser != null) {
-      arrayParser.bindListener(listener.array(ValueDef.UNKNOWN_ARRAY));
-    }
-  }
-
-  public String key() { return key; }
-
-  public ValueListener listener() { return listener; }
-
-  /**
-   * Parses <code>true | false | null | integer | float | string|
-   *              embedded-object | { ... } | [ ... ]</code>
-   */
-  @Override
-  public void parse(TokenIterator tokenizer) {
-    JsonToken token = tokenizer.requireNext();
-    switch (token) {
-    case START_OBJECT:
-      // Position: { ^
-      if (objectParser == null) {
-        // No object parser yet. May be that the value was null,
-        // or may be that it changed types.
-        addObjectParser();
-      }
-      objectParser.parse(tokenizer);
-      break;
-
-    case START_ARRAY:
-      // Position: [ ^
-      if (arrayParser == null) {
-        // No array parser yet. May be that the value was null,
-        // or may be that it changed types.
-        addArrayParser(ValueDefFactory.arrayLookAhead(tokenizer));
-      }
-      arrayParser.parse(tokenizer);
-      break;
-
-    case VALUE_NULL:
-      listener.onNull();
-      break;
-
-    default:
-      valueHandler.accept(tokenizer, token);
-    }
-  }
-
-  public void addObjectParser() {
-    objectParser = new ObjectParser(this, listener().object());
-  }
-
-  private void addArrayParser(ValueDef valueDef) {
-    ArrayListener arrayListener = listener().array(valueDef);
-    arrayParser = new ArrayParser(this, arrayListener);
-    arrayParser.expandStructure(valueDef);
-  }
-
-  public void expandStructure(ValueDef valueDef) {
-    if (valueDef.isArray()) {
-      addArrayParser(valueDef);
-    } else if (valueDef.type().isObject()) {
-      addObjectParser();
-    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonReader.java
index 983aa9f..cacfc34 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonReader.java
@@ -91,7 +91,6 @@ public abstract class BaseJsonReader extends BaseJsonProcessor {
     }
   }
 
-
   private ReadState writeToVector(ComplexWriter writer, JsonToken t)
     throws IOException {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BigIntListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BigIntListener.java
similarity index 66%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BigIntListener.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BigIntListener.java
index a5cc33e..bc2ad0c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BigIntListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BigIntListener.java
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
  * Listener for JSON integer values. Allows conversion from
  * Boolean, double and string types. (The conversion from double
@@ -34,22 +38,34 @@ public class BigIntListener extends ScalarListener {
   }
 
   @Override
-  public void onBoolean(boolean value) {
-    writer.setLong(value ? 1 : 0);
-  }
-
-  @Override
-  public void onInt(long value) {
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    long value;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_TRUE:
+        value = 1;
+        break;
+      case VALUE_FALSE:
+        value = 0;
+        break;
+      case VALUE_NUMBER_INT:
+        value = tokenizer.longValue();
+        break;
+      case VALUE_NUMBER_FLOAT:
+        value = Math.round(tokenizer.doubleValue());
+        break;
+      case VALUE_STRING:
+        parseString(tokenizer.stringValue());
+        return;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
     writer.setLong(value);
   }
 
-  @Override
-  public void onFloat(double value) {
-    writer.setLong(Math.round(value));
-  }
-
-  @Override
-  public void onString(String value) {
+  private void parseString(String value) {
     value = value.trim();
     if (value.isEmpty()) {
       setNull();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BinaryValueListener.java
similarity index 53%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BinaryValueListener.java
index 5f7549a..176e0b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BinaryValueListener.java
@@ -15,48 +15,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
-/**
- * Listener for JSON Boolean fields. Allows conversion from numeric
- * fields (with the usual semantics: 0 = false, ~0 = true) and from
- * strings (using Java Boolean parsing semantics.)
- */
-public class BooleanListener extends ScalarListener {
+import com.fasterxml.jackson.core.JsonToken;
 
-  public BooleanListener(JsonLoaderImpl loader, ScalarWriter writer) {
-    super(loader, writer);
-  }
-
-  @Override
-  public void onBoolean(boolean value) {
-    writer.setBoolean(value);
-  }
+public class BinaryValueListener extends ScalarListener {
 
-  @Override
-  public void onInt(long value) {
-    writer.setBoolean(value != 0);
-  }
+  private static final byte[] EMPTY_BYTES = new byte[0];
 
-  @Override
-  public void onFloat(double value) {
-    writer.setBoolean(value != 0);
+  public BinaryValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
   }
 
   @Override
-  public void onString(String value) {
-    value = value.trim();
-    if (value.isEmpty()) {
-      setNull();
-    } else {
-      writer.setBoolean(Boolean.parseBoolean(value.trim()));
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_STRING:
+        byte[] value = tokenizer.binaryValue();
+        writer.setBytes(value, value.length);
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
     }
   }
 
   @Override
   protected void setArrayNull() {
-    writer.setBoolean(false);
+    writer.setBytes(EMPTY_BYTES, 0);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BooleanListener.java
similarity index 60%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BooleanListener.java
index 5f7549a..3c933f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/BooleanListener.java
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
  * Listener for JSON Boolean fields. Allows conversion from numeric
  * fields (with the usual semantics: 0 = false, ~0 = true) and from
@@ -31,22 +35,36 @@ public class BooleanListener extends ScalarListener {
   }
 
   @Override
-  public void onBoolean(boolean value) {
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    boolean value;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_TRUE:
+        value = true;
+        break;
+      case VALUE_FALSE:
+        value = false;
+        break;
+      case VALUE_NUMBER_INT:
+        value = tokenizer.longValue() != 0;
+        break;
+      case VALUE_NUMBER_FLOAT:
+        value = tokenizer.doubleValue() != 0;
+        break;
+      case VALUE_STRING:
+        parseString(tokenizer.stringValue());
+        return;
+      default:
+        // Won't get here: the Jackson parser catches
+        // errors.
+        throw tokenizer.invalidValue(token);
+    }
     writer.setBoolean(value);
   }
 
-  @Override
-  public void onInt(long value) {
-    writer.setBoolean(value != 0);
-  }
-
-  @Override
-  public void onFloat(double value) {
-    writer.setBoolean(value != 0);
-  }
-
-  @Override
-  public void onString(String value) {
+  private void parseString(String value) {
     value = value.trim();
     if (value.isEmpty()) {
       setNull();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DateValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DateValueListener.java
new file mode 100644
index 0000000..8609a1e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DateValueListener.java
@@ -0,0 +1,70 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import java.time.Duration;
+import java.time.LocalDate;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parse local time dates. Stored internally in a local epoch
+ * offset from the local epoch, in ms. Does no time zone conversions,
+ * simply asserts that the date is in in the same time zone as the
+ * Drillbit.
+ */
+public class DateValueListener extends ScalarListener {
+
+  public DateValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+        writer.setLong(tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+
+          // A Drill date is ms since the epoch, local time. Our input
+          // is in UTC. We DO NOT want to convert from the date, midnight, UTC
+          // to local time since that will change the date. Instead, we just
+          // want to copy the offset since the epoch from UTC to our local
+          // time, so that we retain the date, even if the span of the date
+          // is different locally than UTC. A mess.
+          LocalDate localDate = LocalDate.parse(tokenizer.stringValue());
+          writer.setLong(Duration.between(TimestampValueListener.LOCAL_EPOCH,
+              localDate.atStartOfDay()).toMillis());
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "date", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DecimalValueListener.java
similarity index 50%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DecimalValueListener.java
index 5f7549a..76d51bf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DecimalValueListener.java
@@ -15,48 +15,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
-
-/**
- * Listener for JSON Boolean fields. Allows conversion from numeric
- * fields (with the usual semantics: 0 = false, ~0 = true) and from
- * strings (using Java Boolean parsing semantics.)
- */
-public class BooleanListener extends ScalarListener {
+import java.math.BigDecimal;
 
-  public BooleanListener(JsonLoaderImpl loader, ScalarWriter writer) {
-    super(loader, writer);
-  }
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
-  @Override
-  public void onBoolean(boolean value) {
-    writer.setBoolean(value);
-  }
+import com.fasterxml.jackson.core.JsonToken;
 
-  @Override
-  public void onInt(long value) {
-    writer.setBoolean(value != 0);
-  }
+public class DecimalValueListener extends ScalarListener {
 
-  @Override
-  public void onFloat(double value) {
-    writer.setBoolean(value != 0);
+  public DecimalValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
   }
 
   @Override
-  public void onString(String value) {
-    value = value.trim();
-    if (value.isEmpty()) {
-      setNull();
-    } else {
-      writer.setBoolean(Boolean.parseBoolean(value.trim()));
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+      case VALUE_NUMBER_FLOAT:
+      case VALUE_STRING:
+        try {
+          writer.setDecimal(new BigDecimal(tokenizer.textValue()));
+        } catch (NumberFormatException e) {
+          throw loader.dataConversionError(schema(), "DECIMAL", tokenizer.textValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
     }
   }
 
   @Override
   protected void setArrayNull() {
-    writer.setBoolean(false);
+    writer.setDecimal(BigDecimal.ZERO);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/DoubleListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DoubleListener.java
similarity index 63%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/DoubleListener.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DoubleListener.java
index 7daa42b..0a48fe3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/DoubleListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/DoubleListener.java
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
  * Listener for the JSON double type. Allows conversion from other
  * types. Conversion from Boolean is the usual semantics:
@@ -31,22 +35,36 @@ public class DoubleListener extends ScalarListener {
   }
 
   @Override
-  public void onBoolean(boolean value) {
-    writer.setDouble(value ? 1 : 0);
-  }
-
-  @Override
-  public void onInt(long value) {
-    writer.setDouble(value);
-  }
-
-  @Override
-  public void onFloat(double value) {
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    double value;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_TRUE:
+        value = 1;
+        break;
+      case VALUE_FALSE:
+        value = 0;
+        break;
+      case VALUE_NUMBER_INT:
+        value = tokenizer.longValue();
+        break;
+      case VALUE_NUMBER_FLOAT:
+        value = tokenizer.doubleValue();
+        break;
+      case VALUE_STRING:
+        parseString(tokenizer.stringValue());
+        return;
+      default:
+        // Won't get here: the Jackson parser catches
+        // errors.
+        throw tokenizer.invalidValue(token);
+    }
     writer.setDouble(value);
   }
 
-  @Override
-  public void onString(String value) {
+  private void parseString(String value) {
     value = value.trim();
     if (value.isEmpty()) {
       setNull();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/IntervalValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/IntervalValueListener.java
new file mode 100644
index 0000000..ec1018b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/IntervalValueListener.java
@@ -0,0 +1,57 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.joda.time.format.ISOPeriodFormat;
+import org.joda.time.format.PeriodFormatter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Drill-specific extension for a time interval (AKA time
+ * span or time period).
+ */
+public class IntervalValueListener extends ScalarListener {
+
+  public static final PeriodFormatter FORMATTER = ISOPeriodFormat.standard();
+
+  public IntervalValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        writer.setNull();
+        break;
+      case VALUE_STRING:
+        try {
+          writer.setPeriod(FORMATTER.parsePeriod(tokenizer.stringValue()));
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "date", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/ScalarListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/ScalarListener.java
new file mode 100644
index 0000000..b64cab0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/ScalarListener.java
@@ -0,0 +1,79 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.store.easy.json.parser.ValueListener;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Base class for scalar field listeners
+ */
+public abstract class ScalarListener implements ValueListener {
+
+  protected final JsonLoaderImpl loader;
+  protected final ScalarWriter writer;
+  protected final boolean isArray;
+
+  public ScalarListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    this.loader = loader;
+    this.writer = writer;
+    isArray = writer.schema().isArray();
+  }
+
+  public ColumnMetadata schema() { return writer.schema(); }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    throw typeConversionError(token.name());
+  }
+
+  @Override
+  public void onText(String value) {
+    throw typeConversionError("text");
+  }
+
+  protected void setNull() {
+    try {
+      if (isArray) {
+        setArrayNull();
+      } else {
+        writer.setNull();
+      }
+    } catch (UnsupportedConversionError e) {
+      throw loader.buildError(schema(),
+          UserException.dataReadError()
+            .message("Null value encountered in JSON input where Drill does not allow nulls."));
+    }
+  }
+
+  protected void setArrayNull() {
+    // Default is no "natural" null value
+    throw loader.nullDisallowedError(schema());
+  }
+
+  protected UserException typeConversionError(String jsonType) {
+    return loader.typeConversionError(schema(), jsonType);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictBigIntValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictBigIntValueListener.java
new file mode 100644
index 0000000..9143243
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictBigIntValueListener.java
@@ -0,0 +1,61 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * 64-bit integer (BIGINT) listener with conversions only from
+ * numbers and strings.
+ */
+public class StrictBigIntValueListener extends ScalarListener {
+
+  public StrictBigIntValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+        writer.setLong(tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+          writer.setLong(Long.parseLong(tokenizer.stringValue()));
+        } catch (NumberFormatException e) {
+          throw loader.dataConversionError(schema(), "string", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+
+  @Override
+  protected void setArrayNull() {
+    writer.setLong(0);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictDoubleValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictDoubleValueListener.java
new file mode 100644
index 0000000..6b69637
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictDoubleValueListener.java
@@ -0,0 +1,62 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+public class StrictDoubleValueListener extends ScalarListener {
+
+  public StrictDoubleValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    double value;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_NUMBER_INT:
+        value = tokenizer.longValue();
+        break;
+      case VALUE_NUMBER_FLOAT:
+        value = tokenizer.doubleValue();
+        break;
+      case VALUE_STRING:
+        try {
+          value = Double.parseDouble(tokenizer.stringValue());
+        } catch (NumberFormatException e) {
+          throw loader.dataConversionError(schema(), "string", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+    writer.setDouble(value);
+  }
+
+  @Override
+  protected void setArrayNull() {
+    writer.setDouble(0.0);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictIntValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictIntValueListener.java
new file mode 100644
index 0000000..895dbf2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictIntValueListener.java
@@ -0,0 +1,61 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * 32-bit integer (INT) listener with conversions only from
+ * numbers and strings.
+ */
+public class StrictIntValueListener extends ScalarListener {
+
+  public StrictIntValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+        writer.setInt((int) tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+          writer.setInt(Integer.parseInt(tokenizer.stringValue()));
+        } catch (NumberFormatException e) {
+          throw loader.dataConversionError(schema(), "string", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+
+  @Override
+  protected void setArrayNull() {
+    writer.setInt(0);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictStringValueListener.java
similarity index 53%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictStringValueListener.java
index 5f7549a..6d491e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/BooleanListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/StrictStringValueListener.java
@@ -15,48 +15,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
-/**
- * Listener for JSON Boolean fields. Allows conversion from numeric
- * fields (with the usual semantics: 0 = false, ~0 = true) and from
- * strings (using Java Boolean parsing semantics.)
- */
-public class BooleanListener extends ScalarListener {
-
-  public BooleanListener(JsonLoaderImpl loader, ScalarWriter writer) {
-    super(loader, writer);
-  }
+import com.fasterxml.jackson.core.JsonToken;
 
-  @Override
-  public void onBoolean(boolean value) {
-    writer.setBoolean(value);
-  }
+public class StrictStringValueListener extends ScalarListener {
 
-  @Override
-  public void onInt(long value) {
-    writer.setBoolean(value != 0);
-  }
-
-  @Override
-  public void onFloat(double value) {
-    writer.setBoolean(value != 0);
+  public StrictStringValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
   }
 
   @Override
-  public void onString(String value) {
-    value = value.trim();
-    if (value.isEmpty()) {
-      setNull();
-    } else {
-      writer.setBoolean(Boolean.parseBoolean(value.trim()));
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_STRING:
+        writer.setString(tokenizer.stringValue());
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
     }
   }
 
   @Override
   protected void setArrayNull() {
-    writer.setBoolean(false);
+    writer.setString("");
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimeValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimeValueListener.java
new file mode 100644
index 0000000..31d2fb7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimeValueListener.java
@@ -0,0 +1,65 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Drill-specific extension to allow times only.
+ */
+public class TimeValueListener extends ScalarListener {
+
+  // This uses the Java-provided formatter which handles
+  // HH:MM:SS[.SSS][ZZZ]
+  // The Drill-provided formatters in DateUtility are close, but don't
+  // work for both the Mongo-format and Drill-format times.
+  private static final DateTimeFormatter TIME_FORMAT = DateTimeFormatter.ISO_TIME;
+
+  public TimeValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+        writer.setInt((int) tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+          LocalTime localTime = LocalTime.parse(tokenizer.stringValue(), TIME_FORMAT);
+          writer.setInt((int) ((localTime.toNanoOfDay() + 500_000L) / 1_000_000L)); // round to milliseconds
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "string", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimestampValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimestampValueListener.java
new file mode 100644
index 0000000..56a998e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/TimestampValueListener.java
@@ -0,0 +1,63 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import java.time.Duration;
+import java.time.LocalDateTime;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Drill-flavored version of a timestamp parser. Assumes the date time is in
+ * a local (unspecified) time zone, interpreted to be the default time zone
+ * of the Drillbit machine. Does no time zone conversions.
+ */
+public class TimestampValueListener extends ScalarListener {
+
+  public static final LocalDateTime LOCAL_EPOCH = LocalDateTime.of(1970, 1, 1, 0, 0, 0);
+
+  public TimestampValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_NUMBER_INT:
+        writer.setLong(tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+          LocalDateTime localDT = LocalDateTime.parse(tokenizer.stringValue());
+          writer.setLong(Duration.between(LOCAL_EPOCH, localDT).toMillis());
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "date", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcDateValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcDateValueListener.java
new file mode 100644
index 0000000..ab45ef3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcDateValueListener.java
@@ -0,0 +1,73 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import java.time.LocalDate;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Drill-specific extension to allow dates only, expressed in UTC
+ * to be consistent with Mongo timestamps.
+ * <p>
+ * Drill dates are in the local time zone, so conversion is needed.
+ * Drill dates are stored in ms, which is odd.
+ */
+public class UtcDateValueListener extends ScalarListener {
+
+  public UtcDateValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        break;
+      case VALUE_NUMBER_INT:
+        writer.setLong(tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+
+          // A Drill date is ms since the epoch, local time. Our input
+          // is in UTC. We DO NOT want to convert from the date, midnight, UTC
+          // to local time since that will change the date. Instead, we just
+          // want to copy the offset since the epoch from UTC to our local
+          // time, so that we retain the date, even if the span of the date
+          // is different locally than UTC. A mess.
+          LocalDate localDate = LocalDate.parse(tokenizer.stringValue(), DateUtility.isoFormatDate);
+          ZonedDateTime utc = localDate.atStartOfDay(ZoneOffset.UTC);
+          writer.setLong(utc.toEpochSecond() * 1000);
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "date", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcTimestampValueListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcTimestampValueListener.java
new file mode 100644
index 0000000..662e87d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/UtcTimestampValueListener.java
@@ -0,0 +1,72 @@
+/*
+ * 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.drill.exec.store.easy.json.values;
+
+import java.time.Instant;
+import java.time.ZoneId;
+
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.exec.vector.accessor.ScalarWriter;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Per the <a href="https://docs.mongodb.com/manual/reference/mongodb-extended-json-v1/#bson.data_date">
+ * V1 docs</a>:
+ * <quote>
+ * In Strict mode, {@code <date>} is an ISO-8601 date format with a mandatory time zone field
+ * following the template YYYY-MM-DDTHH:mm:ss.mmm<+/-Offset>.
+ * <p>
+ * In Shell mode, {@code <date>} is the JSON representation of a 64-bit signed
+ * integer giving the number of milliseconds since epoch UTC.
+ * </quote>
+ * <p>
+ * Drill dates are in the local time zone, so conversion is needed.
+ */
+public class UtcTimestampValueListener extends ScalarListener {
+
+  private static final ZoneId LOCAL_ZONE_ID = ZoneId.systemDefault();
+
+  public UtcTimestampValueListener(JsonLoaderImpl loader, ScalarWriter writer) {
+    super(loader, writer);
+  }
+
+  @Override
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    Instant instant;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_NUMBER_INT:
+        instant = Instant.ofEpochMilli(tokenizer.longValue());
+        break;
+      case VALUE_STRING:
+        try {
+          instant = Instant.parse(tokenizer.stringValue());
+        } catch (Exception e) {
+          throw loader.dataConversionError(schema(), "date", tokenizer.stringValue());
+        }
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+    writer.setLong(instant.toEpochMilli() + LOCAL_ZONE_ID.getRules().getOffset(instant).getTotalSeconds() * 1000);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VarCharListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/VarCharListener.java
similarity index 59%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VarCharListener.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/VarCharListener.java
index 1e6cd31..f0b88e2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/loader/VarCharListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/values/VarCharListener.java
@@ -15,10 +15,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.easy.json.loader;
+package org.apache.drill.exec.store.easy.json.values;
 
+import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
 import org.apache.drill.exec.vector.accessor.ScalarWriter;
 
+import com.fasterxml.jackson.core.JsonToken;
+
 /**
  * Value listener for JSON string values. Allows conversion from
  * other scalar types using the Java {@code toString()} semantics.
@@ -35,23 +39,40 @@ public class VarCharListener extends ScalarListener {
   }
 
   @Override
-  public void onBoolean(boolean value) {
-    writer.setString(Boolean.toString(value));
-  }
-
-  @Override
-  public void onInt(long value) {
-    writer.setString(Long.toString(value));
-  }
-
-  @Override
-  public void onFloat(double value) {
-    writer.setString(Double.toString(value));
+  public void onValue(JsonToken token, TokenIterator tokenizer) {
+    String value;
+    switch (token) {
+      case VALUE_NULL:
+        setNull();
+        return;
+      case VALUE_TRUE:
+        value = Boolean.TRUE.toString();
+        break;
+      case VALUE_FALSE:
+        value = Boolean.FALSE.toString();
+        break;
+      case VALUE_NUMBER_INT:
+        value = Long.toString(tokenizer.longValue());
+        break;
+      case VALUE_NUMBER_FLOAT:
+        value = Double.toString(tokenizer.doubleValue());
+        break;
+      case VALUE_STRING:
+        value = tokenizer.stringValue();
+        break;
+      default:
+        throw tokenizer.invalidValue(token);
+    }
+    writer.setString(value);
   }
 
   @Override
-  public void onString(String value) {
-    writer.setString(value);
+  public void onText(String value) {
+    if (value == null) {
+      setNull();
+    } else {
+      writer.setString(value);
+    }
   }
 
   @Override
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/BaseJsonLoaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/BaseJsonLoaderTest.java
index 17cdc06..054a440 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/BaseJsonLoaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/BaseJsonLoaderTest.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.physical.resultSet.ResultSetLoader;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetLoaderImpl;
 import org.apache.drill.exec.physical.resultSet.impl.ResultSetOptionBuilder;
 import org.apache.drill.exec.physical.rowSet.RowSet;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.easy.json.loader.JsonLoaderImpl.JsonLoaderBuilder;
 import org.apache.drill.test.SubOperatorTest;
 
@@ -36,17 +35,16 @@ public class BaseJsonLoaderTest extends SubOperatorTest {
   protected static class JsonLoaderFixture {
 
     public ResultSetOptionBuilder rsLoaderOptions = new ResultSetOptionBuilder();
-    public TupleMetadata providedSchema;
+    public JsonLoaderBuilder builder = new JsonLoaderBuilder();
     public JsonLoaderOptions jsonOptions = new JsonLoaderOptions();
-    public CustomErrorContext errorContext = new EmptyErrorContext();
+    public CustomErrorContext errorContext = EmptyErrorContext.INSTANCE;
     private ResultSetLoader rsLoader;
     private JsonLoader loader;
 
     public void open(InputStream is) {
       rsLoader = new ResultSetLoaderImpl(fixture.allocator(), rsLoaderOptions.build());
-      loader = new JsonLoaderBuilder()
+      loader = builder
           .resultSetLoader(rsLoader)
-          .providedSchema(providedSchema)
           .options(jsonOptions)
           .errorContext(errorContext)
           .fromStream(is)
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestBasics.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestBasics.java
new file mode 100644
index 0000000..10fcbac
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestBasics.java
@@ -0,0 +1,238 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.JsonTest;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(JsonTest.class)
+public class TestBasics extends BaseJsonLoaderTest {
+
+  @Test
+  public void testEmpty() {
+    String json = "";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNull(results);
+    loader.close();
+  }
+
+  @Test
+  public void testEmptyTuple() {
+    final String json = "{} {} {}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertEquals(3, results.rowCount());
+    assertTrue(results.schema().isEmpty());
+    assertNotNull(results);
+    loader.close();
+  }
+
+  @Test
+  public void testRootArray() {
+    final String json = "[{a: 10}, {a: 20}, {a: 30}]";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10)
+        .addRow(20)
+        .addRow(30)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testLeadingTrailingWhitespace() {
+    final String json = "{\" a\": 10, \" b\": 20, \" c \": 30}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .addNullable("b", MinorType.BIGINT)
+        .addNullable("c", MinorType.BIGINT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, 20, 30)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testCaseInsensitive() {
+    final String json = "{a: 10, Bob: 110} {A: 20, bOb: 120} {\" a \": 30, BoB: 130}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .addNullable("Bob", MinorType.BIGINT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, 110)
+        .addRow(20, 120)
+        .addRow(30, 130)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testProjection() {
+    String json =
+        "{a: 10, b: true}\n" +
+        "{a: 20, b: [\"what?\"]}\n" +
+        "{a: 30, b: {c: \"oh, my!\"}}" +
+        "{a: 40}" +
+        "{a: 50, b: [[{x: [[{y: []}]]}]]}";
+
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.rsLoaderOptions.projection(
+        Projections.parse(RowSetTestUtils.projectList("a")));
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10)
+        .addRow(20)
+        .addRow(30)
+        .addRow(40)
+        .addRow(50)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testMissingEndObject() {
+    expectError("{a: 0} {a: 100", "Error parsing JSON");
+  }
+
+  @Test
+  public void testMissingValue() {
+    expectError("{a: 0} {a: ", "Error parsing JSON");
+  }
+
+  /**
+   * When parsing an array, the Jackson JSON parser raises
+   * an error for a missing close bracket.
+   */
+  @Test
+  public void testMissingEndOuterArray() {
+    expectError("[{a: 0}, {a: 100}", "Error parsing JSON");
+  }
+
+  @Test
+  public void testEmptyKey() {
+    expectError("{\"\": 10}", "does not allow empty keys");
+  }
+
+  @Test
+  public void testBlankKey() {
+    expectError("{\"  \": 10}", "does not allow empty keys");
+  }
+
+  @Test
+  public void testRootArrayDisallowed() {
+    final String json = "[{a: 0}, {a: 100}, {a: null}]";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.skipOuterList = false;
+    try {
+      loader.open(json);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("outer array support is not enabled"));
+    }
+  }
+
+  protected static void expectError(String json, String msg) {
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(msg));
+    }
+    loader.close();
+  }
+
+  /**
+   * Test syntax error recover. Recovery is not perfect. The
+   * input contains six records: the second is bad. But, the parser
+   * consumes records 3 and 4 trying to recover.
+   */
+  @Test
+  public void testRecovery() {
+    final String json = "{a: 1} {a: {a: 3} {a: 4} {a: 5} {a: 6}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.skipMalformedRecords = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(1)
+        .addRow(5)
+        .addRow(6)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedArrays.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedArrays.java
new file mode 100644
index 0000000..62a4a02
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedArrays.java
@@ -0,0 +1,570 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.TimeZone;
+
+import static org.apache.drill.test.rowSet.RowSetUtilities.dec;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
+import static org.apache.drill.test.rowSet.RowSetUtilities.longArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.intArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.decArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.doubleArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.binArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.objArray;
+
+import org.apache.drill.categories.JsonTest;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(JsonTest.class)
+public class TestExtendedArrays extends BaseJsonLoaderTest {
+
+  @Test
+  public void testInt() {
+    String json =
+        "{ a: [ { \"$numberInt\": 10 }, 20, { \"$numberInt\": \"30\" } ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        "{ a: [ { \"$numberInt\": 40 }, \"50\", null ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(intArray(10, 20, 30))
+        .addSingleCol(intArray())
+        .addSingleCol(intArray())
+        .addSingleCol(intArray(40, 50, 0))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testLong() {
+    String json =
+        "{ a: [ { \"$numberLong\": 10 }, 20, { \"$numberLong\": \"30\" } ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        "{ a: [ { \"$numberLong\": 40 }, \"50\", null ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.BIGINT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(longArray(10L, 20L, 30L))
+        .addSingleCol(longArray())
+        .addSingleCol(longArray())
+        .addSingleCol(longArray(40L, 50L, 0L))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDecimal() {
+    String json =
+        "{ a: [ { \"$numberDecimal\": 10 }, null, { \"$numberDecimal\": \"30\" }, " +
+        "       { \"$numberDecimal\": 40.2345 } ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        "{ a: [ 60, \"70.890\", 80.765 ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.VARDECIMAL, 38, 10)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(decArray(dec("10"), dec("0"), dec("30"), dec("40.2345")))
+        .addSingleCol(decArray())
+        .addSingleCol(decArray())
+        .addSingleCol(decArray(dec("60"), dec("70.89"), dec("80.765")))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDouble() {
+    String json =
+        "{ a: [ { \"$numberDouble\": 10 }, null, { \"$numberDouble\": \"30\" } ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        "{ a: [ { \"$numberDouble\": 40.125 }, 60, \"70.125\", 80.375 ] }\n" +
+        "{ a: [ { \"$numberDouble\": \"-Infinity\" }, " +
+        "       { \"$numberDouble\": \"Infinity\" }," +
+        "       { \"$numberDouble\": \"NaN\" } ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.FLOAT8)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(doubleArray(10D, 0D, 30D))
+        .addSingleCol(doubleArray())
+        .addSingleCol(doubleArray())
+        .addSingleCol(doubleArray(40.125D, 60D, 70.125D, 80.375D))
+        .addSingleCol(doubleArray(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NaN))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDate() {
+    LocalDateTime local = LocalDateTime.of(2020, 4, 21, 11, 22, 33);
+    Instant instant = local.atZone(ZoneId.systemDefault()).toInstant();
+    long ts = instant.toEpochMilli();
+    String utc = DateTimeFormatter.ISO_INSTANT.format(instant);
+    long localTs = ts + TimeZone.getDefault().getOffset(ts);
+    String json =
+        // V1 string, V2 relaxed
+        "{ a: [ { \"$date\": \"" + utc + "\" },\n" +
+        // V1 "shell mode"
+        "       { \"$date\": " + ts + " } ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        // V2 canonical
+        "{ a: [ { \"$date\": { \"$numberLong\": " + ts + " } },\n" +
+        // Harmless extensions, only valid after the above
+        "      " + ts + ",\n" +
+        "      \"" + utc + "\" ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.TIMESTAMP)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(longArray(localTs, localTs))
+        .addSingleCol(longArray())
+        .addSingleCol(longArray())
+        .addSingleCol(longArray(localTs, localTs, localTs))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDateNull() {
+    LocalDateTime local = LocalDateTime.of(2020, 4, 21, 11, 22, 33);
+    Instant instant = local.atZone(ZoneId.systemDefault()).toInstant();
+    String utc = DateTimeFormatter.ISO_INSTANT.format(instant);
+    String json =
+        "{ a: [ { \"$date\": \"" + utc + "\" }, null ] }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("does not allow null values"));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testBinary() {
+    String json =
+        // V2 format
+        "{ a: [ { \"$binary\": { base64: \"ZHJpbGw=\", subType: \"0\" } },\n" +
+        "       { \"$binary\": { subType: \"0\", base64: \"ZHJpbGw=\" } },\n" +
+        // Harmless extension
+        "       { \"$binary\": { base64: \"ZHJpbGw=\" } }, null ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        // V1 format
+        "{ a: [ { \"$binary\": \"ZHJpbGw=\", \"$type\": 1 },\n" +
+        // Harmless extension
+        "       { \"$binary\": \"ZHJpbGw=\" },\n" +
+        // Only valid after the above
+        "       \"ZHJpbGw=\" ] }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.VARBINARY)
+        .build();
+    byte[] bytes = "Drill".getBytes();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(binArray(bytes, bytes, bytes, new byte[] { }))
+        .addSingleCol(binArray())
+        .addSingleCol(binArray())
+        .addSingleCol(binArray(bytes, bytes, bytes))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testObjectID() {
+    String json =
+        "{ a: [ { \"$oid\": \"foo\" },\n" +
+        // Harmless extension. A Real OID can't be a "blank"
+        // value, but here we just store it as a string.
+        "       null ] }\n" +
+        "{ a: null }\n" +
+        "{ a: [] }\n" +
+        // Only valid after the above
+        "{ a: [ \"foo\" ] }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.VARCHAR)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(strArray("foo", ""))
+        .addSingleCol(strArray())
+        .addSingleCol(strArray())
+        .addSingleCol(strArray("foo"))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // A Mongo document is just a regular JSON map.
+  @Test
+  public void testDocument() {
+    String json =
+        "{ m: [ { a: { \"$numberLong\": 10 }, b: \"foo\" },\n" +
+        "       { a: { \"$numberLong\": \"20\" }, b: null },\n" +
+        "       { a: 30 } ] }\n" +
+        // Harmless extension
+        "{ m: null }\n" +
+        "{ m: [] }\n" +
+        "{ m: [ null, { a: { \"$numberLong\": 40 }, b: \"bar\" } ] }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addMapArray("m")
+          .addNullable("a", MinorType.BIGINT)
+          .addNullable("b", MinorType.VARCHAR)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(objArray(mapValue(10L, "foo"),
+            mapValue(20L, null), mapValue(30L, null)))
+        .addSingleCol(objArray())
+        .addSingleCol(objArray())
+        .addSingleCol(objArray(mapValue(null, null), mapValue(40L, "bar")))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: date only
+  @Test
+  public void testDateDay() {
+    String json =
+        "{ a: { \"$dateDay\": \"2020-04-21\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"2020-04-21\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.DATE)
+        .build();
+    org.joda.time.LocalDate date = new org.joda.time.LocalDate(2020, 04, 21);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(date)
+        .addSingleCol(null)
+        .addRow(date)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time only
+  @Test
+  public void testTime() {
+    String json =
+        "{ a: { \"$time\": \"11:22:33\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"11:22:33\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.TIME)
+        .build();
+    org.joda.time.LocalTime time  = new org.joda.time.LocalTime(11, 22, 33);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(time)
+        .addSingleCol(null)
+        .addRow(time)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time interval
+  @Test
+  public void testInterval() {
+    String json =
+        "{ a: { \"$interval\": \"P1Y2M3DT4H5M6S\" } }\n" +
+        "{ a: { \"$interval\": \"P1Y2M3D\" } }\n" +
+        "{ a: { \"$interval\": \"PT4H5M6S\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"P1Y2M3DT4H5M6S\" }\n" +
+        "{ a: \"P1Y2M3D\" }\n" +
+        "{ a: \"PT4H5M6S\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.INTERVAL)
+        .build();
+    org.joda.time.Period full = org.joda.time.Period.years(1).withMonths(2).withDays(3).withHours(4).withMinutes(5).withSeconds(6);
+    org.joda.time.Period ymd = org.joda.time.Period.years(1).withMonths(2).withDays(3);
+    org.joda.time.Period hms = org.joda.time.Period.hours(4).withMinutes(5).withSeconds(6);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(full)
+        .addRow(ymd)
+        .addRow(hms)
+        .addSingleCol(null)
+        .addRow(full)
+        .addRow(ymd)
+        .addRow(hms)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testNonExtended() {
+    String json =
+        "{ a: 10, b: { }, c: { d: 30 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .addMap("b")
+          .resumeSchema()
+        .addMap("c")
+          .addNullable("d", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, mapValue(), mapValue(30))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testUnknownType() {
+    String json =
+        "{ a: { \"$bogus\": 10 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addMap("a")
+          .addNullable("$bogus", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(mapValue(10))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  private final String LONG_HINT = "<{\"$numberLong\": scalar}>";
+  @Test
+  public void testInvalidTypeToken() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: [ ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidTypeObject() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidTypeName() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$bogus\": 20 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidValueToken() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": [ ] } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidValue() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": 20.3 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("Unexpected JSON value: VALUE_NUMBER_FLOAT"));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testExtraField() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": 20, bogus: 30 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedTypes.java
new file mode 100644
index 0000000..866aea2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedTypes.java
@@ -0,0 +1,562 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.TimeZone;
+
+import static org.apache.drill.test.rowSet.RowSetUtilities.dec;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
+
+import org.apache.drill.categories.JsonTest;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(JsonTest.class)
+public class TestExtendedTypes extends BaseJsonLoaderTest {
+
+  @Test
+  public void testInt() {
+    String json =
+        "{ a: { \"$numberInt\": 10 } }\n" +
+        "{ a: null }\n" +
+        "{ a: { \"$numberInt\": \"30\" } }\n" +
+        "{ a: 40 }\n" +
+        "{ a: \"50\" }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.INT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10)
+        .addSingleCol(null)
+        .addRow(30)
+        .addRow(40)
+        .addRow(50)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testLong() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: null }\n" +
+        "{ a: { \"$numberLong\": \"30\" } }\n" +
+        "{ a: 40 }\n" +
+        "{ a: \"50\" }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10L)
+        .addSingleCol(null)
+        .addRow(30L)
+        .addRow(40L)
+        .addRow(50L)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDecimal() {
+    String json =
+        "{ a: { \"$numberDecimal\": 10 } }\n" +
+        "{ a: null }\n" +
+        "{ a: { \"$numberDecimal\": \"30\" } }\n" +
+        "{ a: { \"$numberDecimal\": 40.2345 } }\n" +
+        "{ a: 60 }\n" +
+        "{ a: \"70.890\" }\n" +
+        "{ a: 80.765 }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.VARDECIMAL, 38, 10)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(dec("10"))
+        .addSingleCol(null)
+        .addRow(dec("30"))
+        .addRow(dec("40.2345"))
+        .addRow(dec("60"))
+        .addRow(dec("70.89"))
+        .addRow(dec("80.765"))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDouble() {
+    String json =
+        "{ a: { \"$numberDouble\": 10 } }\n" +
+        "{ a: null }\n" +
+        "{ a: { \"$numberDouble\": \"30\" } }\n" +
+        "{ a: { \"$numberDouble\": 40.125 } }\n" +
+        "{ a: 60 }\n" +
+        "{ a: \"70.125\" }\n" +
+        "{ a: 80.375 }\n" +
+        "{ a: { \"$numberDouble\": \"-Infinity\" } }\n" +
+        "{ a: { \"$numberDouble\": \"Infinity\" } }\n" +
+        "{ a: { \"$numberDouble\": \"NaN\" } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.FLOAT8)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10D)
+        .addSingleCol(null)
+        .addRow(30D)
+        .addRow(40.125D)
+        .addRow(60D)
+        .addRow(70.125D)
+        .addRow(80.375D)
+        .addRow(Double.NEGATIVE_INFINITY)
+        .addRow(Double.POSITIVE_INFINITY)
+        .addRow(Double.NaN)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDate() {
+    LocalDateTime local = LocalDateTime.of(2020, 4, 21, 11, 22, 33);
+    Instant instant = local.atZone(ZoneId.systemDefault()).toInstant();
+    long ts = instant.toEpochMilli();
+    String utc = DateTimeFormatter.ISO_INSTANT.format(instant);
+    long localTs = ts + TimeZone.getDefault().getOffset(ts);
+    String json =
+        // V1 string, V2 relaxed
+        "{ a: { \"$date\": \"" + utc + "\" } }\n" +
+        // V1 "shell mode"
+        "{ a: { \"$date\": " + ts + " } }\n" +
+        "{ a: null }\n" +
+        // V2 canonical
+        "{ a: { \"$date\": { \"$numberLong\": " + ts + " } } }\n" +
+        // Harmless extensions, only valid after the above
+        "{ a: " + ts + " }\n" +
+        "{ a: \"" + utc + "\" }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.TIMESTAMP)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(localTs)
+        .addRow(localTs)
+        .addSingleCol(null)
+        .addRow(localTs)
+        .addRow(localTs)
+        .addRow(localTs)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testBinary() {
+    String json =
+        // V2 format
+        "{ a: { \"$binary\": { base64: \"ZHJpbGw=\", subType: \"0\" } } }\n" +
+        "{ a: { \"$binary\": { subType: \"0\", base64: \"ZHJpbGw=\" } } }\n" +
+        // Harmless extension
+        "{ a: { \"$binary\": { base64: \"ZHJpbGw=\" } } }\n" +
+        "{ a: null }\n" +
+        // V1 format
+        "{ a: { \"$binary\": \"ZHJpbGw=\", \"$type\": 1 } }\n" +
+        // Drill-supported variation of V1
+        "{ a: { \"$type\": 1, \"$binary\": \"ZHJpbGw=\" } }\n" +
+        // Harmless extension
+        "{ a: { \"$binary\": \"ZHJpbGw=\" } }\n" +
+        // Only valid after the above
+        "{ a: \"ZHJpbGw=\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.VARBINARY)
+        .build();
+    byte[] bytes = "Drill".getBytes();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(bytes)
+        .addRow(bytes)
+        .addRow(bytes)
+        .addSingleCol(null)
+        .addRow(bytes)
+        .addRow(bytes)
+        .addRow(bytes)
+        .addRow(bytes)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testObjectID() {
+    String json =
+        "{ a: { \"$oid\": \"foo\" } }\n" +
+        // Harmless extension
+        "{ a: null }\n" +
+        // Only valid after the above
+        "{ a: \"foo\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.VARCHAR)
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow("foo")
+        .addSingleCol(null)
+        .addRow("foo")
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // A Mongo document is just a regular JSON map.
+  @Test
+  public void testDocument() {
+    String json =
+        "{ m: { a: { \"$numberLong\": 10 }, b: \"foo\" } }\n" +
+        // Harmless extension
+        "{ m: null }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addMap("m")
+          .addNullable("a", MinorType.BIGINT)
+          .addNullable("b", MinorType.VARCHAR)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(mapValue(10L, "foo"))
+        .addSingleCol(mapValue(null, null))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: date only
+  @Test
+  public void testDateDay() {
+    String json =
+        "{ a: { \"$dateDay\": \"2020-04-21\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"2020-04-21\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.DATE)
+        .build();
+    org.joda.time.LocalDate date = new org.joda.time.LocalDate(2020, 04, 21);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(date)
+        .addSingleCol(null)
+        .addRow(date)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time only
+  @Test
+  public void testTime() {
+    String json =
+        "{ a: { \"$time\": \"11:22:33\" } }\n" +
+        "{ a: { \"$time\": \"11:22:33.123\" } }\n" +
+        // Drill's assumed format, though not really valid
+        "{ a: { \"$time\": \"11:22:33.123Z\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"11:22:33\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.TIME)
+        .build();
+    org.joda.time.LocalTime time  = new org.joda.time.LocalTime(11, 22, 33);
+    org.joda.time.LocalTime time2  = new org.joda.time.LocalTime(11, 22, 33, 123);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(time)
+        .addRow(time2)
+        .addRow(time2)
+        .addSingleCol(null)
+        .addRow(time)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time interval
+  @Test
+  public void testInterval() {
+    String json =
+        "{ a: { \"$interval\": \"P1Y2M3DT4H5M6S\" } }\n" +
+        "{ a: { \"$interval\": \"P1Y2M3D\" } }\n" +
+        "{ a: { \"$interval\": \"PT4H5M6S\" } }\n" +
+        "{ a: null }\n" +
+        "{ a: \"P1Y2M3DT4H5M6S\" }\n" +
+        "{ a: \"P1Y2M3D\" }\n" +
+        "{ a: \"PT4H5M6S\" }\n";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.INTERVAL)
+        .build();
+    org.joda.time.Period full = org.joda.time.Period.years(1).withMonths(2).withDays(3).withHours(4).withMinutes(5).withSeconds(6);
+    org.joda.time.Period ymd = org.joda.time.Period.years(1).withMonths(2).withDays(3);
+    org.joda.time.Period hms = org.joda.time.Period.hours(4).withMinutes(5).withSeconds(6);
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(full)
+        .addRow(ymd)
+        .addRow(hms)
+        .addSingleCol(null)
+        .addRow(full)
+        .addRow(ymd)
+        .addRow(hms)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testNonExtended() {
+    String json =
+        "{ a: 10, b: { }, c: { d: 30 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+        .addMap("b")
+          .resumeSchema()
+        .addMap("c")
+          .addNullable("d", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(10, mapValue(), mapValue(30))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testUnknownType() {
+    String json =
+        "{ a: { \"$bogus\": 10 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addMap("a")
+          .addNullable("$bogus", MinorType.BIGINT)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(mapValue(10))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  private final String LONG_HINT = "<{\"$numberLong\": scalar}>";
+  @Test
+  public void testInvalidTypeToken() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: [ ] }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidTypeObject() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidTypeName() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$bogus\": 20 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidValueToken() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": [ ] } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testInvalidValue() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": 20.3 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("Unexpected JSON value: VALUE_NUMBER_FLOAT"));
+    }
+    loader.close();
+  }
+
+  @Test
+  public void testExtraField() {
+    String json =
+        "{ a: { \"$numberLong\": 10 } }\n" +
+        "{ a: { \"$numberLong\": 20, bogus: 30 } }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.open(json);
+    try {
+      loader.next();
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains(LONG_HINT));
+    }
+    loader.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedWithSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedWithSchema.java
new file mode 100644
index 0000000..7560192
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestExtendedWithSchema.java
@@ -0,0 +1,449 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import static org.apache.drill.test.rowSet.RowSetUtilities.dec;
+import static org.apache.drill.test.rowSet.RowSetUtilities.intArray;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.TimeZone;
+
+import org.apache.drill.categories.JsonTest;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test extended types with a schema. We see the schema slightly before
+ * we see the full extended type syntax, especially if a field is null
+ * or "relaxed" in the first row.
+ * <p>
+ * The provided type must match the extended type: an extended type of
+ * {@code numberLong} must have a provided type of {@code BIGINT}, for example.
+ */
+@Category(JsonTest.class)
+public class TestExtendedWithSchema extends BaseJsonLoaderTest {
+
+  @Test
+  public void testInt() {
+    String json =
+        "{ a: { \"$numberInt\": 10 },\n" +
+        "  b: null,\n" +
+        "  c: { \"$numberInt\": \"30\" },\n" +
+        "  d: 40,\n" +
+        "  e: \"50\" }\n" +
+        "{ a: 110,\n" +
+        "  b: 120,\n" +
+        "  c: null,\n" +
+        "  d: { \"$numberInt\": 140 },\n" +
+        "  e: { \"$numberInt\": 150 } }\n" +
+        "{ a: { \"$numberInt\": \"210\" },\n" +
+        "  b: { \"$numberInt\": 220 },\n" +
+        "  c: 230,\n" +
+        "  d: null,\n" +
+        "  e: { \"$numberInt\": \"250\" } }";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .addNullable("b", MinorType.INT)
+        .addNullable("c", MinorType.INT)
+        .addNullable("d", MinorType.INT)
+        .add("e", MinorType.INT)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow( 10, null,   30,   40,  50)
+        .addRow(110,  120, null,  140, 150)
+        .addRow(210,  220,  230, null, 250)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  /**
+   * Test only the int array. All other arrays use the same
+   * code path.
+   */
+  @Test
+  public void testIntArray() {
+    String json =
+        "{ a: [ { \"$numberInt\": 10 }, 20, { \"$numberInt\": \"30\" } ],\n" +
+        "  b: null,\n" +
+        "  c: [],\n" +
+        "  d: [ { \"$numberInt\": 40 }, \"50\", null ] }\n" +
+        "{ a: [],\n" +
+        "  b: [ { \"$numberInt\": 140 }, \"150\", null ],\n" +
+        "  c: [ { \"$numberInt\": 110 }, 120, { \"$numberInt\": \"130\" } ],\n" +
+        "  d: null }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .addArray("b", MinorType.INT)
+        .addArray("c", MinorType.INT)
+        .addArray("d", MinorType.INT)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(intArray(10, 20, 30),
+                intArray(),
+                intArray(),
+                intArray(40, 50, 0))
+        .addRow(intArray(),
+                intArray(140, 150, 0),
+                intArray(110, 120, 130),
+                intArray())
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testLong() {
+    String json =
+        "{ a: { \"$numberLong\": 10 },\n" +
+        "  b: null,\n" +
+        "  c: { \"$numberLong\": \"30\" },\n" +
+        "  d: 40,\n" +
+        "  e: \"50\" }\n" +
+        "{ a: 110,\n" +
+        "  b: 120,\n" +
+        "  c: null,\n" +
+        "  d: { \"$numberLong\": 140 },\n" +
+        "  e: { \"$numberLong\": 150 } }\n" +
+        "{ a: { \"$numberLong\": \"210\" },\n" +
+        "  b: { \"$numberLong\": 220 },\n" +
+        "  c: 230,\n" +
+        "  d: null,\n" +
+        "  e: { \"$numberLong\": \"250\" } }";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.BIGINT)
+        .addNullable("b", MinorType.BIGINT)
+        .addNullable("c", MinorType.BIGINT)
+        .addNullable("d", MinorType.BIGINT)
+        .add("e", MinorType.BIGINT)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow( 10L,  null,   30L,   40L,  50L)
+        .addRow(110L,  120L,  null,  140L, 150L)
+        .addRow(210L,  220L,  230L,  null, 250L)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDouble() {
+    String json =
+        "{ a: { \"$numberDouble\": 10 },\n" +
+        "  b: null,\n" +
+        "  c: { \"$numberDouble\": \"30\" },\n" +
+        "  d: 40,\n" +
+        "  e: \"50\" }\n" +
+        "{ a: 110,\n" +
+        "  b: 120,\n" +
+        "  c: null,\n" +
+        "  d: { \"$numberDouble\": 140 },\n" +
+        "  e: { \"$numberDouble\": 150 } }\n" +
+        "{ a: { \"$numberDouble\": \"210\" },\n" +
+        "  b: { \"$numberDouble\": 220 },\n" +
+        "  c: 230,\n" +
+        "  d: null,\n" +
+        "  e: { \"$numberDouble\": \"250\" } }";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.FLOAT8)
+        .addNullable("b", MinorType.FLOAT8)
+        .addNullable("c", MinorType.FLOAT8)
+        .addNullable("d", MinorType.FLOAT8)
+        .add("e", MinorType.FLOAT8)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow( 10D,  null,   30D,   40D,  50D)
+        .addRow(110D,  120D,  null,  140D, 150D)
+        .addRow(210D,  220D,  230D,  null, 250D)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDecimal() {
+    String json =
+        "{ a: { \"$numberDecimal\": 10 },\n" +
+        "  b: null,\n" +
+        "  c: { \"$numberDecimal\": \"30\" },\n" +
+        "  d: 40,\n" +
+        "  e: \"50\" }\n" +
+        "{ a: 110,\n" +
+        "  b: 120,\n" +
+        "  c: null,\n" +
+        "  d: { \"$numberDecimal\": 140 },\n" +
+        "  e: { \"$numberDecimal\": 150 } }\n" +
+        "{ a: { \"$numberDecimal\": \"210\" },\n" +
+        "  b: { \"$numberDecimal\": 220 },\n" +
+        "  c: 230,\n" +
+        "  d: null,\n" +
+        "  e: { \"$numberDecimal\": \"250\" } }";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.VARDECIMAL, 4, 0)
+        .addNullable("b", MinorType.VARDECIMAL, 5, 1)
+        .addNullable("c", MinorType.VARDECIMAL, 6, 2)
+        .addNullable("d", MinorType.VARDECIMAL, 7, 3)
+        .add("e", MinorType.VARDECIMAL, 8, 4)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(dec("10"),         null,   dec("30"),   dec("40"),  dec("50"))
+        .addRow(dec("110"),  dec("120"),        null,  dec("140"), dec("150"))
+        .addRow(dec("210"),  dec("220"),  dec("230"),        null, dec("250"))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDate() {
+    LocalDateTime local = LocalDateTime.of(2020, 4, 21, 11, 22, 33);
+    Instant instant = local.atZone(ZoneId.systemDefault()).toInstant();
+    long ts = instant.toEpochMilli();
+    String utc = DateTimeFormatter.ISO_INSTANT.format(instant);
+    long localTs = ts + TimeZone.getDefault().getOffset(ts);
+    String json =
+        "{ a: { \"$date\": \"" + utc + "\" },\n" +
+        "  b: null,\n" +
+        "  c: { \"$date\": " + ts + " },\n" +
+        "  d: " + ts + ",\n" +
+        "  e: \"" + utc + "\",\n" +
+        "  f: { \"$date\": { \"$numberLong\": " + ts + " } } }\n" +
+        "{ a: \"" + utc + "\",\n" +
+        "  b: " + ts + ",\n" +
+        "  c: null,\n" +
+        "  d: { \"$date\": \"" + utc + "\" },\n" +
+        "  e: { \"$date\": { \"$numberLong\": " + ts + " } },\n" +
+        "  f: " + ts + " }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.TIMESTAMP)
+        .addNullable("b", MinorType.TIMESTAMP)
+        .addNullable("c", MinorType.TIMESTAMP)
+        .addNullable("d", MinorType.TIMESTAMP)
+        .add("e", MinorType.TIMESTAMP)
+        .add("f", MinorType.TIMESTAMP)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(localTs,    null, localTs, localTs, localTs, localTs)
+        .addRow(localTs, localTs,    null, localTs, localTs, localTs)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: date only
+  @Test
+  public void testDateDay() {
+    String json =
+        "{ a: { \"$dateDay\": \"2020-04-21\" },\n" +
+        "  b: null,\n" +
+        "  c: \"2020-04-21\" }\n" +
+        "{ a: \"2020-04-21\",\n" +
+        "  b: { \"$dateDay\": \"2020-04-21\" },\n" +
+        "  c: null }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.DATE)
+        .addNullable("b", MinorType.DATE)
+        .addNullable("c", MinorType.DATE)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    org.joda.time.LocalDate date = new org.joda.time.LocalDate(2020, 04, 21);
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(date, null, date)
+        .addRow(date, date, null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time only
+  @Test
+  public void testTime() {
+    String json =
+        "{ a: { \"$time\": \"11:22:33\" },\n" +
+        "  b: null,\n" +
+        "  c: \"11:22:33\" }\n" +
+        "{ a: \"11:22:33\",\n" +
+        "  b: { \"$time\": \"11:22:33\" },\n" +
+        "  c: null }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.TIME)
+        .addNullable("b", MinorType.TIME)
+        .addNullable("c", MinorType.TIME)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    org.joda.time.LocalTime time  = new org.joda.time.LocalTime(11, 22, 33);
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(time, null, time)
+        .addRow(time, time, null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  // Drill extension: time interval
+  @Test
+  public void testInterval() {
+    String json =
+        "{ a: { \"$interval\": \"P1Y2M3DT4H5M6S\"  },\n" +
+        "  b: null,\n" +
+        "  c: \"P1Y2M3D\" }\n" +
+        "{ a: \"P1Y2M3D\",\n" +
+        "  b: { \"$interval\": \"P1Y2M3DT4H5M6S\"  },\n" +
+        "  c: null }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .add("a", MinorType.INTERVAL)
+        .addNullable("b", MinorType.INTERVAL)
+        .addNullable("c", MinorType.INTERVAL)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    org.joda.time.Period full = org.joda.time.Period.years(1).withMonths(2).withDays(3).withHours(4).withMinutes(5).withSeconds(6);
+    org.joda.time.Period ymd = org.joda.time.Period.years(1).withMonths(2).withDays(3);
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(full, null,  ymd)
+        .addRow( ymd, full, null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testBinary() {
+    String json =
+        "{ a: { \"$binary\": { base64: \"ZHJpbGw=\", subType: \"0\" } },\n" +
+        "  b: { \"$binary\": { subType: \"0\", base64: \"ZHJpbGw=\" } },\n" +
+        "  c: { \"$binary\": { base64: \"ZHJpbGw=\" } },\n" +
+        "  d: null,\n" +
+        "  e: { \"$binary\": \"ZHJpbGw=\", \"$type\": 1 },\n" +
+        "  f: { \"$type\": 1, \"$binary\": \"ZHJpbGw=\" },\n" +
+        "  g: { \"$binary\": \"ZHJpbGw=\" },\n" +
+        "  h: \"ZHJpbGw=\" }\n" +
+        "{ a: null,\n" +
+        "  b: { \"$binary\": \"ZHJpbGw=\", \"$type\": 1 },\n" +
+        "  c: { \"$type\": 1, \"$binary\": \"ZHJpbGw=\" },\n" +
+        "  d: { \"$binary\": \"ZHJpbGw=\" },\n" +
+        "  e: \"ZHJpbGw=\",\n" +
+        "  f: { \"$binary\": { base64: \"ZHJpbGw=\", subType: \"0\" } },\n" +
+        "  g: { \"$binary\": { subType: \"0\", base64: \"ZHJpbGw=\" } },\n" +
+        "  h: { \"$binary\": { base64: \"ZHJpbGw=\" } } }\n";
+    TupleMetadata providedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.VARBINARY)
+        .add("b", MinorType.VARBINARY)
+        .add("c", MinorType.VARBINARY)
+        .addNullable("d", MinorType.VARBINARY)
+        .add("e", MinorType.VARBINARY)
+        .add("f", MinorType.VARBINARY)
+        .add("g", MinorType.VARBINARY)
+        .add("h", MinorType.VARBINARY)
+        .build();
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.jsonOptions.enableExtendedTypes = true;
+    loader.builder.providedSchema(providedSchema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    byte[] bytes = "Drill".getBytes();
+    RowSet expected = fixture.rowSetBuilder(providedSchema)
+        .addRow(bytes, bytes, bytes,  null, bytes, bytes, bytes, bytes)
+        .addRow( null, bytes, bytes, bytes, bytes, bytes, bytes, bytes)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestMessageParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestMessageParser.java
new file mode 100644
index 0000000..54860e9
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestMessageParser.java
@@ -0,0 +1,248 @@
+/*
+ * 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.drill.exec.store.easy.json.loader;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.drill.categories.JsonTest;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.exec.store.easy.json.parser.MessageParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+@Category(JsonTest.class)
+public class TestMessageParser extends BaseJsonLoaderTest {
+
+  /**
+   * Example message parser. A real parser would provide much better
+   * error messages for badly-formed JSON or error codes.
+   */
+  private static class MessageParserFixture implements MessageParser {
+
+    @Override
+    public boolean parsePrefix(TokenIterator tokenizer) {
+      assertEquals(JsonToken.START_OBJECT, tokenizer.requireNext());
+      assertEquals(JsonToken.FIELD_NAME, tokenizer.requireNext());
+      assertEquals(JsonToken.VALUE_STRING, tokenizer.requireNext());
+      if (!"ok".equals(tokenizer.stringValue())) {
+        return false;
+      }
+      assertEquals(JsonToken.FIELD_NAME, tokenizer.requireNext());
+      JsonToken token = tokenizer.requireNext();
+      assertEquals(JsonToken.START_ARRAY, token);
+      tokenizer.unget(token);
+      return true;
+    }
+
+    @Override
+    public void parseSuffix(TokenIterator tokenizer) {
+      assertEquals(JsonToken.END_OBJECT, tokenizer.requireNext());
+    }
+  }
+
+  /**
+   * Test the ability to wrap the data objects with a custom message
+   * structure, typical of a REST call.
+   */
+  @Test
+  public void testMessageParser() {
+    final String json =
+        "{ status: \"ok\", data: [{a: 0}, {a: 100}, {a: null}]}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.messageParser(new MessageParserFixture());
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(0)
+        .addRow(100)
+        .addSingleCol(null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  /**
+   * Test the ability to cancel the data load if a message header
+   * indicates that there is no data.
+   */
+  @Test
+  public void testMessageParserEOF() {
+    final String json =
+        "{ status: \"fail\", data: [{a: 0}, {a: 100}, {a: null}]}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.messageParser(new MessageParserFixture());
+    loader.open(json);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  /**
+   * Test the case where the returned message has a single data
+   * object: <code>{ data: { ... } }</code>.
+   */
+  @Test
+  public void testDataPathObject() {
+    final String json =
+        "{ status: \"ok\", data: [{a: 0}, {a: 100}, {a: null}]}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.dataPath("data");
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(0)
+        .addRow(100)
+        .addSingleCol(null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  /**
+   * Test the case where the returned message has an array
+   * objects: <code>{ data: [ { ... }, { ... } ... ] }</code>.
+   */
+  @Test
+  public void testDataPathArray() {
+    final String json =
+        "{ status: \"ok\", data: [{a: 0}, {a: 100}, {a: null}]}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.dataPath("data");
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(0)
+        .addRow(100)
+        .addSingleCol(null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testComplexDataPath() {
+    final String json =
+        "{ status: {result : \"ok\", runtime: 123},\n" +
+        "  response: { rowCount: 1,\n" +
+        "    data: [{a: 0}, {a: 100}, {a: null}]},\n" +
+        "  footer: \"some stuff\"}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.dataPath("response/data");
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addNullable("a", MinorType.BIGINT)
+         .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addRow(0)
+        .addRow(100)
+        .addSingleCol(null)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  /**
+   * Test the case where the returned message has a null in place
+   * of the data: <code>{ data: null }</code>. This is harmlessly
+   * treated as no data and is needed for the case where the
+   * message normally returns a single object.
+   */
+  @Test
+  public void testDataPathNull() {
+    final String json =
+        "{ status: \"fail\", data: null}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.messageParser(new MessageParserFixture());
+    loader.open(json);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDataPathMissing() {
+    final String json =
+        "{ status: \"fail\"}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.messageParser(new MessageParserFixture());
+    loader.open(json);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testDataPathErrorRoot() {
+    final String json = "\"Bogus!\"";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.dataPath("data");
+    try {
+      loader.open(json);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("Syntax error"));
+      assertTrue(e.getCause() instanceof MessageParser.MessageContextException);
+    }
+  }
+
+  @Test
+  public void testDataPathErrorLeaf() {
+    final String json =
+        "{ status: \"bogus\", data: \"must be array or object\" }";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.dataPath("data");
+    try {
+      loader.open(json);
+      fail();
+    } catch (UserException e) {
+      assertTrue(e.getMessage().contains("Syntax error"));
+      assertTrue(e.getCause() instanceof MessageParser.MessageContextException);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestObjects.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestObjects.java
index 7c6475d..e085132 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestObjects.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestObjects.java
@@ -17,14 +17,14 @@
  */
 package org.apache.drill.exec.store.easy.json.loader;
 
-import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
 import static org.apache.drill.test.rowSet.RowSetUtilities.mapArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.mapValue;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.categories.JsonTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.RowSet;
@@ -35,7 +35,7 @@ import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Category(RowSetTests.class)
+@Category(JsonTest.class)
 public class TestObjects extends BaseJsonLoaderTest {
 
   @Test
@@ -98,7 +98,7 @@ public class TestObjects extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -136,7 +136,7 @@ public class TestObjects extends BaseJsonLoaderTest {
     m.setProperty(JsonLoader.JSON_MODE, JsonLoader.JSON_LITERAL_MODE);
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -225,7 +225,7 @@ public class TestObjects extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -270,7 +270,7 @@ public class TestObjects extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -294,7 +294,7 @@ public class TestObjects extends BaseJsonLoaderTest {
       loader.next();
       fail();
     } catch (UserException e) {
-      assertTrue(e.getMessage().contains("integer"));
+      assertTrue(e.getMessage().contains("JSON object expected"));
     }
     loader.close();
   }
@@ -309,7 +309,7 @@ public class TestObjects extends BaseJsonLoaderTest {
       loader.next();
       fail();
     } catch (UserException e) {
-      assertTrue(e.getMessage().contains("integer[]"));
+      assertTrue(e.getMessage().contains("JSON object expected"));
     }
     loader.close();
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestRepeatedList.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestRepeatedList.java
index 76d48f3..997655e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestRepeatedList.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestRepeatedList.java
@@ -25,7 +25,7 @@ import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
-import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.categories.JsonTest;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.RowSet;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
@@ -37,7 +37,7 @@ import org.junit.experimental.categories.Category;
 /**
  * Tests repeated lists to form a 2D or 3D array of various data types.
  */
-@Category(RowSetTests.class)
+@Category(JsonTest.class)
 public class TestRepeatedList extends BaseJsonLoaderTest {
 
   @Test
@@ -102,7 +102,7 @@ public class TestRepeatedList extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -138,7 +138,7 @@ public class TestRepeatedList extends BaseJsonLoaderTest {
         .build();
     RowSet expected = fixture.rowSetBuilder(expectedSchema)
         .addSingleCol(null)
-        .addSingleCol(singleObjArray(strArray("")))
+        .addSingleCol(singleObjArray(strArray("null")))
         .addSingleCol(objArray(
             strArray("1", "2"), strArray("3", "4", "5")))
         .build();
@@ -217,7 +217,7 @@ public class TestRepeatedList extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -251,7 +251,7 @@ public class TestRepeatedList extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -425,4 +425,30 @@ public class TestRepeatedList extends BaseJsonLoaderTest {
     assertNull(loader.next());
     loader.close();
   }
+
+  @Test
+  public void testForced2DArrayResolve() {
+    String json =
+        "{a: null} {a: []} {a: [[]]}\n" +
+        "{a: [[\"foo\"], [20]]}";
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .addRepeatedList("a")
+          .addArray(MinorType.VARCHAR)
+          .resumeSchema()
+        .build();
+    RowSet expected = fixture.rowSetBuilder(expectedSchema)
+        .addSingleCol(objArray())
+        .addSingleCol(objArray())
+        .addSingleCol(singleObjArray(strArray()))
+        .addSingleCol(objArray(strArray("\"foo\""), strArray("20")))
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalarArrays.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalarArrays.java
index ea6b0e9..5b4c3a6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalarArrays.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalarArrays.java
@@ -18,15 +18,15 @@
 package org.apache.drill.exec.store.easy.json.loader;
 
 import static org.apache.drill.test.rowSet.RowSetUtilities.boolArray;
-import static org.apache.drill.test.rowSet.RowSetUtilities.longArray;
 import static org.apache.drill.test.rowSet.RowSetUtilities.doubleArray;
+import static org.apache.drill.test.rowSet.RowSetUtilities.longArray;
 import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.drill.categories.RowSetTests;
+import org.apache.drill.categories.JsonTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.rowSet.RowSet;
@@ -46,7 +46,7 @@ import org.junit.experimental.categories.Category;
  * Verifies that null array elements are converted to a default
  * value for the type (false, 0 or empty string.)
  */
-@Category(RowSetTests.class)
+@Category(JsonTest.class)
 public class TestScalarArrays extends BaseJsonLoaderTest {
 
   @Test
@@ -110,7 +110,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
         .addArray("a", MinorType.BIT)
         .build();
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -217,7 +217,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -295,7 +295,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -373,7 +373,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -403,7 +403,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
       loader.next();
       fail();
     } catch (UserException e) {
-      assertTrue(e.getMessage().contains("integer[][]"));
+      assertTrue(e.getMessage().contains("Structure value found where scalar expected"));
     } finally {
       loader.close();
     }
@@ -422,7 +422,7 @@ public class TestScalarArrays extends BaseJsonLoaderTest {
       loader.next();
       fail();
     } catch (UserException e) {
-      assertTrue(e.getMessage().contains("object"));
+      assertTrue(e.getMessage().contains("JSON array expected"));
     } finally {
       loader.close();
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalars.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalars.java
index 09ba62f..7f7a097 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalars.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/json/loader/TestScalars.java
@@ -17,17 +17,19 @@
  */
 package org.apache.drill.exec.store.easy.json.loader;
 
+import static org.apache.drill.test.rowSet.RowSetUtilities.dec;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.drill.categories.RowSetTests;
+import java.time.Duration;
+import java.time.LocalDateTime;
+
+import org.apache.drill.categories.JsonTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.resultSet.project.Projections;
 import org.apache.drill.exec.physical.rowSet.RowSet;
-import org.apache.drill.exec.physical.rowSet.RowSetTestUtils;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.rowSet.RowSetUtilities;
@@ -46,7 +48,7 @@ import org.junit.experimental.categories.Category;
  * to a few messy rows a billion rows in, or due to the order that the scanners
  * see the data.
  */
-@Category(RowSetTests.class)
+@Category(JsonTest.class)
 public class TestScalars extends BaseJsonLoaderTest {
 
   /**
@@ -127,7 +129,7 @@ public class TestScalars extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -255,7 +257,7 @@ public class TestScalars extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -390,7 +392,7 @@ public class TestScalars extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.jsonOptions.allowNanInf = true;
     loader.open(json);
     RowSet results = loader.next();
@@ -504,7 +506,7 @@ public class TestScalars extends BaseJsonLoaderTest {
         .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.providedSchema = schema;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
@@ -527,26 +529,159 @@ public class TestScalars extends BaseJsonLoaderTest {
   }
 
   @Test
-  public void testProjection() {
+  public void testProvidedSchemaNumbers() {
     String json =
-        "{a: 10, b: true}\n" +
-        "{a: 20, b: [\"what?\"]}\n" +
-        "{a: 30, b: {c: \"oh, my!\"}}";
+
+        // null is ambiguous
+        "{s: null, i: null, bi: null, f4: null, f8: null, d: null}\n" +
+        // Strings are also
+        "{s: \"10\", i: \"10\", bi: \"10\", f4: \"10\", f8: \"10\", d: \"10\"}\n" +
+        "{             f4: \"10.5\", f8: \"10.5\", d: \"10.5\"}\n" +
+        "{             f4: \"-1e5\", f8: \"-1e5\", d: \"-1e5\"}\n" +
+
+        // Float-only values
+        "{             f4: \"NaN\", f8: \"NaN\"}\n" +
+        "{             f4: \"Infinity\", f8: \"Infinity\"}\n" +
+        "{             f4: \"-Infinity\", f8: \"-Infinity\"}\n" +
+
+        // Large decimal
+        "{d: \"123456789012345678901234.5678\" }\n" +
+
+        // Ambiguous numbers
+        "{s: 10, i: 10, bi: 10, f4: 10, f8: 10, d: 10}\n" +
+        "{             f4: 10.5, f8: 10.5, d: 10.5}\n" +
+        "{             f4: -1e5, f8: -1e5, d: -1e5}\n" +
+
+        // Float-only values
+        "{             f4: NaN, f8: NaN}\n" +
+        "{             f4: Infinity, f8: Infinity}\n" +
+        "{             f4: -Infinity, f8: -Infinity}\n" +
+
+        // Large decimal
+        "{d: 123456789012345678901234.5678 }\n";
+    TupleMetadata schema = new SchemaBuilder()
+        .addNullable("s", MinorType.SMALLINT)
+        .addNullable("i", MinorType.INT)
+        .addNullable("bi", MinorType.BIGINT)
+        .addNullable("f4", MinorType.FLOAT4)
+        .addNullable("f8", MinorType.FLOAT8)
+        .addNullable("d", MinorType.VARDECIMAL, 38, 4)
+        .build();
 
     JsonLoaderFixture loader = new JsonLoaderFixture();
-    loader.rsLoaderOptions.projection(
-        Projections.parse(RowSetTestUtils.projectList("a")));
+    loader.jsonOptions.allowNanInf = true;
+    loader.builder.providedSchema(schema);
     loader.open(json);
     RowSet results = loader.next();
     assertNotNull(results);
 
-    TupleMetadata expectedSchema = new SchemaBuilder()
-        .addNullable("a", MinorType.BIGINT)
+    RowSet expected = fixture.rowSetBuilder(schema)
+        //      s     i     bi    f4    f8    d
+        .addRow(null, null, null, null, null, null)
+        .addRow(10,   10,   10,   10,   10,   dec("10"))
+        .addRow(null, null, null, 10.5, 10.5D, dec("10.5"))
+        .addRow(null, null, null, -1e5,  -1e5D, dec("-1e5"))
+        .addRow(null, null, null, Float.NaN, Double.NaN, null)
+        .addRow(null, null, null, Float.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, null)
+        .addRow(null, null, null, Float.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY, null)
+        .addRow(null, null, null, null, null, dec("123456789012345678901234.5678"))
+        .addRow(10,   10,   10,   10,   10,   dec("10"))
+        .addRow(null, null, null, 10.5, 10.5D, dec("10.5"))
+        .addRow(null, null, null, -1e5,  -1e5D, dec("-1e5"))
+        .addRow(null, null, null, Float.NaN, Double.NaN, null)
+        .addRow(null, null, null, Float.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, null)
+        .addRow(null, null, null, Float.NEGATIVE_INFINITY, Double.NEGATIVE_INFINITY, null)
+        .addRow(null, null, null, null, null, dec("123456789012345678901234.5678"))
         .build();
-    RowSet expected = fixture.rowSetBuilder(expectedSchema)
-        .addRow(10)
-        .addRow(20)
-        .addRow(30)
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testProvidedSchemaWithDates() {
+    LocalDateTime local = LocalDateTime.of(2020, 4, 21, 11, 22, 33, 456_000_000);
+    LocalDateTime localEpoch = LocalDateTime.of(1970, 1, 1, 0, 0, 0);
+    long localTs = Duration.between(localEpoch, local).toMillis();
+    LocalDateTime localDate = LocalDateTime.of(2020, 4, 21, 0, 0, 0);
+    long localDateTs = Duration.between(localEpoch, localDate).toMillis();
+    int localTimeTs = (int) (localTs - localDateTs);
+    String json =
+        "{ts: null, d: null, t: null}\n" +
+        "{ts: \"2020-04-21T11:22:33.456\", d: \"2020-04-21\", t: \"11:22:33.456\"}\n" +
+        "{ts: " + localTs + ", d: " + localDateTs + ", t: " + localTimeTs + "}\n";
+    TupleMetadata schema = new SchemaBuilder()
+        .addNullable("ts", MinorType.TIMESTAMP)
+        .addNullable("d", MinorType.DATE)
+        .addNullable("t", MinorType.TIME)
+        .build();
+
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.providedSchema(schema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    RowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(null, null, null)
+        .addRow(localTs, localDateTs, localTimeTs)
+        .addRow(localTs, localDateTs, localTimeTs)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
+
+  @Test
+  public void testProvidedSchemaWithIntervals() {
+    String json =
+        "{i: null, iy: null, id: null}\n" +
+        "{i: \"P1Y2M3DT4H5M6S\", iy: \"P1Y2M\", id: \"P3DT4H5M6S\"}";
+    TupleMetadata schema = new SchemaBuilder()
+        .addNullable("i", MinorType.INTERVAL)
+        .addNullable("iy", MinorType.INTERVALYEAR)
+        .addNullable("id", MinorType.INTERVALDAY)
+        .build();
+
+    JsonLoaderFixture loader = new JsonLoaderFixture();
+    loader.builder.providedSchema(schema);
+    loader.open(json);
+    RowSet results = loader.next();
+    assertNotNull(results);
+
+    org.joda.time.Period full = org.joda.time.Period.years(1).withMonths(2)
+        .withDays(3).withHours(4).withMinutes(5).withSeconds(6);
+    org.joda.time.Period ym = org.joda.time.Period.years(1).withMonths(2);
+    org.joda.time.Period dhms = org.joda.time.Period.days(3).withHours(4)
+        .withMinutes(5).withSeconds(6);
+    RowSet expected = fixture.rowSetBuilder(schema)
+        .addRow(null, null, null)
+        .addRow(full, ym, dhms)
+        .build();
+    RowSetUtilities.verify(expected, results);
+    assertNull(loader.next());
+    loader.close();
+  }
... 2095 lines suppressed ...