You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ag...@apache.org on 2020/02/21 12:03:36 UTC

[drill] 07/10: DRILL-7572: JSON structure parser

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

agozhiy pushed a commit to branch MERGE-200221-00
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 8e6ba4811cccc8f642532e9bdff7581d2d6aba42
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Thu Feb 6 23:25:11 2020 -0800

    DRILL-7572: JSON structure parser
    
    Provides a structure-driven parser for JSON. Provides an intermediate
    level beteen the Jackson JSON parser and the "shims" that write to
    Drill value vectors via the EVF.
    
    close #1971
---
 .../easy/json/parser/AbstractElementParser.java    |  39 ++
 .../exec/store/easy/json/parser/ArrayListener.java | 151 ++++++++
 .../exec/store/easy/json/parser/ArrayParser.java   |  68 ++++
 .../store/easy/json/parser/DummyValueParser.java   |  80 +++++
 .../exec/store/easy/json/parser/ElementParser.java |  39 ++
 .../exec/store/easy/json/parser/ErrorFactory.java  |  76 ++++
 .../easy/json/parser/JsonStructureOptions.java     |  54 +++
 .../easy/json/parser/JsonStructureParser.java      | 218 +++++++++++
 .../exec/store/easy/json/parser/JsonType.java      |  25 ++
 .../store/easy/json/parser/ObjectListener.java     | 131 +++++++
 .../exec/store/easy/json/parser/ObjectParser.java  | 167 +++++++++
 .../exec/store/easy/json/parser/RootParser.java    | 121 +++++++
 .../exec/store/easy/json/parser/TokenIterator.java | 149 ++++++++
 .../exec/store/easy/json/parser/ValueFactory.java  | 222 ++++++++++++
 .../exec/store/easy/json/parser/ValueListener.java | 152 ++++++++
 .../exec/store/easy/json/parser/ValueParser.java   | 174 +++++++++
 .../exec/store/json/parser/BaseTestJsonParser.java | 345 ++++++++++++++++++
 .../store/json/parser/TestJsonParserArrays.java    | 397 +++++++++++++++++++++
 .../store/json/parser/TestJsonParserBasics.java    | 279 +++++++++++++++
 .../store/json/parser/TestJsonParserErrors.java    |  94 +++++
 .../store/json/parser/TestJsonParserObjects.java   | 158 ++++++++
 21 files changed, 3139 insertions(+)

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
new file mode 100644
index 0000000..ed811d5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/AbstractElementParser.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+
+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; }
+
+  protected ErrorFactory errorFactory() {
+    return structParser.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
new file mode 100644
index 0000000..c5d588d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayListener.java
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+/**
+ * Represents one level within an array. The first time the parser sees
+ * the array element, it will call one of the "Element" methods with the
+ * look-ahead values visible to the parser. Since JSON is flexible, later
+ * data shapes may not necessarily follow the first shape. The implementation
+ * must handle this or throw an error if not supported.
+ * <p>
+ * When creating a multi-dimensional array, each array level is built one
+ * by one. each will receive the same type information (decreased by one
+ * array level.)
+ * <p>
+ * Then, while parsing, the parser calls events on the start and end of the
+ * array, as well as on each element.
+ * <p>
+ * The array listener is an attribute of a value listener, represent the
+ * "arrayness" of that value, if the value allows an array.
+ *
+ * <h4>Elements</h4>
+ *
+ * The array listener has a child listener that represents each element
+ * in the array. The structure parser asks this listener to create that
+ * child on the first element seen for the array. The structure parser
+ * provides "look-ahead" type information for that element, when available.
+ * <p>
+ * Three JSON-specific cases warrant attention:
+ * <ol>
+ * <li>The first occurrence of the array is empty: {@code [ ]}. In this case,
+ * the structure parser will ask for an element child by providing the
+ * {@link JsonType#EMPTY} type, which is not very useful, but is all that
+ * the parser knows. The listener is responsible for implementing some kind of
+ * "deferred type" logic to wait and see what kind of element appears
+ * later..</li>
+ * <li>The first occurrence of the array has, as its first element, a
+ * {@code null} value. The structure parser will ask this listener to create
+ * an array child for the {@code null} value, but the listener has no type
+ * information. Again, the listener is responsible for type-deferal.</li>
+ * <li>A generalized form of the above is that the structure parser only
+ * knows what it sees on the first element when it asks for an element
+ * child. In a well-formed file, that first token will predict the type
+ * of all future tokens. But, JSON allows anything. The first element
+ * might be {@code null}, an empty array, or a String. The second element
+ * could be anything else (a number or an object). The listener, as always
+ * is responsible for deciding how to handle type changes.</li>
+ * </ol>
+ *
+ * <h4>Multi-Dimensional Arrays</h4>
+ *
+ * A multi-dimensional array is one of the form {@code [ [ ... }, that is,
+ * the parser returns multiple levels of array start tokens. In this case,
+ * listeners are structured as:
+ * <ul>
+ * <li>{@code ObjectListener} for the enclosing object which has a</li>
+ * <li>{@code FieldListener} for the array value which has a</li>
+ * <li>{@code ArrayListener} for the array, which has a</li>
+ * <li>{@code ValueListener} for the elements. If the array is 1D,
+ * the nesting stops here. But if it is 2+D, then the value has a</li>
+ * <li>{@code ArrayListener} for the inner array, which has a</li>
+ * <li>{@code ValueListener} for the elements. And so on recursively
+ * for as many levels as needed or the array.</li>
+ * </ul>
+ */
+public interface ArrayListener {
+
+  /**
+   * Called at the start of a set of values for an array. That is, called
+   * when the structure parser accepts the {@code [} token.
+   */
+  void onStart();
+
+  /**
+   * Called for each element of the array. The array element is represented
+   * by its own listener which receives the value of the element (if
+   * scalar) or element events (if structured.)
+   */
+  void onElement();
+
+  /**
+   * Called at the end of a set of values for an array. That is, called
+   * when the structure parser accepts the {@code ]} token.
+   */
+  void onEnd();
+
+  /**
+   * The first element seen is a scalar, {@code null} or empty. That is,
+   * {@code [ <scalar>}, {@code [ null} or {@code [ ]}.
+   *
+   * @param type the JSON type of the object as given by the token
+   * which the Jackson parser returned for the value. The type can
+   * be {@code null}, which means that the parser does not know what
+   * actual type might occur later
+   * @return a value listener for the scalar type, or if {@code null},
+   * perhaps waiting for more information to commit to a type
+   */
+  ValueListener scalarElement(JsonType type);
+
+  /**
+   * The first element an array or scalars (or {@code null}.That is,
+   * {@code [ [+ <scalar>}.
+   *
+   * @param arrayDims the number of dimensions observed during the
+   * first-element parse, not including the surrounding array
+   * itself. As in all cases, there is no guarantee that
+   * that this number will remain valid later, and may be wrong if the
+   * first-seen element was empty: {@code []}.
+   * @return a listener for the value of the top-level element (which
+   * the listener can assume will turn out to be an array.)
+   */
+  ValueListener arrayElement(int arrayDims, JsonType type);
+
+  /**
+   * The first element seen for an array is an object. That is,
+   * <code>[ {</code>.
+   *
+   * @return a listener for the value of the top-level element (which
+   * the listener can assume will turn out to be an object.)
+   */
+  ValueListener objectElement();
+
+  /**
+   * The first element seen is an object array.That is,
+   * <code>[ [* {</code>.
+   *
+   * @param arrayDims the number of dimensions observed during the
+   * first-element parse, not including the surrounding array
+   * itself. As in all cases, there is no guarantee that
+   * that this number will remain valid later, and may be wrong if the
+   * first-seen element was empty: {@code []}.
+   * @return a listener for the value of the top-level element (which
+   * the listener can assume will turn out to be an array.)
+   */
+  ValueListener objectArrayElement(int arrayDims);
+}
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
new file mode 100644
index 0000000..2d4431a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ArrayParser.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.parser;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parses a JSON array, which consists of a list of <i>elements</i>,
+ * represented by a {@code ValueListener}. There is a single listener
+ * for all the elements, which are presumed to be of the same type.
+ * <p>
+ * This parser <i>does not</i> attempt to parse an array as a poor-man's
+ * tuple: {@code [ 101, "fred", 23.45 ]}. The listener could handle this
+ * case. But, if we need to handle such a case, it would be better to
+ * create a new parser for that case, with an element listener per
+ * element as is done for objects.
+ */
+public class ArrayParser extends AbstractElementParser {
+
+  private final ArrayListener arrayListener;
+  private final ValueParser elementParser;
+
+  public ArrayParser(ValueParser parent, ArrayListener arrayListener, ValueListener elementListener) {
+    super(parent);
+    this.arrayListener = arrayListener;
+    this.elementParser = new ValueParser(this, "[]", elementListener);
+  }
+
+  public ValueParser elementParser() { return elementParser; }
+
+  /**
+   * Parses <code>[ ^ ((value)(, (value)* )? ]</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    arrayListener.onStart();
+    top: for (;;) {
+      // Position: [ (value, )* ^ ?
+     JsonToken token = tokenizer.requireNext();
+      switch (token) {
+        case END_ARRAY:
+          break top;
+
+        default:
+          tokenizer.unget(token);
+          arrayListener.onElement();
+          elementParser.parse(tokenizer);
+          break;
+      }
+    }
+    arrayListener.onEnd();
+  }
+}
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
new file mode 100644
index 0000000..86b345c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/DummyValueParser.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+/**
+ * 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 DummyValueParser(ElementParser parent) {
+    super(parent);
+  }
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    switch (token) {
+      case START_ARRAY:
+      case START_OBJECT:
+        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);
+    }
+  }
+
+  public void parseTail(TokenIterator tokenizer) {
+
+    // Parse (field: value)* }
+
+    for (;;) {
+      JsonToken token = tokenizer.requireNext();
+      switch (token) {
+
+        // Not exactly precise, but the JSON parser handles the
+        // details.
+
+        case END_OBJECT:
+        case END_ARRAY:
+          return;
+
+        case START_OBJECT:
+        case START_ARRAY:
+          parseTail(tokenizer); // Recursively ignore objects
+          break;
+
+        default:
+          break; // Ignore all else
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..b21b034
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ElementParser.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+/**
+ * Parser for a JSON element. Parsers are structured in a hierarchy:
+ * <ul>
+ * <li>Root - handles top-level objects and arrays, as well as EOF
+ * detection.</li>
+ * <li>Object - Parses {@code field: value} pairs.</li>
+ * <li>Value - Parses a value, which may be an array or an object.</li>
+ * <li>Array - Nested within a Value; parses one level of an array.
+ * Its children are Values (which may contain more array levels.</li>
+ * <li>
+ * JSON is completely generic; the element parsers handle JSON's full
+ * flexibility. Listeners attached to each parser determine if the
+ * actual value in any position makes sense for the structure being
+ * 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/ErrorFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ErrorFactory.java
new file mode 100644
index 0000000..b58763f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ErrorFactory.java
@@ -0,0 +1,76 @@
+/*
+ * 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.io.IOException;
+
+import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * To avoid coupling the JSON structure parser with Drill's error
+ * reporting mechanism, the caller passes in an instance of this
+ * error factory which will build the required errors, including
+ * filling in caller-specific context.
+ */
+public interface ErrorFactory {
+
+  /**
+   * The Jackson JSON parser failed to start on the input file.
+   */
+  RuntimeException parseError(String string, JsonParseException e);
+
+  /**
+   * I/O error reported from the Jackson JSON parser.
+   */
+  RuntimeException ioException(IOException e);
+
+  /**
+   * General structure-level error: something very unusual occurred
+   * in the JSON that passed Jackson, but failed in the structure
+   * parser.
+=   */
+  RuntimeException structureError(String string);
+
+  /**
+   * The Jackson parser reported a syntax error. Will not
+   * occur if recovery is enabled.
+   */
+  RuntimeException syntaxError(JsonParseException e);
+
+  /**
+   * The Jackson parser reported an error when trying to convert
+   * a value to a specific type. Should never occur since we only
+   * convert to the type that Jackson itself identified.
+   */
+  RuntimeException typeError(UnsupportedConversionError e);
+
+  /**
+   * Received an unexpected token. Should never occur as
+   * the Jackson parser itself catches errors.
+   */
+  RuntimeException syntaxError(JsonToken token);
+
+  /**
+   * Error recover is on, the structure parser tried to recover, but
+   * encountered too many other errors and gave up.
+   */
+  RuntimeException unrecoverableError();
+}
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
new file mode 100644
index 0000000..78d2e67
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureOptions.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+/**
+ * Input to the JSON structure parser which defines guidelines
+ * for low-level parsing as well as listeners for higher-level
+ * semantics.
+ */
+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.
+   */
+
+  public boolean allowNanInf;
+
+  /**
+   * Describes whether or not this reader can unwrap a single root array record
+   * and treat it like a set of distinct records.
+   */
+  public boolean skipOuterList = true;
+
+  /**
+   * If true, the structure parser will attempt to recover from JSON syntax
+   * errors by starting over at the next record boundary. The Jackson
+   * parser has limited recovery abilities. At present, recover can consume
+   * two or three valid records before it stabilizes.
+   */
+  public boolean skipMalformedRecords;
+}
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
new file mode 100644
index 0000000..23693ca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
@@ -0,0 +1,218 @@
+/*
+ * 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.io.IOException;
+import java.io.InputStream;
+
+import org.apache.drill.exec.store.easy.json.parser.RootParser.RootArrayParser;
+import org.apache.drill.exec.store.easy.json.parser.RootParser.RootObjectParser;
+import org.apache.drill.exec.store.easy.json.parser.TokenIterator.RecoverableJsonException;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+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.databind.ObjectMapper;
+
+/**
+ * Parser for JSON that converts a stream of tokens from the Jackson JSON
+ * parser into a set of events on listeners structured to follow the
+ * data structure of the incoming data. JSON can assume many forms. This
+ * class assumes that the data is in a tree structure that corresponds
+ * to the Drill row structure: a series of object with (mostly) the
+ * same schema. Members of the top-level object can be Drill types:
+ * scalars, arrays, nested objects (Drill "MAP"s), and so on.
+ * <p>
+ * The structure parser follows the structure of the incoming data,
+ * whatever it might be. This class imposes no semantic rules on that
+ * data, it just "calls 'em as it sees 'em" as they say. The listeners
+ * are responsible for deciding if the data data makes sense, and if
+ * so, how it should be handled.
+ * <p>
+ * The root listener will receive an event to fields in the top-level
+ * object as those fields first appear. Each field is a value object
+ * and can correspond to a scalar, array, another object, etc. The
+ * type of the value is declared when known, but sometimes it is not
+ * known, such as if the value is {@code null}. And, of course, according
+ * to JSON, the value is free to change from one row to the next. The
+ * listener decides it if wants to handle such "schema change", and if
+ * so, how.
+ */
+public class JsonStructureParser {
+  protected static final Logger logger = LoggerFactory.getLogger(JsonStructureParser.class);
+
+  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 int errorRecoveryCount;
+
+  /**
+   * Constructor for the structure parser.
+   *
+   * @param stream the source of JSON text
+   * @param options configuration options for the Jackson JSON parser
+   * and this structure parser
+   * @param rootListener listener for the top-level objects in the
+   * JSON stream
+   * @param errorFactory factory for errors thrown for various
+   * conditions
+   */
+  public JsonStructureParser(InputStream stream, JsonStructureOptions options,
+      ObjectListener rootListener, ErrorFactory errorFactory) {
+    this.options = Preconditions.checkNotNull(options);
+    this.rootListener = Preconditions.checkNotNull(rootListener);
+    this.errorFactory = Preconditions.checkNotNull(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);
+
+      parser = mapper.getFactory().createParser(stream);
+    } catch (JsonParseException e) {
+      throw errorFactory().parseError("Failed to create the JSON parser", e);
+    } catch (IOException e) {
+      throw errorFactory().ioException(e);
+    }
+    tokenizer = new TokenIterator(parser, options, errorFactory());
+    rootState = makeRootState();
+  }
+
+  public JsonStructureOptions options() { return options; }
+  public ErrorFactory errorFactory() { return errorFactory; }
+  public ObjectListener rootListener() { return rootListener; }
+
+  private RootParser makeRootState() {
+    JsonToken token = tokenizer.next();
+    if (token == null) {
+      return null;
+    }
+    switch (token) {
+
+      // File contains an array of records.
+      case START_ARRAY:
+        if (options.skipOuterList) {
+          return new RootArrayParser(this);
+        } else {
+          throw errorFactory().structureError(
+              "JSON includes an outer array, but outer array support is not enabled");
+        }
+
+      // File contains a sequence of one or more records,
+      // presumably sequentially.
+      case START_OBJECT:
+        tokenizer.unget(token);
+        return new RootObjectParser(this);
+
+      // Not a valid JSON file for Drill.
+      // Won't get here because the Jackson parser catches errors.
+      default:
+        throw errorFactory().syntaxError(token);
+    }
+  }
+
+  public boolean next() {
+    if (rootState == null) {
+      // Only occurs for an empty document
+      return false;
+    }
+    for (;;) {
+      try {
+        return rootState.parseRoot(tokenizer);
+      } catch (RecoverableJsonException e) {
+        if (! recover()) {
+          return false;
+        }
+      }
+    }
+  }
+
+  /**
+   * Attempt recovery from a JSON syntax error by skipping to the next
+   * record. The Jackson parser is quite limited in its recovery abilities.
+   *
+   * @return {@code true}  if another record can be read, {@code false}
+   * if EOF.
+   * @throws UserException if the error is unrecoverable
+   * @see <a href="https://issues.apache.org/jira/browse/DRILL-4653">DRILL-4653</a>
+   * @see <a href="https://issues.apache.org/jira/browse/DRILL-5953">DRILL-5953</a>
+   */
+  private boolean recover() {
+    logger.warn("Attempting recovery from JSON syntax error. " + tokenizer.context());
+    boolean firstAttempt = true;
+    for (;;) {
+      for (;;) {
+        try {
+          if (parser.isClosed()) {
+            throw errorFactory().unrecoverableError();
+          }
+          JsonToken token = tokenizer.next();
+          if (token == null) {
+            if (firstAttempt) {
+              throw errorFactory().unrecoverableError();
+            }
+            return false;
+          }
+          if (token == JsonToken.NOT_AVAILABLE) {
+            return false;
+          }
+          if (token == JsonToken.END_OBJECT) {
+            break;
+          }
+          firstAttempt = false;
+        } catch (RecoverableJsonException e) {
+          // Ignore, keep trying
+        }
+      }
+      try {
+        JsonToken token = tokenizer.next();
+        if (token == null || token == JsonToken.NOT_AVAILABLE) {
+          return false;
+        }
+        if (token == JsonToken.START_OBJECT) {
+          logger.warn("Attempting to resume JSON parse. " + tokenizer.context());
+          tokenizer.unget(token);
+          errorRecoveryCount++;
+          return true;
+        }
+      } catch (RecoverableJsonException e) {
+        // Ignore, keep trying
+      }
+    }
+  }
+
+  public int recoverableErrorCount() { return errorRecoveryCount; }
+
+  public void close() {
+    if (errorRecoveryCount > 0) {
+      logger.warn("Read JSON input with {} recoverable error(s).",
+          errorRecoveryCount);
+    }
+    try {
+      parser.close();
+    } catch (IOException e) {
+      logger.warn("Ignored failure when closing JSON source", e);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonType.java
new file mode 100644
index 0000000..9481a4a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonType.java
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/**
+ * Description of JSON types as derived from JSON tokens.
+ */
+public enum JsonType {
+  ARRAY, OBJECT, NULL, EMPTY, BOOLEAN, INTEGER, FLOAT, STRING
+}
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
new file mode 100644
index 0000000..ec40b0e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectListener.java
@@ -0,0 +1,131 @@
+/*
+ * 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;
+
+/**
+ * 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 {
+
+  /**
+   * 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 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();
+
+  /**
+   * Called by the structure parser when it first sees a new field for
+   * and object to determine if that field is to be projected (is needed
+   * by the listener.) 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.
+   *
+   * @param key the object field name
+   * @return {@code true} if this listener wants to provide a listener
+   * for the field, {@code false} if the field should be ignored
+   */
+  boolean isProjected(String key);
+
+  /**
+   * A new field has appeared with a scalar (or {@code null}) value.
+   * That is: {@code key: <scalar>}.
+   *
+   * @param key the field name
+   * @param type the type as given by the JSON token for the value
+   * @return a value listener for the scalar value
+   */
+  ValueListener addScalar(String key, JsonType type);
+
+  /**
+   * A new field has appeared with a scalar, {@code null} or empty array
+   * value. That is, one of:
+   * <ul>
+   * <li><code>key: [+ &lt;scalar></code></li>
+   * <li><code>key: [+ null</code></li>
+   * <li><code>key: [+ ]</code></li>
+   * </ul>
+   * Where "[+" means one or more opening array elements.
+   *
+   * @param key the field name
+   * @param arrayDims number of dimensions observed in the first appearance
+   * of the array (more may appear later)
+   * @param type the observed type of the first element of the array, or
+   * {@link JsonType.NULL} if {@code null} was see, or
+   * {@link JsonType.EMPTY} if an empty array was seen
+   * @return a listener for the field itself which is prepared to
+   * return an array listener
+   */
+  ValueListener addArray(String key, int arrayDims, JsonType type);
+
+  /**
+   * A new field has appeared with an object value.
+   * That is: {@code key: <scalar>}.
+   *
+   * @param key the field name
+   * @return a value listener which assumes the value is an object
+   */
+  ValueListener addObject(String key);
+
+  /**
+   * A new field has appeared with an object array value.
+   * That is: <code>key: ]+ {</code>.
+   *
+   * @param key the field name
+   * @return a value listener which assumes the value is an object
+   * array
+   */
+  ValueListener addObjectArray(String key, int dims);
+}
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
new file mode 100644
index 0000000..cfc0c76
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ObjectParser.java
@@ -0,0 +1,167 @@
+/*
+ * 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.Map;
+
+import org.apache.drill.common.map.CaseInsensitiveMap;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * Parses a JSON object: <code>{ name : value ... }</code>
+ * <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.
+ * Thereafter, the previous parser is reused.
+ * <p>
+ * The object listener provides semantics. One key decision is whether
+ * to project a field or not. An unprojected field is parsed with
+ * a "dummy" parser that "free-wheels" over all valid JSON structures.
+ * Otherwise, the listener is given whatever type information that the
+ * parser can discover when creating the field.
+ * <p>
+ * Work is divided between this class, which discovers fields, and
+ * the listeners which determine the meaning of field values. A field,
+ * via a properly-defined listener, can accept one or more different
+ * value kinds.
+ * <p>
+ * The parser accepts JSON tokens as they appear in the file. The
+ * question of whether those tokens make sense is left to the listeners.
+ * The listeners decide if the tokens make sense for a particular column.
+ * The listener should provide a clear error if a particular token is not
+ * valid for a given listener.
+ *
+ * <h4>Nulls</h4>
+ *
+ * Null values are handled at the semantic, not syntax level. If the
+ * first appearance of a field contains a null value, then the parser can
+ * provide no hints about the expected field type. The listener must
+ * implement a solution such as referring to a schema, waiting for a
+ * non-null value to appear, etc.
+ * <p>
+ * Since the parser classes handle syntax, they are blissfully ignorant
+ * of any fancy logic needed for null handling. Each field is
+ * represented by a field parser whether that field is null or not.
+ * It is the listener that may have to swap out one mechanism for
+ * another as types are discovered.
+ *
+ * <h4>Complex Types</h4>
+ *
+ * Parsers handle arrays and objects using a two-level system. Each field
+ * always is driven by a field parser. If the field is discovered to be an
+ * array, then we add an array parser to the field parser to handle array
+ * contents. The same is true of objects.
+ * <p>
+ * Both objects and arrays are collections of values, and a value can
+ * optionally contain an array or object. (JSON allows any given field
+ * name to map to both objects and arrays in different rows. The parser
+ * structure reflects this syntax. The listeners can enforce more
+ * relational-like semantics).
+ * <p>
+ * If an array is single-dimension, then the field parse contains an array
+ * parser which contains another value parser for the array contents. If
+ * 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;
+  private final Map<String, ElementParser> members = CaseInsensitiveMap.newHashMap();
+
+  public ObjectParser(ElementParser parent, ObjectListener listener) {
+    super(parent);
+    this.listener = listener;
+  }
+
+  public ObjectListener listener() { return listener; }
+
+  /**
+   * Parses <code>{ ^ ... }</code>
+   */
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    listener.onStart();
+
+    // Parse (field: value)* }
+
+    top: for (;;) {
+      JsonToken token = tokenizer.requireNext();
+      // Position: { (key: value)* ? ^
+      switch (token) {
+        case END_OBJECT:
+          // Position: { (key: value)* } ^
+          break top;
+
+        case FIELD_NAME:
+          // Position: { (key: value)* key: ^
+          parseMember(tokenizer);
+          break;
+
+        default:
+          // Position: { (key: value)* ~(key | }) ^
+          // Invalid JSON.
+          // Actually, we probably won't get here, the JSON parser
+          // itself will throw an exception.
+          throw errorFactory().syntaxError(token);
+      }
+    }
+    listener.onEnd();
+  }
+
+  /**
+   * Parse a field. Two cases. First, this is a field we've already seen. If so,
+   * look up the parser for that field and use it. If this is the first time
+   * we've seen the field, "sniff" tokens to determine field type, create a
+   * parser, then parse.
+   */
+  private void parseMember(TokenIterator tokenizer) {
+    // Position: key: ^ ?
+    final String key = tokenizer.textValue().trim();
+    ElementParser fieldParser = members.get(key);
+    if (fieldParser == null) {
+      // 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);
+      members.put(key, fieldParser);
+    }
+    // Parse the field value.
+    // Position: key: ^ value ...
+    fieldParser.parse(tokenizer);
+  }
+
+  /**
+   * If the column is not projected, create a dummy parser to "free wheel" over
+   * the value. Otherwise, look ahead a token or two to determine the the type
+   * of the field. Then the caller will backtrack to parse the field.
+   *
+   * @param key name of the field
+   * @return parser for the field
+   */
+  private ElementParser detectValueParser(TokenIterator tokenizer, final String key) {
+    if (key.isEmpty()) {
+      throw errorFactory().structureError(
+          "Drill does not allow empty keys in JSON key/value pairs");
+    }
+    if (!listener.isProjected(key)) {
+      return new DummyValueParser(this);
+    }
+    return ValueFactory.createFieldParser(this, key, tokenizer);
+  }
+}
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
new file mode 100644
index 0000000..f81fd3a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/RootParser.java
@@ -0,0 +1,121 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.JsonToken;
+
+/**
+ * The root parsers are special: they must detect EOF. Drill supports
+ * top-level objects either enclosed in an array (which forms legal
+ * JSON), or as a series JSON objects (which is a common, if not
+ * entirely legal, form of JSON.)
+ */
+public abstract class RootParser implements ElementParser {
+  protected static final Logger logger = LoggerFactory.getLogger(RootParser.class);
+
+  private final JsonStructureParser structParser;
+  protected final ObjectParser rootObject;
+
+  public RootParser(JsonStructureParser structParser) {
+    this.structParser = structParser;
+    this.rootObject = new ObjectParser(this, structParser.rootListener());
+  }
+
+  public abstract boolean parseRoot(TokenIterator tokenizer);
+
+  @Override
+  public void parse(TokenIterator tokenizer) {
+    throw new UnsupportedOperationException();
+  }
+
+  protected boolean parseRootObject(JsonToken token, TokenIterator tokenizer) {
+    // Position: ^ ?
+    switch (token) {
+      case NOT_AVAILABLE:
+        return false; // Should never occur
+
+      case START_OBJECT:
+        // Position: { ^
+        rootObject.parse(tokenizer);
+        break;
+
+      default:
+        // Position ~{ ^
+        // Not a valid object.
+        // Won't actually get here: the Jackson parser prevents it.
+        throw errorFactory().syntaxError(token); // Nothing else is valid
+    }
+    return true;
+  }
+
+  protected ErrorFactory errorFactory() {
+    return structParser.errorFactory();
+  }
+
+  @Override
+  public ElementParser parent() { return null; }
+
+  @Override
+  public JsonStructureParser structParser() { return structParser; }
+
+  public static class RootObjectParser extends RootParser {
+
+    public RootObjectParser(JsonStructureParser structParser) {
+      super(structParser);
+    }
+
+    @Override
+    public boolean parseRoot(TokenIterator tokenizer) {
+      JsonToken token = tokenizer.next();
+      if (token == null) {
+        // Position: EOF ^
+        return false;
+      } else {
+        return parseRootObject(token, tokenizer);
+      }
+    }
+  }
+
+  public static class RootArrayParser extends RootParser {
+
+    public RootArrayParser(JsonStructureParser structParser) {
+      super(structParser);
+    }
+
+    @Override
+    public boolean parseRoot(TokenIterator tokenizer) {
+      JsonToken token = tokenizer.next();
+      if (token == null) {
+        // Position: { ... EOF ^
+        // Saw EOF, but no closing ]. Warn and ignore.
+        // Note that the Jackson parser won't let us get here;
+        // it will have already thrown a syntax error.
+        logger.warn("Failed to close outer array. {}",
+            tokenizer.context());
+        return false;
+      } else if (token == JsonToken.END_ARRAY) {
+        return false;
+      } else {
+        return parseRootObject(token, tokenizer);
+      }
+    }
+  }
+}
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
new file mode 100644
index 0000000..cfb7440
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/TokenIterator.java
@@ -0,0 +1,149 @@
+/*
+ * 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.io.IOException;
+
+import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
+
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+
+public class TokenIterator {
+  public static final int MAX_LOOKAHEAD = 30;
+
+  /**
+   * Internal exception to unwind the stack when a syntax
+   * error is detected within a record. Allows for recovery.
+   */
+  @SuppressWarnings("serial")
+  class RecoverableJsonException extends RuntimeException {
+  }
+
+  private final JsonParser parser;
+  private final JsonStructureOptions options;
+  private final ErrorFactory errorFactory;
+  private final JsonToken[] lookahead = new JsonToken[MAX_LOOKAHEAD];
+  private int count;
+
+  public TokenIterator(JsonParser parser, JsonStructureOptions options, ErrorFactory errorFactory) {
+    this.parser = parser;
+    this.options = options;
+    this.errorFactory = errorFactory;
+  }
+
+  public ErrorFactory errorFactory() { return errorFactory; }
+
+  public JsonToken next() {
+    if (count > 0) {
+      return lookahead[--count];
+    }
+    try {
+      return parser.nextToken();
+    } catch (JsonParseException e) {
+      if (options.skipMalformedRecords) {
+        throw new RecoverableJsonException();
+      } else {
+        throw errorFactory.syntaxError(e);
+      }
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    }
+  }
+
+  public String context() {
+    JsonLocation location = parser.getCurrentLocation();
+    String token;
+    try {
+      token = parser.getText();
+    } catch (IOException e) {
+      token = "<unknown>";
+    }
+    return new StringBuilder()
+        .append("line ")
+        .append(location.getLineNr())
+        .append(", column ")
+        .append(location.getColumnNr())
+        .append(", near token \"")
+        .append(token)
+        .append("\"")
+        .toString();
+  }
+
+  public JsonToken requireNext() {
+    JsonToken token = next();
+    if (token == null) {
+      throw errorFactory.structureError("Premature EOF of JSON file");
+    }
+    return token;
+  }
+
+  public JsonToken peek() {
+    JsonToken token = requireNext();
+    unget(token);
+    return token;
+  }
+
+  public void unget(JsonToken token) {
+    if (count == lookahead.length) {
+      throw errorFactory.structureError(
+          String.format("Excessive JSON array nesting. Max allowed: %d", lookahead.length));
+    }
+    lookahead[count++] = token;
+  }
+
+  public String textValue() {
+    try {
+      return parser.getText();
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    }
+  }
+
+  public long longValue() {
+    try {
+      return parser.getLongValue();
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    } catch (UnsupportedConversionError e) {
+      throw errorFactory.typeError(e);
+    }
+  }
+
+  public String stringValue() {
+    try {
+      return parser.getValueAsString();
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    } catch (UnsupportedConversionError e) {
+      throw errorFactory.typeError(e);
+    }
+  }
+
+  public double doubleValue() {
+    try {
+      return parser.getValueAsDouble();
+    } catch (IOException e) {
+      throw errorFactory.ioException(e);
+    } catch (UnsupportedConversionError e) {
+      throw errorFactory.typeError(e);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueFactory.java
new file mode 100644
index 0000000..97ca0ca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueFactory.java
@@ -0,0 +1,222 @@
+/*
+ * 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;
+
+/**
+ * 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>
+ * This class 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 class
+ * looks only at the first appearance of a value. JSON allows anything.
+ * The listener must enforce semantic rules that say whether a different
+ * type is allowed for later values.
+ */
+public class ValueFactory {
+
+  public static class FieldDescrip {
+    protected int arrayDims;
+    protected JsonType type;
+
+    public boolean isArray() { return arrayDims > 0; }
+
+    public boolean isObject() { return type == JsonType.OBJECT; }
+  }
+
+  private ValueFactory() { }
+
+  /**
+   * Parse position: <code>{ ... field : ^ ?</code> for a newly-seen field.
+   * Look ahead to guess the field type, then declare the field.
+   *
+   * @param parent the object parser declaring the field
+   * @param key the name of the field
+   * @param tokenizer the token parser
+   * @return the value parser for the element, which may contain additional
+   * structure for objects or arrays
+   */
+  public static ElementParser createFieldParser(ObjectParser parent, String key, TokenIterator tokenizer) {
+    FieldDescrip descrip = new FieldDescrip();
+    inferFieldType(descrip, tokenizer);
+    ObjectListener objListener = parent.listener();
+    ValueListener fieldListener;
+    if (descrip.isObject()) {
+      if (descrip.isArray()) {
+        // Object array field
+        fieldListener = objListener.addObjectArray(key, descrip.arrayDims);
+      } else {
+        // Object field
+        fieldListener = objListener.addObject(key);
+      }
+    } else {
+      if (descrip.isArray()) {
+        // Scalar (or unknown) array field
+        fieldListener = objListener.addArray(key, descrip.arrayDims, descrip.type);
+      } else {
+        // Scalar field
+        fieldListener = objListener.addScalar(key, descrip.type);
+      }
+    }
+    ValueParser fp = new ValueParser(parent, key, fieldListener);
+    createStructureParser(fp, descrip);
+    return fp;
+  }
+
+  /**
+   * Add the object or array parser, if the structured type is known.
+   */
+  private static void createStructureParser(ValueParser valueParser,
+      FieldDescrip descrip) {
+    if (descrip.isArray()) {
+      valueParser.bindArrayParser(createArrayParser(valueParser, descrip));
+    } else if (descrip.isObject()) {
+      valueParser.bindObjectParser(objectParser(valueParser));
+    }
+  }
+
+  /**
+   * Parse position: <code>... [ ?</code> for a field or array element not previously
+   * known to be an array. Look ahead to determine if the array is nested and its
+   * element types.
+   *
+   * @param parent the parser for the value that has been found to contain an
+   * array
+   * @param tokenizer the JSON token parser
+   * @return an array parser to bind to the parent value parser to parse the
+   * array
+   */
+  public static ArrayParser createArrayParser(ValueParser parent, TokenIterator tokenizer) {
+    FieldDescrip descrip = new FieldDescrip();
+    // Already in an array, so add the outer dimension.
+    descrip.arrayDims++;
+    inferFieldType(descrip, tokenizer);
+    return createArrayParser(parent, descrip);
+  }
+
+  public static ArrayParser createArrayParser(ValueParser parent, FieldDescrip descrip) {
+    ValueListener fieldListener = parent.listener();
+    ArrayListener arrayListener;
+    if (descrip.isObject()) {
+      // Object array elements
+      arrayListener = fieldListener.objectArray(descrip.arrayDims);
+    } else {
+      arrayListener = fieldListener.array(descrip.arrayDims, descrip.type);
+    }
+    descrip.arrayDims--;
+    ValueListener elementListener;
+    if (descrip.isObject()) {
+      if (descrip.isArray()) {
+        // Object array elements
+        elementListener = arrayListener.objectArrayElement(descrip.arrayDims);
+      } else {
+        // Object elements
+        elementListener = arrayListener.objectElement();
+      }
+    } else {
+      if (descrip.isArray()) {
+        // Scalar (or unknown) array elements
+        elementListener = arrayListener.arrayElement(descrip.arrayDims, descrip.type);
+      } else {
+        // Scalar elements
+        elementListener = arrayListener.scalarElement(descrip.type);
+      }
+    }
+    ArrayParser arrayParser = new ArrayParser(parent, arrayListener, elementListener);
+    createStructureParser(arrayParser.elementParser(), descrip);
+    return arrayParser;
+  }
+
+  public static ObjectParser objectParser(ValueParser parent) {
+    ValueListener valueListener = parent.listener();
+    ObjectListener objListener = valueListener.object();
+    return new ObjectParser(parent, objListener);
+  }
+
+  protected static void inferFieldType(FieldDescrip descrip, TokenIterator tokenizer) {
+    JsonToken token = tokenizer.requireNext();
+    switch (token) {
+      case START_ARRAY:
+        // Position: key: [ ^
+        descrip.arrayDims++;
+        inferFieldType(descrip, tokenizer);
+        break;
+
+      case END_ARRAY:
+        if (descrip.arrayDims == 0) {
+          throw tokenizer.errorFactory().syntaxError(token);
+        }
+        descrip.type = JsonType.EMPTY;
+        break;
+
+      case START_OBJECT:
+        // Position: key: { ^
+        descrip.type = JsonType.OBJECT;
+        break;
+
+      case VALUE_NULL:
+
+        // Position: key: null ^
+        descrip.type = JsonType.NULL;
+        break;
+
+      case VALUE_FALSE:
+      case VALUE_TRUE:
+        descrip.type = JsonType.BOOLEAN;
+        break;
+
+      case VALUE_NUMBER_INT:
+        descrip.type = JsonType.INTEGER;
+        break;
+
+      case VALUE_NUMBER_FLOAT:
+        descrip.type = JsonType.FLOAT;
+        break;
+
+      case VALUE_STRING:
+        descrip.type = JsonType.STRING;
+        break;
+
+      default:
+        // Won't get here: the Jackson parser catches
+        // errors.
+        throw tokenizer.errorFactory().syntaxError(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
new file mode 100644
index 0000000..9625f55
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueListener.java
@@ -0,0 +1,152 @@
+/*
+ * 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;
+
+/**
+ * Represents a JSON object, either a direct object field, or level
+ * within an array. That is:
+ * <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><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 {
+
+  /**
+   * 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.
+   *
+   * @return {@code true} if the field is to be read in "all-text mode" even
+   * if the global setting is off, {@code false} to read the field as
+   * typed values.
+   */
+  boolean isText();
+
+  /**
+   * 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 onEmbedddObject(String value);
+
+  /**
+   * The parser has encountered a object value for the field for the first
+   * time. That is: {@code foo: {</code}.
+   *
+   * @return an object listener for the object
+   */
+  ObjectListener object();
+
+  /**
+   * The parser has encountered a array value for the first
+   * time, and that array is scalar, null or empty.
+   *
+   * @param arrayDims the number of observed array dimensions
+   * @param type the observed JSON token type for the array element
+   * @return an array listener for the array
+   */
+  ArrayListener array(int arrayDims, JsonType type);
+
+  /**
+   * The parser has encountered a array value for the first
+   * time, and that array contains an object.
+   *
+   * @param arrayDims the number of observed array dimensions
+   * @return an array listener for the array
+   */
+  ArrayListener objectArray(int arrayDims);
+}
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
new file mode 100644
index 0000000..ac7ee31
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/ValueParser.java
@@ -0,0 +1,174 @@
+/*
+ * 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 a JSON value. JSON allows any value type to appear anywhere a
+ * value is allowed; this parser reflects that rule. The associated listener
+ * is responsible for semantics: whether a given value should be allowed.
+ * <p>
+ * Scalar value processing occurs in one of two ways:
+ * <ul>
+ * <li><b>Typed</b>: The type of the JSON value determines which of the
+ * listener "on" method is called. This ensures that the JSON text
+ * is parsed into typed values using JSON's parsing rules.</li>
+ * <li><b>Text</b>: The text value is passed to the listener's
+ * {@code onString()} method regardless of the JSON type. (That is,
+ * according to Drill's "all-text mode."</li>
+ * </ul>
+ * Listeners can enforce one type only, or can be more flexible and
+ * allow multiple types.
+ */
+public class ValueParser extends AbstractElementParser {
+
+  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.onEmbedddObject(tokenizer.stringValue());
+        default:
+          // Won't get here: the Jackson parser catches
+          // errors.
+          throw errorFactory().syntaxError(token);
+      }
+    }
+  }
+
+  /**
+   * 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;
+  private final ValueListener listener;
+  private final ValueHandler valueHandler;
+  private ObjectParser objectParser;
+  private ArrayParser arrayParser;
+
+  public ValueParser(ElementParser parent, String key, ValueListener listener) {
+    super(parent);
+    this.key = key;
+    this.listener = listener;
+    if (listener.isText() || structParser().options().allTextMode) {
+      valueHandler = new TextValueHandler();
+    } else {
+      valueHandler = new TypedValueHandler();
+    }
+  }
+
+  public String key() { return key; }
+
+  public ValueListener listener() { return listener; }
+
+  public void bindObjectParser(ObjectParser parser) {
+    objectParser = parser;
+  }
+
+  public void bindArrayParser(ArrayParser parser) {
+    arrayParser = parser;
+  }
+
+  /**
+   * 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 = ValueFactory.objectParser(this);
+      }
+      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 = ValueFactory.createArrayParser(this, tokenizer);
+      }
+      arrayParser.parse(tokenizer);
+      break;
+
+    case VALUE_NULL:
+      listener.onNull();
+      break;
+
+    default:
+      valueHandler.accept(tokenizer, token);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/BaseTestJsonParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/BaseTestJsonParser.java
new file mode 100644
index 0000000..3f9b715
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/BaseTestJsonParser.java
@@ -0,0 +1,345 @@
+/*
+ * 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.json.parser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.io.input.ReaderInputStream;
+import org.apache.drill.exec.store.easy.json.parser.ArrayListener;
+import org.apache.drill.exec.store.easy.json.parser.ErrorFactory;
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureOptions;
+import org.apache.drill.exec.store.easy.json.parser.JsonStructureParser;
+import org.apache.drill.exec.store.easy.json.parser.JsonType;
+import org.apache.drill.exec.store.easy.json.parser.ObjectListener;
+import org.apache.drill.exec.store.easy.json.parser.ValueListener;
+import org.apache.drill.exec.vector.accessor.UnsupportedConversionError;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonToken;
+
+public class BaseTestJsonParser {
+
+  @SuppressWarnings("serial")
+  protected static class JsonErrorFixture extends RuntimeException {
+    String errorType;
+
+    public JsonErrorFixture(String errorType, String msg, Exception e) {
+      super(msg, e);
+      this.errorType = errorType;
+    }
+
+    public JsonErrorFixture(String errorType, String msg) {
+      super(msg);
+      this.errorType = errorType;
+    }
+  }
+
+  /**
+   * Convert JSON errors to a simple form for use in tests.
+   * Not all errors are throw in normal operation; some require
+   * faults in the I/O system or in the Jackson parser.
+   */
+  protected static class ErrorFactoryFixture implements ErrorFactory {
+
+    @Override
+    public RuntimeException parseError(String msg, JsonParseException e) {
+      throw new JsonErrorFixture("parseError", msg, e);
+    }
+
+    @Override
+    public RuntimeException ioException(IOException e) {
+      throw new JsonErrorFixture("ioException", "", e);
+    }
+
+    @Override
+    public RuntimeException structureError(String msg) {
+      throw new JsonErrorFixture("structureError", msg);
+    }
+
+    @Override
+    public RuntimeException syntaxError(JsonParseException e) {
+      throw new JsonErrorFixture("syntaxError", "", e);
+    }
+
+    @Override
+    public RuntimeException typeError(UnsupportedConversionError e) {
+      throw new JsonErrorFixture("typeError", "", e);
+    }
+
+    @Override
+    public RuntimeException syntaxError(JsonToken token) {
+      throw new JsonErrorFixture("syntaxError", token.toString());
+    }
+
+    @Override
+    public RuntimeException unrecoverableError() {
+      throw new JsonErrorFixture("unrecoverableError", "");
+    }
+  }
+
+  protected static class ValueListenerFixture implements ValueListener {
+
+    final int dimCount;
+    final JsonType type;
+    int nullCount;
+    int valueCount;
+    Object value;
+    ObjectListenerFixture objectValue;
+    ArrayListenerFixture arrayValue;
+
+    public ValueListenerFixture(int dimCount, JsonType type) {
+      this.dimCount = dimCount;
+      this.type = type;
+    }
+
+    @Override
+    public boolean isText() { return false; }
+
+    @Override
+    public void onNull() {
+      nullCount++;
+    }
+
+    @Override
+    public void onBoolean(boolean value) {
+      this.value = value;
+      valueCount++;
+    }
+
+    @Override
+    public void onInt(long value) {
+      this.value = value;
+      valueCount++;
+    }
+
+    @Override
+    public void onFloat(double value) {
+      this.value = value;
+      valueCount++;
+    }
+
+    @Override
+    public void onString(String value) {
+      this.value = value;
+      valueCount++;
+    }
+
+    @Override
+    public void onEmbedddObject(String value) {
+      this.value = value;
+      valueCount++;
+    }
+
+    @Override
+    public ObjectListener object() {
+      assertNull(objectValue);
+      objectValue = new ObjectListenerFixture();
+      return objectValue;
+    }
+
+    @Override
+    public ArrayListener array(int arrayDims, JsonType type) {
+      assertNull(arrayValue);
+      arrayValue = new ArrayListenerFixture(arrayDims, type);
+      return arrayValue;
+    }
+
+    @Override
+    public ArrayListener objectArray(int arrayDims) {
+      assertNull(arrayValue);
+      arrayValue = new ArrayListenerFixture(arrayDims, JsonType.OBJECT);
+      return arrayValue;
+    }
+  }
+
+  protected static class ArrayListenerFixture implements ArrayListener {
+
+    final int dimCount;
+    final JsonType type;
+    int startCount;
+    int endCount;
+    int elementCount;
+    ValueListenerFixture element;
+
+    public ArrayListenerFixture(int dimCount, JsonType type) {
+      this.dimCount = dimCount;
+      this.type = type;
+    }
+
+    @Override
+    public void onStart() {
+      startCount++;
+    }
+
+    @Override
+    public void onElement() {
+      elementCount++;
+    }
+
+    @Override
+    public void onEnd() {
+      endCount++;
+    }
+
+    @Override
+    public ValueListener objectArrayElement(int arrayDims) {
+      return element(arrayDims, JsonType.OBJECT);
+    }
+
+    @Override
+    public ValueListener objectElement() {
+      return element(0, JsonType.OBJECT);
+    }
+
+    @Override
+    public ValueListener arrayElement(int arrayDims, JsonType type) {
+      return element(arrayDims, type);
+    }
+
+    @Override
+    public ValueListener scalarElement(JsonType type) {
+      return element(0, type);
+    }
+
+    private ValueListener element(int arrayDims, JsonType type) {
+      assertNull(element);
+      element = new ValueListenerFixture(arrayDims, type);
+      return element;
+    }
+  }
+
+  protected static class ObjectListenerFixture implements ObjectListener {
+
+    final Map<String, ValueListenerFixture> fields = new HashMap<>();
+    Set<String> projectFilter;
+    int startCount;
+    int endCount;
+
+    @Override
+    public void onStart() {
+      startCount++;
+    }
+
+    @Override
+    public void onEnd() {
+      endCount++;
+    }
+
+    @Override
+    public boolean isProjected(String key) {
+      return projectFilter == null || projectFilter.contains(key);
+    }
+
+    @Override
+    public ValueListener addScalar(String key, JsonType type) {
+      return field(key, 0, type);
+    }
+
+    @Override
+    public ValueListener addArray(String key, int dims, JsonType type) {
+      return field(key, dims, type);
+    }
+
+    @Override
+    public ValueListener addObject(String key) {
+      return field(key, 0, JsonType.OBJECT);
+    }
+
+    @Override
+    public ValueListener addObjectArray(String key, int dims) {
+      return field(key, dims, JsonType.OBJECT);
+    }
+
+    private ValueListener field(String key, int dims, JsonType type) {
+      assertFalse(fields.containsKey(key));
+      ValueListenerFixture field = new ValueListenerFixture(dims, type);
+      fields.put(key, field);
+      return field;
+    }
+
+    public ValueListenerFixture field(String key) {
+      ValueListenerFixture field = fields.get(key);
+      assertNotNull(field);
+      return field;
+    }
+  }
+
+  protected static class JsonParserFixture {
+    JsonStructureOptions options = new JsonStructureOptions();
+    JsonStructureParser parser;
+    ObjectListenerFixture rootObject = new ObjectListenerFixture();
+    ErrorFactory errorFactory = new ErrorFactoryFixture();
+
+    public void open(String json) {
+      InputStream inStream = new
+          ReaderInputStream(new StringReader(json));
+      parser = new JsonStructureParser(inStream, options, rootObject,
+          errorFactory);
+    }
+
+    public boolean next() {
+      assertNotNull(parser);
+      return parser.next();
+    }
+
+    public int read() {
+      int i = 0;
+      while (next()) {
+        i++;
+      }
+      return i;
+    }
+
+    public ValueListenerFixture field(String key) {
+      return rootObject.field(key);
+    }
+
+    public void close() {
+      if (parser != null) {
+        parser.close();
+      }
+    }
+  }
+
+  protected static void expectError(String json, String kind) {
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    expectError(fixture, kind);
+    fixture.close();
+  }
+
+  protected static void expectError(JsonParserFixture fixture, String kind) {
+    try {
+      fixture.read();
+      fail();
+    } catch (JsonErrorFixture e) {
+      assertEquals(kind, e.errorType);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserArrays.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserArrays.java
new file mode 100644
index 0000000..97ccfc8
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserArrays.java
@@ -0,0 +1,397 @@
+/*
+ * 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.json.parser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonType;
+import org.junit.Test;
+
+/**
+ * Tests array (including multi-dimensional and object) support
+ * for the JSON structure parser.
+ */
+public class TestJsonParserArrays extends BaseTestJsonParser {
+
+  @Test
+  public void test1DArray() {
+    final String json =
+        "{a: [1, 100]} {a: [null]} \n" +
+        "{a: []} {a: null} {}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    // {a: [1, 100]}
+    assertTrue(fixture.next());
+
+    // Value of object.a
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.INTEGER, a.type);
+    assertEquals(1, a.dimCount);
+
+    // Array aspect of a
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture aArray = a.arrayValue;
+    assertEquals(1, aArray.startCount);
+    assertEquals(aArray.startCount, aArray.endCount);
+    assertEquals(1, aArray.dimCount);
+
+    // Value of each element of array aspect of a
+    assertNotNull(aArray.element);
+    ValueListenerFixture aElement = aArray.element;
+    assertEquals(JsonType.INTEGER, aElement.type);
+    assertEquals(0, aElement.dimCount);
+    assertNull(aElement.arrayValue);
+    assertEquals(2, aElement.valueCount);
+    assertEquals(100L, aElement.value);
+    assertEquals(0, aElement.nullCount);
+
+    // {a: [null]}
+    assertTrue(fixture.next());
+    assertEquals(2, aArray.startCount);
+    assertEquals(aArray.startCount, aArray.endCount);
+    assertEquals(2, aElement.valueCount);
+    assertEquals(1, aElement.nullCount);
+
+    // {a: []}
+    assertTrue(fixture.next());
+    assertEquals(3, aArray.startCount);
+    assertEquals(aArray.startCount, aArray.endCount);
+    assertEquals(2, aElement.valueCount);
+    assertEquals(1, aElement.nullCount);
+    assertEquals(0, a.nullCount);
+
+    // {a: null}
+    assertTrue(fixture.next());
+    assertEquals(3, aArray.startCount);
+    assertEquals(1, a.nullCount);
+
+    // {}
+    assertTrue(fixture.next());
+    assertEquals(3, aArray.startCount);
+    assertEquals(1, a.nullCount);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testNullToArray() {
+    final String json =
+        "{a: null} {a: [1, 100]}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    // Can't predict the future, all we know is a is null.
+    // {a: null}
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.NULL, a.type);
+    assertEquals(0, a.dimCount);
+    assertNull(a.arrayValue);
+
+    // See an array, can revise estimate of field type
+    // {a: [1, 100]}
+    assertTrue(fixture.next());
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture aArray = a.arrayValue;
+    assertEquals(1, aArray.dimCount);
+    ValueListenerFixture aElement = aArray.element;
+    assertEquals(2, aElement.valueCount);
+    assertEquals(100L, aElement.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testEmptyArray() {
+    final String json =
+        "{a: []} {a: [1, 100]}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    // Can't predict the future, all we know is a is an array.
+    // "{a: []}
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.EMPTY, a.type);
+    assertEquals(1, a.dimCount);
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture aArray = a.arrayValue;
+    assertEquals(1, aArray.dimCount);
+    ValueListenerFixture aElement = aArray.element;
+    assertEquals(JsonType.EMPTY, aElement.type);
+
+    // See elements, can revise estimate of element type
+    // {a: [1, 100]}
+    assertTrue(fixture.next());
+    assertEquals(2, aElement.valueCount);
+    assertEquals(100L, aElement.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void test2DArray() {
+    final String json =
+        "{a: [ [10, 1], [20, 2]]}\n" +
+        "{a: [[null]]} {a: [[]]} {a: [null]} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    // {a: [ [10, 1], [20, 2]]}
+    assertTrue(fixture.next());
+
+    // Value of a
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.INTEGER, a.type);
+    assertEquals(2, a.dimCount);
+
+    // Array for a[]
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture outerArr = a.arrayValue;
+    assertEquals(2, outerArr.dimCount);
+    assertEquals(1, outerArr.startCount);
+    assertEquals(outerArr.startCount, outerArr.endCount);
+
+    // Value of a[] elements
+    ValueListenerFixture outerElement = outerArr.element;
+    assertEquals(JsonType.INTEGER, outerElement.type);
+    assertEquals(1, outerElement.dimCount);
+    assertNotNull(outerElement.arrayValue);
+
+    // Array for a[][]
+    assertNotNull(outerElement.arrayValue);
+    ArrayListenerFixture innerArr = outerElement.arrayValue;
+    assertEquals(1, innerArr.dimCount);
+    assertEquals(2, innerArr.startCount);
+    assertEquals(innerArr.startCount, innerArr.endCount);
+
+    // Value of a[][] elements
+    ValueListenerFixture innerElement = innerArr.element;
+    assertEquals(JsonType.INTEGER, innerElement.type);
+    assertEquals(0, innerElement.dimCount);
+    assertEquals(4, innerElement.valueCount);
+    assertEquals(0, innerElement.nullCount);
+    assertEquals(2L, innerElement.value);
+
+    // {a: [[null]]}
+    assertTrue(fixture.next());
+    assertEquals(2, outerArr.startCount);
+    assertEquals(outerArr.startCount, outerArr.endCount);
+    assertEquals(0, outerElement.nullCount);
+    assertEquals(3, innerArr.startCount);
+    assertEquals(innerArr.startCount, innerArr.endCount);
+    assertEquals(4, innerElement.valueCount);
+    assertEquals(1, innerElement.nullCount);
+
+    // {a: [[]]}
+    assertTrue(fixture.next());
+    assertEquals(3, outerArr.startCount);
+    assertEquals(outerArr.startCount, outerArr.endCount);
+    assertEquals(0, outerElement.nullCount);
+    assertEquals(4, innerArr.startCount);
+    assertEquals(innerArr.startCount, innerArr.endCount);
+    assertEquals(4, innerElement.valueCount);
+    assertEquals(1, innerElement.nullCount);
+
+    // {a: [null]}
+    assertTrue(fixture.next());
+    assertEquals(0, a.nullCount);
+    assertEquals(4, outerArr.startCount);
+    assertEquals(outerArr.startCount, outerArr.endCount);
+    assertEquals(1, outerElement.nullCount);
+    assertEquals(4, innerArr.startCount);
+    assertEquals(4, innerElement.valueCount);
+    assertEquals(1, innerElement.nullCount);
+
+    // {a: null}
+    assertTrue(fixture.next());
+    assertEquals(1, a.nullCount);
+    assertEquals(4, outerArr.startCount);
+    assertEquals(outerArr.startCount, outerArr.endCount);
+    assertEquals(1, outerElement.nullCount);
+    assertEquals(4, innerArr.startCount);
+    assertEquals(4, innerElement.valueCount);
+    assertEquals(1, innerElement.nullCount);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void test1DEmptyTo2DArray() {
+    final String json =
+        "{a: []}\n" +
+        "{a: [ [10, 1], [20, 2]]}\n";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    // Check of details done in other tests. Just cut to
+    // the chase to verify proper structure.
+    assertEquals(2, fixture.read());
+    ValueListenerFixture element =
+        fixture.field("a").arrayValue.element.arrayValue.element;
+    assertEquals(4, element.valueCount);
+    assertEquals(2L, element.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testObjArray() {
+    final String json =
+        "{a: [ {b: \"fred\"}, {b: \"barney\"} ] }";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+
+    // Value of object.a
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.OBJECT, a.type);
+    assertEquals(1, a.dimCount);
+
+    // a[]
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture aArray = a.arrayValue;
+    assertEquals(1, aArray.startCount);
+    assertEquals(aArray.startCount, aArray.endCount);
+    assertEquals(1, aArray.dimCount);
+
+    // Value of each element of a[]
+    assertNotNull(aArray.element);
+    ValueListenerFixture aElement = aArray.element;
+    assertEquals(JsonType.OBJECT, aElement.type);
+    assertEquals(0, aElement.dimCount);
+    assertNull(aElement.arrayValue);
+    assertEquals(0, aElement.valueCount);
+    assertEquals(0, aElement.nullCount);
+
+    // Object for a[] elements
+    assertNotNull(aElement.objectValue);
+    ObjectListenerFixture elementObj = aElement.objectValue;
+    assertEquals(2, elementObj.startCount);
+    assertEquals(elementObj.startCount, elementObj.endCount);
+
+    // b field within a[]{}
+    ValueListenerFixture b = elementObj.field("b");
+    assertEquals(2, b.valueCount);
+    assertEquals("barney", b.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void test2DObjArray() {
+    final String json =
+        "{a: [ [ {b: \"fred\"}, {b: \"wilma\"} ],\n" +
+        "      [ {b: \"barney\"}, {b: \"betty\"} ] ] }";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+
+    // Value of object.a
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.OBJECT, a.type);
+    assertEquals(2, a.dimCount);
+
+    // a[]
+    assertNotNull(a.arrayValue);
+    ArrayListenerFixture outerArray = a.arrayValue;
+    assertEquals(1, outerArray.startCount);
+    assertEquals(outerArray.startCount, outerArray.endCount);
+    assertEquals(2, outerArray.dimCount);
+
+    // Value of each element of a[]
+    assertNotNull(outerArray.element);
+    ValueListenerFixture outerElement = outerArray.element;
+    assertEquals(JsonType.OBJECT, outerElement.type);
+    assertEquals(1, outerElement.dimCount);
+    assertEquals(0, outerElement.valueCount);
+    assertEquals(0, outerElement.nullCount);
+
+    // a[][]
+    assertNotNull(outerElement.arrayValue);
+    ArrayListenerFixture innerArray = outerElement.arrayValue;
+    assertEquals(2, innerArray.startCount);
+    assertEquals(innerArray.startCount, innerArray.endCount);
+    assertEquals(1, innerArray.dimCount);
+
+    // Value of each element of a[][]
+    assertNotNull(innerArray.element);
+    ValueListenerFixture innerElement = innerArray.element;
+    assertEquals(JsonType.OBJECT, innerElement.type);
+    assertEquals(0, innerElement.dimCount);
+    assertEquals(0, innerElement.valueCount);
+    assertEquals(0, innerElement.nullCount);
+
+    // Object for a[][] elements
+    assertNotNull(innerElement.objectValue);
+    ObjectListenerFixture elementObj = innerElement.objectValue;
+    assertEquals(4, elementObj.startCount);
+    assertEquals(elementObj.startCount, elementObj.endCount);
+
+    // b field within a[][]{}
+    ValueListenerFixture b = elementObj.field("b");
+    assertEquals(4, b.valueCount);
+    assertEquals("betty", b.value);
+
+    assertFalse(fixture.next());
+  }
+
+  /**
+   * JSON allows any combination of value types.
+   */
+  @Test
+  public void testMixArray() {
+    final String json =
+        "{a: [10, 11] }\n" +
+        "{a: {b: \"fred\"}}\n" +
+        "{a: 20.5}\n" +
+        "{a: null}\n";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertEquals(4, fixture.read());
+    ValueListenerFixture a = fixture.field("a");
+    // Type first seen
+    assertEquals(JsonType.INTEGER, a.type);
+    assertEquals(1, a.dimCount);
+
+    // Everything populated
+
+    assertEquals(2, a.arrayValue.element.valueCount);
+    assertEquals(11L, a.arrayValue.element.value);
+    assertEquals(1, a.objectValue.startCount);
+    assertEquals("fred", a.objectValue.field("b").value);
+    assertEquals(20.5D, a.value);
+    assertEquals(1, a.nullCount);
+
+    fixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserBasics.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserBasics.java
new file mode 100644
index 0000000..4215ff5
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserBasics.java
@@ -0,0 +1,279 @@
+/*
+ * 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.json.parser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonType;
+import org.junit.Test;
+
+/**
+ * Tests JSON structure parser functionality excluding nested objects
+ * and arrays. Tests the "happy path."
+ */
+public class TestJsonParserBasics extends BaseTestJsonParser {
+
+  @Test
+  public void testEmpty() {
+    String json = "";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertFalse(fixture.next());
+    assertEquals(0, fixture.rootObject.startCount);
+    fixture.close();
+  }
+
+  @Test
+  public void testEmptyTuple() {
+    final String json = "{} {} {}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+    assertEquals(3, fixture.rootObject.startCount);
+    assertEquals(3, fixture.rootObject.endCount);
+    assertTrue(fixture.rootObject.fields.isEmpty());
+    fixture.close();
+  }
+
+  @Test
+  public void testBoolean() {
+    final String json = "{a: true} {a: false} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertTrue(fixture.next());
+    assertEquals(1, fixture.rootObject.startCount);
+    assertEquals(1, fixture.rootObject.fields.size());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.BOOLEAN, a.type);
+    assertEquals(0, a.dimCount);
+    assertEquals(0, a.nullCount);
+    assertEquals(Boolean.TRUE, a.value);
+    assertEquals(2, fixture.read());
+    assertEquals(1, a.nullCount);
+    assertEquals(Boolean.FALSE, a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testInteger() {
+    final String json = "{a: 0} {a: 100} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.INTEGER, a.type);
+    assertEquals(2, fixture.read());
+    assertEquals(1, a.nullCount);
+    assertEquals(100L, a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testFloat() {
+    final String json = "{a: 0.0} {a: 100.5} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.FLOAT, a.type);
+    assertEquals(2, fixture.read());
+    assertEquals(1, a.nullCount);
+    assertEquals(2, a.valueCount);
+    assertEquals(100.5D, a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testExtendedFloat() {
+    final String json =
+        "{a: NaN} {a: Infinity} {a: -Infinity}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.options.allowNanInf = true;
+    fixture.open(json);
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.FLOAT, a.type);
+    assertEquals(2, fixture.read());
+    assertEquals(3, a.valueCount);
+    assertEquals(Double.NEGATIVE_INFINITY, a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testString() {
+    final String json = "{a: \"\"} {a: \"hi\"} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.STRING, a.type);
+    assertEquals(2, fixture.read());
+    assertEquals(1, a.nullCount);
+    assertEquals(2, a.valueCount);
+    assertEquals("hi", a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testMixedTypes() {
+    final String json = "{a: \"hi\"} {a: 10} {a: 10.5}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(JsonType.STRING, a.type);
+    assertEquals("hi", a.value);
+    assertTrue(fixture.next());
+    assertEquals(10L, a.value);
+    assertTrue(fixture.next());
+    assertEquals(10.5D, a.value);
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testRootTuple() {
+    final String json =
+      "{id: 1, name: \"Fred\", balance: 100.0}\n" +
+      "{id: 2, name: \"Barney\"}\n" +
+      "{id: 3, name: \"Wilma\", balance: 500.00}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+
+    ValueListenerFixture name = fixture.field("name");
+    assertEquals(3, name.valueCount);
+    assertEquals("Wilma", name.value);
+    ValueListenerFixture balance = fixture.field("balance");
+    assertEquals(2, balance.valueCount);
+    assertEquals(500.00D, balance.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testRootArray() {
+    final String json = "[{a: 0}, {a: 100}, {a: null}]";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(2, a.valueCount);
+    assertEquals(1, a.nullCount);
+    fixture.close();
+  }
+
+  @Test
+  public void testLeadingTrailingWhitespace() {
+    final String json = "{\" a\": 10, \" b\": 20, \" c \": 30}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(1, fixture.read());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(10L, a.value);
+    ValueListenerFixture b = fixture.field("b");
+    assertEquals(20L, b.value);
+    ValueListenerFixture c = fixture.field("c");
+    assertEquals(30L, c.value);
+    fixture.close();
+  }
+
+  /**
+   * Verify that names are case insensitive, first name determine's
+   * Drill's column name.
+   */
+  @Test
+  public void testCaseInsensitive() {
+    final String json = "{a: 10} {A: 20} {\" a \": 30}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(3, a.valueCount);
+    assertEquals(30L, a.value);
+    fixture.close();
+  }
+
+  /**
+   * Verify that the first name wins when determining case.
+   */
+  @Test
+  public void testMixedCase() {
+    final String json = "{Bob: 10} {bOb: 20} {BoB: 30}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+    ValueListenerFixture bob = fixture.field("Bob");
+    assertEquals(3, bob.valueCount);
+    assertEquals(30L, bob.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testProjection() {
+    final String json =
+        "{a: 1, b: [[{x: [[{y: []}]]}]]}\n" +
+        "{a: 2}\n" +
+        "{b: \"bar\"}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.rootObject.projectFilter = new HashSet<>();
+    fixture.rootObject.projectFilter.add("a");
+    fixture.open(json);
+
+    assertEquals(3, fixture.read());
+    assertEquals(1, fixture.rootObject.fields.size());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(2, a.valueCount);
+    assertEquals(2L, a.value);
+    fixture.close();
+  }
+
+  @Test
+  public void testAllTextMode() {
+    final String json =
+        "{a: 1} {a: \"foo\"} {a: true} {a: 20.5} {a: null}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.options.allTextMode = true;
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals("1", a.value);
+
+    assertTrue(fixture.next());
+    assertEquals("foo", a.value);
+
+    assertTrue(fixture.next());
+    assertEquals("true", a.value);
+
+    assertTrue(fixture.next());
+    assertEquals("20.5", a.value);
+    assertEquals(0, a.nullCount);
+
+    assertTrue(fixture.next());
+    assertEquals("20.5", a.value);
+    assertEquals(1, a.nullCount);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserErrors.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserErrors.java
new file mode 100644
index 0000000..1b99ca2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserErrors.java
@@ -0,0 +1,94 @@
+/*
+ * 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.json.parser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.junit.Test;
+
+/**
+ * Tests the un-happy path cases in the JSON structure parser. Some
+ * error cases can't occur because the Jackson parser catches them
+ * first.
+ */
+public class TestJsonParserErrors extends BaseTestJsonParser {
+
+  @Test
+  public void testMissingEndObject() {
+    expectError("{a: 0} {a: 100", "syntaxError");
+  }
+
+  @Test
+  public void testMissingValue() {
+    expectError("{a: 0} {a: ", "syntaxError");
+  }
+
+  /**
+   * 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}", "syntaxError");
+  }
+
+  @Test
+  public void testEmptyKey() {
+    expectError("{\"\": 10}", "structureError");
+  }
+
+  @Test
+  public void testBlankKey() {
+    expectError("{\"  \": 10}", "structureError");
+  }
+
+  @Test
+  public void testRootArrayDisallowed() {
+    final String json = "[{a: 0}, {a: 100}, {a: null}]";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.options.skipOuterList = false;
+    try {
+      fixture.open(json);
+      fail();
+    } catch (JsonErrorFixture e) {
+      assertEquals("structureError", e.errorType);
+      assertTrue(e.getMessage().contains("includes an outer array"));
+    }
+    fixture.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}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.options.skipMalformedRecords = true;
+    fixture.open(json);
+    assertEquals(3, fixture.read());
+    assertEquals(1, fixture.parser.recoverableErrorCount());
+    ValueListenerFixture a = fixture.field("a");
+    assertEquals(3, a.valueCount);
+    fixture.close();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserObjects.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserObjects.java
new file mode 100644
index 0000000..9a350b6
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/parser/TestJsonParserObjects.java
@@ -0,0 +1,158 @@
+/*
+ * 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.json.parser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.exec.store.easy.json.parser.JsonType;
+import org.junit.Test;
+
+/**
+ * Tests nested object support in the JSON structure parser.
+ */
+public class TestJsonParserObjects extends BaseTestJsonParser {
+
+  @Test
+  public void testNestedTuple() {
+    final String json =
+        "{id: 1, customer: { name: \"fred\" }}\n" +
+        "{id: 2, customer: { name: \"barney\" }}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+    assertEquals(1, fixture.rootObject.startCount);
+    assertEquals(fixture.rootObject.startCount, fixture.rootObject.endCount);
+    ValueListenerFixture cust = fixture.field("customer");
+    assertNotNull(cust.objectValue);
+    ObjectListenerFixture custObj = cust.objectValue;
+    assertEquals(1, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+    ValueListenerFixture name = custObj.field("name");
+    assertEquals(JsonType.STRING, name.type);
+    assertEquals("fred", name.value);
+
+    assertTrue(fixture.next());
+    assertEquals(2, fixture.rootObject.startCount);
+    assertEquals(fixture.rootObject.startCount, fixture.rootObject.endCount);
+    assertEquals("barney", name.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testObjectToNull() {
+    final String json =
+        "{id: 1, customer: {name: \"fred\"}}\n" +
+        "{id: 2, customer: null}\n" +
+        "{id: 3}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+    ValueListenerFixture cust = fixture.field("customer");
+    assertEquals(0, cust.valueCount);
+    assertEquals(0, cust.nullCount);
+    ObjectListenerFixture custObj = cust.objectValue;
+    assertEquals(1, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+    ValueListenerFixture name = custObj.field("name");
+    assertEquals("fred", name.value);
+
+    assertTrue(fixture.next());
+    assertEquals(1, cust.nullCount);
+    assertEquals(1, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+
+    assertTrue(fixture.next());
+    assertEquals(1, cust.nullCount);
+    assertEquals(1, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testNullToObject() {
+    final String json =
+        "{id: 1}\n" +
+        "{id: 2, customer: null}\n" +
+        "{id: 3, customer: {name: \"fred\"}}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+    assertEquals(1, fixture.rootObject.fields.size());
+
+    assertTrue(fixture.next());
+    ValueListenerFixture cust = fixture.field("customer");
+    assertEquals(0, cust.valueCount);
+    assertEquals(1, cust.nullCount);
+    assertNull(cust.objectValue);
+
+    assertTrue(fixture.next());
+    assertNotNull(cust.objectValue);
+    ObjectListenerFixture custObj = cust.objectValue;
+    assertEquals(1, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+    ValueListenerFixture name = custObj.field("name");
+    assertEquals("fred", name.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+
+  @Test
+  public void testMixedObject() {
+    final String json =
+        "{id: 1, customer: null}\n" +
+        "{id: 2, customer: {name: \"fred\"}}\n" +
+        "{id: 3, customer: 123}\n" +
+        "{id: 4, customer: {name: \"barney\"}}";
+    JsonParserFixture fixture = new JsonParserFixture();
+    fixture.open(json);
+
+    assertTrue(fixture.next());
+
+    assertTrue(fixture.next());
+    ValueListenerFixture cust = fixture.field("customer");
+    assertNotNull(cust.objectValue);
+    ObjectListenerFixture custObj = cust.objectValue;
+    ValueListenerFixture name = custObj.field("name");
+    assertEquals("fred", name.value);
+
+    assertTrue(fixture.next());
+    assertEquals(1, cust.valueCount);
+    assertEquals(123L, cust.value);
+
+    assertTrue(fixture.next());
+    assertNotNull(cust.objectValue);
+    assertEquals(2, custObj.startCount);
+    assertEquals(custObj.startCount, custObj.endCount);
+    assertEquals("barney", name.value);
+
+    assertFalse(fixture.next());
+    fixture.close();
+  }
+}