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:29 UTC

[drill] branch MERGE-200221-00 created (now 8a8e58b)

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

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


      at 8a8e58b  DRILL-7514: Update Apache POI to Latest Version

This branch includes the following new commits:

     new 7517176  DRILL-7594: Remove unused DrillStoreRel
     new a81c496  DRILL-7589: Set temporary tests folder for UDF_DIRECTORY_LOCAL, fix allocators closing in BloomFilterTest and TestWriteToDisk, fix permissions issue for TestGracefulShutdown tests
     new e8f9b7e  DRILL-7586: Fix loading incorrect version of commons-lang3
     new 5015d0e  DRILL-7583: Remove STOP status from operator outcome
     new b0ab3a6  DRILL-7574: Generalize the projection parser
     new 39457bf  DRILL-7573: Support htpasswd based authentication
     new 8e6ba48  DRILL-7572: JSON structure parser
     new 7951e4c  DRILL-7565: ANALYZE TABLE ... REFRESH METADATA does not work for empty Parquet files
     new c529791  DRILL-7562: Support HTTP Basic authentication for REST API calls
     new 8a8e58b  DRILL-7514: Update Apache POI to Latest Version

The 10 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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

Posted by ag...@apache.org.
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();
+  }
+}


[drill] 09/10: DRILL-7562: Support HTTP Basic authentication for REST API calls

Posted by ag...@apache.org.
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 c5297911de21ee874e785492a967c6f875090e20
Author: Dobes Vandermeer <do...@gmail.com>
AuthorDate: Fri Feb 7 16:47:27 2020 -0800

    DRILL-7562: Support HTTP Basic authentication for REST API calls
    
    This can greatly simplify the development of HTTP clients, as well as
    use from the command line using curl/wget, since you don't have to
    deal with storing the session cookie.
    
    closes #1972
---
 .../src/main/resources/drill-override-example.conf |  4 +--
 .../auth/DrillHttpConstraintSecurityHandler.java   | 40 ++++++++++++++--------
 .../auth/DrillHttpSecurityHandlerProvider.java     |  9 +++++
 .../exec/server/rest/auth/FormSecurityHandler.java |  9 ++---
 ...dler.java => HttpBasicAuthSecurityHandler.java} | 20 +++++------
 5 files changed, 46 insertions(+), 36 deletions(-)

diff --git a/distribution/src/main/resources/drill-override-example.conf b/distribution/src/main/resources/drill-override-example.conf
index e72396c..0c04175 100644
--- a/distribution/src/main/resources/drill-override-example.conf
+++ b/distribution/src/main/resources/drill-override-example.conf
@@ -103,8 +103,8 @@ drill.exec: {
     },
     auth: {
         # Http Auth mechanisms to configure. If not provided but user.auth is enabled
-        # then default value is FORM.
-        mechanisms: ["FORM", "SPNEGO"],
+        # then default value is ["FORM"].
+        mechanisms: ["BASIC", "FORM", "SPNEGO"],
         # Spnego principal to be used by WebServer when Spnego authentication is enabled.
         spnego.principal: "HTTP://<localhost>"
         # Location to keytab file for above spnego principal
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
index 8ba6ddf..99ca966 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpConstraintSecurityHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.server.rest.auth;
 
+import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.drill.common.exceptions.DrillException;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -38,24 +39,33 @@ import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.AUTHENTI
  **/
 public abstract class DrillHttpConstraintSecurityHandler extends ConstraintSecurityHandler {
 
-    @Override
-    public void doStart() throws Exception {
-        super.doStart();
-    }
+  @Override
+  public void doStart() throws Exception {
+    super.doStart();
+  }
 
-    @Override
-    public void doStop() throws Exception {
-        super.doStop();
-    }
+  @Override
+  public void doStop() throws Exception {
+    super.doStop();
+  }
+
+  public abstract void doSetup(DrillbitContext dbContext) throws DrillException;
 
-    public abstract void doSetup(DrillbitContext dbContext) throws DrillException;
+  public void setup(LoginAuthenticator authenticator, LoginService loginService) {
+    final Set<String> knownRoles = ImmutableSet.of(AUTHENTICATED_ROLE, ADMIN_ROLE);
+    setConstraintMappings(Collections.<ConstraintMapping>emptyList(), knownRoles);
+    setAuthenticator(authenticator);
+    setLoginService(loginService);
+  }
 
-    public void setup(LoginAuthenticator authenticator, LoginService loginService) {
-      final Set<String> knownRoles = ImmutableSet.of(AUTHENTICATED_ROLE, ADMIN_ROLE);
-      setConstraintMappings(Collections.<ConstraintMapping>emptyList(), knownRoles);
-      setAuthenticator(authenticator);
-      setLoginService(loginService);
+  protected void requireAuthProvider(DrillbitContext dbContext, String name) throws DrillException {
+    // Check if PAMAuthenticator is available or not which is required for FORM authentication
+    if (!dbContext.getAuthProvider().containsFactory(PlainFactory.SIMPLE_NAME)) {
+      throw new DrillException(String.format("%1$s auth mechanism was configured but %2$s mechanism is not enabled to provide an " +
+        "authenticator. Please configure user authentication with %2$s mechanism and authenticator to use " +
+        "%1$s authentication", getImplName(), name));
     }
+  }
 
-    public abstract String getImplName();
+  public abstract String getImplName();
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
index e95a057..e3087ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/DrillHttpSecurityHandlerProvider.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.rpc.security.AuthStringUtil;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.rest.WebServerConstants;
+import org.eclipse.jetty.http.HttpHeader;
 import org.eclipse.jetty.security.ConstraintSecurityHandler;
 import org.eclipse.jetty.security.authentication.SessionAuthentication;
 import org.eclipse.jetty.server.Handler;
@@ -139,10 +140,14 @@ public class DrillHttpSecurityHandlerProvider extends ConstraintSecurityHandler
       if (isSpnegoEnabled() && (!isFormEnabled() || uri.equals(WebServerConstants.SPENGO_LOGIN_RESOURCE_PATH))) {
         securityHandler = securityHandlers.get(Constraint.__SPNEGO_AUTH);
         securityHandler.handle(target, baseRequest, request, response);
+      } else if(isBasicEnabled() && request.getHeader(HttpHeader.AUTHORIZATION.asString()) != null) {
+        securityHandler = securityHandlers.get(Constraint.__BASIC_AUTH);
+        securityHandler.handle(target, baseRequest, request, response);
       } else if (isFormEnabled()) {
         securityHandler = securityHandlers.get(Constraint.__FORM_AUTH);
         securityHandler.handle(target, baseRequest, request, response);
       }
+
     }
     // If user has logged in, use the corresponding handler to handle the request
     else {
@@ -175,6 +180,10 @@ public class DrillHttpSecurityHandlerProvider extends ConstraintSecurityHandler
     return securityHandlers.containsKey(Constraint.__FORM_AUTH);
   }
 
+  public boolean isBasicEnabled() {
+    return securityHandlers.containsKey(Constraint.__BASIC_AUTH);
+  }
+
   /**
    * Return's list of configured mechanisms for HTTP authentication. For backward
    * compatibility if authentication is enabled it will include FORM mechanism by default.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java
index 2c19d41..8169a40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java
@@ -33,15 +33,10 @@ public class FormSecurityHandler extends DrillHttpConstraintSecurityHandler {
   @Override
   public void doSetup(DrillbitContext dbContext) throws DrillException {
 
-    // Check if PAMAuthenticator is available or not which is required for FORM authentication
-    if (!dbContext.getAuthProvider().containsFactory(PlainFactory.SIMPLE_NAME)) {
-      throw new DrillException("FORM mechanism was configured but PLAIN mechanism is not enabled to provide an " +
-          "authenticator. Please configure user authentication with PLAIN mechanism and authenticator to use " +
-          "FORM authentication");
-    }
+    requireAuthProvider(dbContext, PlainFactory.SIMPLE_NAME);
 
     setup(new FormAuthenticator(WebServerConstants.FORM_LOGIN_RESOURCE_PATH,
-        WebServerConstants.FORM_LOGIN_RESOURCE_PATH, true), new DrillRestLoginService(dbContext));
+      WebServerConstants.FORM_LOGIN_RESOURCE_PATH, true), new DrillRestLoginService(dbContext));
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/HttpBasicAuthSecurityHandler.java
similarity index 58%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/HttpBasicAuthSecurityHandler.java
index 2c19d41..2657186 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/FormSecurityHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/auth/HttpBasicAuthSecurityHandler.java
@@ -20,28 +20,24 @@ package org.apache.drill.exec.server.rest.auth;
 import org.apache.drill.common.exceptions.DrillException;
 import org.apache.drill.exec.rpc.security.plain.PlainFactory;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.rest.WebServerConstants;
-import org.eclipse.jetty.security.authentication.FormAuthenticator;
+import org.eclipse.jetty.security.authentication.BasicAuthenticator;
 import org.eclipse.jetty.util.security.Constraint;
 
-public class FormSecurityHandler extends DrillHttpConstraintSecurityHandler {
+/**
+ * Implement HTTP Basic authentication for REST API access
+ */
+public class HttpBasicAuthSecurityHandler extends DrillHttpConstraintSecurityHandler {
   @Override
   public String getImplName() {
-    return Constraint.__FORM_AUTH;
+    return Constraint.__BASIC_AUTH;
   }
 
   @Override
   public void doSetup(DrillbitContext dbContext) throws DrillException {
 
-    // Check if PAMAuthenticator is available or not which is required for FORM authentication
-    if (!dbContext.getAuthProvider().containsFactory(PlainFactory.SIMPLE_NAME)) {
-      throw new DrillException("FORM mechanism was configured but PLAIN mechanism is not enabled to provide an " +
-          "authenticator. Please configure user authentication with PLAIN mechanism and authenticator to use " +
-          "FORM authentication");
-    }
+    requireAuthProvider(dbContext, PlainFactory.SIMPLE_NAME);
 
-    setup(new FormAuthenticator(WebServerConstants.FORM_LOGIN_RESOURCE_PATH,
-        WebServerConstants.FORM_LOGIN_RESOURCE_PATH, true), new DrillRestLoginService(dbContext));
+    setup(new BasicAuthenticator(), new DrillRestLoginService(dbContext));
   }
 
 }


[drill] 06/10: DRILL-7573: Support htpasswd based authentication

Posted by ag...@apache.org.
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 39457bf8008e73cca8c9de8a677d5f29717b6d31
Author: Dobes Vandermeer <do...@gmail.com>
AuthorDate: Mon Feb 10 21:51:54 2020 -0800

    DRILL-7573: Support htpasswd based authentication
    
    In containerized environments, PAM based authentication is not convenient.
    
    This provides a simple mechanism for setting up users' passwords
    that can be managed using docker volume mounts.
    
    closes #1977
---
 .../java/org/apache/drill/exec/ExecConstants.java  |   1 +
 .../security/HtpasswdFileUserAuthenticator.java    | 152 +++++++++++++++++++++
 .../TestHtpasswdFileUserAuthenticator.java         | 151 ++++++++++++++++++++
 3 files changed, 304 insertions(+)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index adb3a4d..5938ca1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -245,6 +245,7 @@ public final class ExecConstants {
   public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
   public static final String USER_AUTHENTICATION_ENABLED = "drill.exec.security.user.auth.enabled";
   public static final String USER_AUTHENTICATOR_IMPL = "drill.exec.security.user.auth.impl";
+  public static final String HTPASSWD_AUTHENTICATOR_PATH = "drill.exec.security.user.auth.htpasswd.path";
   public static final String PAM_AUTHENTICATOR_PROFILES = "drill.exec.security.user.auth.pam_profiles";
   public static final String BIT_AUTHENTICATION_ENABLED = "drill.exec.security.bit.auth.enabled";
   public static final String BIT_AUTHENTICATION_MECHANISM = "drill.exec.security.bit.auth.mechanism";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.java
new file mode 100644
index 0000000..635e8a6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.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.rpc.user.security;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.Md5Crypt;
+import org.apache.commons.io.Charsets;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.text.MessageFormat;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Implementation of UserAuthenticator that reads passwords from an htpasswd
+ * formatted file.
+ * <p>
+ * Currently supports MD5, SHA-1, and plaintext passwords.
+ * <p>
+ * Use the htpasswd command line tool to create and modify htpasswd files.
+ * <p>
+ * By default this loads the passwords from <code>/opt/drill/conf/htpasswd</code>.  Users can change the path by
+ * putting the absolute file path as <code>drill.exec.security.user.auth.htpasswd.path</code> in
+ * <code>drill-override.conf</code>.
+ * <p>
+ * This is intended for situations where the list of users is relatively static, and you are running
+ * drill in a container so using pam is not convenient.
+ */
+@UserAuthenticatorTemplate(type = "htpasswd")
+public class HtpasswdFileUserAuthenticator implements UserAuthenticator {
+  private static final Logger logger = LoggerFactory.getLogger(HtpasswdFileUserAuthenticator.class);
+  private static final Pattern HTPASSWD_LINE_PATTERN = Pattern.compile("^([^:]+):([^:]+)");
+  public static final String DEFAULT_HTPASSWD_AUTHENTICATOR_PATH = "/opt/drill/conf/htpasswd";
+
+  private String path = DEFAULT_HTPASSWD_AUTHENTICATOR_PATH;
+  private long lastModified;
+  private long lastFileSize;
+  private Map<String, String> userToPassword;
+
+  @Override
+  public void setup(DrillConfig drillConfig) throws DrillbitStartupException {
+    if (drillConfig.hasPath(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH)) {
+      path = drillConfig.getString(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH);
+    }
+  }
+
+  /**
+   * Check password against hash read from the file
+   *
+   * @param password User provided password
+   * @param hash     Hash stored in the htpasswd file
+   * @return true if the password matched the hash
+   */
+  public static boolean isPasswordValid(String password, String hash) {
+    if (hash.startsWith("$apr1$")) {
+      return hash.equals(Md5Crypt.apr1Crypt(password, hash));
+    } else if (hash.startsWith("$1$")) {
+      return hash.equals(Md5Crypt.md5Crypt(password.getBytes(Charsets.UTF_8), hash));
+    } else if (hash.startsWith("{SHA}")) {
+      return hash.substring(5).equals(Base64.getEncoder().encodeToString(DigestUtils.sha1(password)));
+    } else if (hash.startsWith("$2y$")) {
+      // bcrypt not supported currently
+      return false;
+    } else {
+      return hash.equals(password);
+    }
+  }
+
+  /**
+   * Validate the given username and password against the password file
+   *
+   * @param username Username provided
+   * @param password Password provided
+   * @throws UserAuthenticationException If the username and password could not be validated
+   */
+  @Override
+  public void authenticate(String username, String password) throws UserAuthenticationException {
+    read();
+    String hash = this.userToPassword.get(username);
+    boolean credentialsAccepted = (hash != null && isPasswordValid(password, hash));
+    if (!credentialsAccepted) {
+      throw new UserAuthenticationException(String.format("htpasswd auth failed for user '%s'",
+        username));
+    }
+  }
+
+  /**
+   * Read the password file into the map, if the file has changed since we last read it
+   */
+  protected synchronized void read() {
+    File file = new File(path);
+    long newLastModified = file.exists() ? file.lastModified() : 0;
+    long newFileSize = file.exists() ? file.length() : 0;
+    if (userToPassword == null || newLastModified != lastModified || newFileSize != lastFileSize) {
+      HashMap<String, String> newMap = new HashMap<>();
+      if(newFileSize != 0) {
+        try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
+          String line;
+          while ((line = reader.readLine()) != null) {
+            if (!line.isEmpty() && !line.startsWith("#")) {
+              Matcher m = HTPASSWD_LINE_PATTERN.matcher(line);
+              if (m.matches()) {
+                newMap.put(m.group(1), m.group(2));
+              }
+            }
+          }
+        } catch (Exception e) {
+          logger.error(MessageFormat.format("Failed to read htpasswd file at path {0}", file), e);
+        }
+      } else {
+        logger.error(MessageFormat.format("Empty or missing htpasswd file at path {0}", file));
+      }
+      lastFileSize = newFileSize;
+      lastModified = newLastModified;
+      userToPassword = newMap;
+    }
+  }
+
+  /**
+   * Free resources associated with this authenticator
+   */
+  @Override
+  public void close() {
+    lastModified = 0;
+    userToPassword = null;
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.java
new file mode 100644
index 0000000..9c27408
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.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.rpc.user.security;
+
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestHtpasswdFileUserAuthenticator extends ClusterTest {
+  private File tempPasswdFile;
+
+
+  private void setupCluster(String passwdContent) throws IOException {
+    tempPasswdFile = new File(dirTestWatcher.getTmpDir(), "htpasswd." + System.currentTimeMillis());
+    Files.write(tempPasswdFile.toPath(), passwdContent.getBytes());
+
+    cluster = ClusterFixture.bareBuilder(dirTestWatcher)
+      .clusterSize(3)
+      .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, "htpasswd")
+      .configProperty(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH, tempPasswdFile.toString())
+      .build();
+  }
+
+
+  @Test
+  public void passwordChecksGiveCorrectResults() throws Exception {
+    String passwdContent = "alice:pass1\n" +
+      "bob:buzzkill\n" +
+      "jane:$apr1$PrwDfXy9$ajkhotQW6RFnoVQtPKoW4/\n" +
+      "john:$apr1$UxZgBU8k$K4UzdubNa741TnWAZY2QV0\n";
+    setupCluster(passwdContent);
+
+
+    assertTrue(true);
+
+    tryCredentials("alice", "pass1", cluster, true);
+    tryCredentials("bob", "buzzkill", cluster, true);
+    tryCredentials("notalice", "pass1", cluster, false);
+    tryCredentials("notbob", "buzzkill", cluster, false);
+    tryCredentials("alice", "wrong", cluster, false);
+    tryCredentials("bob", "incorrect", cluster, false);
+    tryCredentials("jane", "pass", cluster, true);
+    tryCredentials("john", "foobar", cluster, true);
+    tryCredentials("jane", "wrong", cluster, false);
+    tryCredentials("john", "incorrect1", cluster, false);
+  }
+
+  @Test
+  public void rejectsLoginsWhenHtpasswdFileMissing() throws Exception {
+    cluster = ClusterFixture.bareBuilder(dirTestWatcher)
+      .clusterSize(3)
+      .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, "htpasswd")
+      .configProperty(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH, "/nonexistant-file")
+      .build();
+    tryCredentials("bob", "bob", cluster, false);
+  }
+
+  @Test
+  public void detectsChanges() throws Exception {
+    String passwdContent = "alice:pass1\nbob:buzzkill\n";
+    setupCluster(passwdContent);
+
+    tryCredentials("alice", "pass1", cluster, true);
+    tryCredentials("alice", "pass2", cluster, false);
+    tryCredentials("bob", "buzzkill", cluster, true);
+    tryCredentials("bob", "yolo", cluster, false);
+
+    String passwdContent2 = "alice:pass2\nbob:yolo\n";
+    Files.write(tempPasswdFile.toPath(), passwdContent2.getBytes());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, true);
+    tryCredentials("bob", "buzzkill", cluster, false);
+    tryCredentials("bob", "yolo", cluster, true);
+
+    // Invalid file is treated as empty
+    String passwdContent3 = "invalid file";
+    Files.write(tempPasswdFile.toPath(), passwdContent3.getBytes());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, false);
+
+    // Missing file is treated as empty
+    Files.delete(tempPasswdFile.toPath());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, false);
+
+  }
+
+  private static void tryCredentials(String user, String password, ClusterFixture cluster, boolean shouldSucceed) throws Exception {
+    try {
+      ClientFixture client = cluster.clientBuilder()
+        .property(DrillProperties.USER, user)
+        .property(DrillProperties.PASSWORD, password)
+        .build();
+
+      // Run few queries using the new client
+      List<String> queries = Arrays.asList(
+        "SHOW SCHEMAS",
+        "USE INFORMATION_SCHEMA",
+        "SHOW TABLES",
+        "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_NAME LIKE 'COLUMNS'",
+        "SELECT * FROM cp.`region.json` LIMIT 5");
+
+      for (String query : queries) {
+        client.queryBuilder().sql(query).run();
+      }
+
+      if (!shouldSucceed) {
+        fail("Expected connect to fail because of incorrect username / password combination, but it succeeded");
+      }
+    } catch (IllegalStateException e) {
+      if (shouldSucceed) {
+        throw e;
+      }
+    }
+  }
+
+}


[drill] 08/10: DRILL-7565: ANALYZE TABLE ... REFRESH METADATA does not work for empty Parquet files

Posted by ag...@apache.org.
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 7951e4ca6ec2948df5abdf23a3206be9f53f0c25
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Fri Feb 14 17:51:52 2020 +0200

    DRILL-7565: ANALYZE TABLE ... REFRESH METADATA does not work for empty Parquet files
    
    - Fixed ConvertMetadataAggregateToDirectScanRule rule to distinguish array columns correctly and proceed using other parquet metadata if such columns are found.
    - Added new implicit column which signalizes whether the empty result is obtained during collecting metadata and helps to distinguish real data results from metadata results.
    - Updated scan to return row with metadata if the above implicit column is present.
    - Added unit tests for checking the correctness of both optional and required columns from empty files.
    
    closes #1985
---
 .../java/org/apache/drill/exec/ExecConstants.java  |  21 +-
 .../drill/exec/metastore/ColumnNamesOptions.java   |  19 +-
 .../metastore/analyze/AnalyzeFileInfoProvider.java |   1 +
 .../analyze/MetadataAggregateContext.java          |  24 +-
 .../apache/drill/exec/physical/impl/ScanBatch.java |  16 ++
 .../impl/metadata/MetadataAggregateHelper.java     |  40 ++-
 .../impl/metadata/MetadataControllerBatch.java     |   2 +-
 .../ConvertMetadataAggregateToDirectScanRule.java  |  43 +--
 .../sql/handlers/MetastoreAnalyzeTableHandler.java |  16 +-
 .../exec/server/options/SystemOptionManager.java   |   1 +
 .../apache/drill/exec/store/ColumnExplorer.java    |   7 +-
 .../store/parquet/BaseParquetMetadataProvider.java |   2 +-
 .../java-exec/src/main/resources/drill-module.conf |   1 +
 .../drill/exec/sql/TestMetastoreCommands.java      | 310 +++++++++++++++++++++
 14 files changed, 436 insertions(+), 67 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 5938ca1..1276384 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -487,31 +487,36 @@ public final class ExecConstants {
   public static final OptionValidator IMPLICIT_SUFFIX_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_SUFFIX_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.10. Sets the implicit column name for the suffix column."));
   public static final String IMPLICIT_FQN_COLUMN_LABEL = "drill.exec.storage.implicit.fqn.column.label";
-  public static final OptionValidator IMPLICIT_FQN_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_FQN_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_FQN_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_FQN_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.10. Sets the implicit column name for the fqn column."));
   public static final String IMPLICIT_FILEPATH_COLUMN_LABEL = "drill.exec.storage.implicit.filepath.column.label";
-  public static final OptionValidator IMPLICIT_FILEPATH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_FILEPATH_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_FILEPATH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_FILEPATH_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.10. Sets the implicit column name for the filepath column."));
   public static final String IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_index.column.label";
-  public static final OptionValidator IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group index (rgi) column. " +
           "For internal usage when producing Metastore analyze."));
 
   public static final String IMPLICIT_ROW_GROUP_START_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_start.column.label";
-  public static final OptionValidator IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_START_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_START_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group start (rgs) column. " +
           "For internal usage when producing Metastore analyze."));
 
   public static final String IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL = "drill.exec.storage.implicit.row_group_length.column.label";
-  public static final OptionValidator IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the row group length (rgl) column. " +
           "For internal usage when producing Metastore analyze."));
 
   public static final String IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL = "drill.exec.storage.implicit.last_modified_time.column.label";
-  public static final OptionValidator IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL,
+  public static final StringValidator IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL,
       new OptionDescription("Available as of Drill 1.17. Sets the implicit column name for the lastModifiedTime column. " +
           "For internal usage when producing Metastore analyze."));
 
+  public static final String IMPLICIT_PROJECT_METADATA_COLUMN_LABEL = "drill.exec.storage.implicit.project_metadata.column.label";
+  public static final StringValidator IMPLICIT_PROJECT_METADATA_COLUMN_LABEL_VALIDATOR = new StringValidator(IMPLICIT_PROJECT_METADATA_COLUMN_LABEL,
+      new OptionDescription("Available as of Drill 1.18. Sets the implicit column name for the $project_metadata$ column. " +
+          "For internal usage when producing Metastore analyze."));
+
   public static final String JSON_READ_NUMBERS_AS_DOUBLE = "store.json.read_numbers_as_double";
   public static final BooleanValidator JSON_READ_NUMBERS_AS_DOUBLE_VALIDATOR = new BooleanValidator(JSON_READ_NUMBERS_AS_DOUBLE,
       new OptionDescription("Reads numbers with or without a decimal point as DOUBLE. Prevents schema change errors."));
@@ -1109,14 +1114,14 @@ public final class ExecConstants {
    */
   public static final String METASTORE_USE_SCHEMA_METADATA = "metastore.metadata.use_schema";
   public static final BooleanValidator METASTORE_USE_SCHEMA_METADATA_VALIDATOR = new BooleanValidator(METASTORE_USE_SCHEMA_METADATA,
-      new OptionDescription("Enables schema usage, stored to the Metastore. Default is false. (Drill 1.17+)"));
+      new OptionDescription("Enables schema usage, stored to the Metastore. Default is true. (Drill 1.17+)"));
 
   /**
    * Option for enabling statistics usage, stored in the Metastore, at the planning stage.
    */
   public static final String METASTORE_USE_STATISTICS_METADATA = "metastore.metadata.use_statistics";
   public static final BooleanValidator METASTORE_USE_STATISTICS_METADATA_VALIDATOR = new BooleanValidator(METASTORE_USE_STATISTICS_METADATA,
-      new OptionDescription("Enables statistics usage, stored in the Metastore, at the planning stage. Default is false. (Drill 1.17+)"));
+      new OptionDescription("Enables statistics usage, stored in the Metastore, at the planning stage. Default is true. (Drill 1.17+)"));
 
   /**
    * Option for collecting schema and / or column statistics for every table after CTAS and CTTAS execution.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/ColumnNamesOptions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/ColumnNamesOptions.java
index 0b9faca..7a23c12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/ColumnNamesOptions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/ColumnNamesOptions.java
@@ -32,14 +32,16 @@ public class ColumnNamesOptions {
   private final String rowGroupStart;
   private final String rowGroupLength;
   private final String lastModifiedTime;
+  private final String projectMetadataColumn;
 
   public ColumnNamesOptions(OptionManager optionManager) {
-    this.fullyQualifiedName = optionManager.getOption(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL).string_val;
-    this.partitionColumnNameLabel = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
-    this.rowGroupIndex = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL).string_val;
-    this.rowGroupStart = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL).string_val;
-    this.rowGroupLength = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL).string_val;
-    this.lastModifiedTime = optionManager.getOption(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL).string_val;
+    this.fullyQualifiedName = optionManager.getOption(ExecConstants.IMPLICIT_FQN_COLUMN_LABEL_VALIDATOR);
+    this.partitionColumnNameLabel = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR);
+    this.rowGroupIndex = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_INDEX_COLUMN_LABEL_VALIDATOR);
+    this.rowGroupStart = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR);
+    this.rowGroupLength = optionManager.getOption(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR);
+    this.lastModifiedTime = optionManager.getOption(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR);
+    this.projectMetadataColumn = optionManager.getOption(ExecConstants.IMPLICIT_PROJECT_METADATA_COLUMN_LABEL_VALIDATOR);
   }
 
   public String partitionColumnNameLabel() {
@@ -66,6 +68,10 @@ public class ColumnNamesOptions {
     return lastModifiedTime;
   }
 
+  public String projectMetadataColumn() {
+    return projectMetadataColumn;
+  }
+
   @Override
   public String toString() {
     return new StringJoiner(", ", ColumnNamesOptions.class.getSimpleName() + "[", "]")
@@ -75,6 +81,7 @@ public class ColumnNamesOptions {
         .add("rowGroupStart='" + rowGroupStart + "'")
         .add("rowGroupLength='" + rowGroupLength + "'")
         .add("lastModifiedTime='" + lastModifiedTime + "'")
+        .add("projectMetadataColumn='" + projectMetadataColumn + "'")
         .toString();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java
index a4bf0ad..1d2f2db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/AnalyzeFileInfoProvider.java
@@ -64,6 +64,7 @@ public abstract class AnalyzeFileInfoProvider implements AnalyzeInfoProvider {
     List<SchemaPath> projectionList = new ArrayList<>(getSegmentColumns(table, columnNamesOptions));
     projectionList.add(SchemaPath.getSimplePath(columnNamesOptions.fullyQualifiedName()));
     projectionList.add(SchemaPath.getSimplePath(columnNamesOptions.lastModifiedTime()));
+    projectionList.add(SchemaPath.getSimplePath(columnNamesOptions.projectMetadataColumn()));
     return Collections.unmodifiableList(projectionList);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java
index 9108345..99db025 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/analyze/MetadataAggregateContext.java
@@ -35,7 +35,11 @@ import java.util.StringJoiner;
 public class MetadataAggregateContext {
   private final List<NamedExpression> groupByExpressions;
   private final List<SchemaPath> interestingColumns;
-  private final List<SchemaPath> excludedColumns;
+
+  /**
+   * List of columns which do not belong to table schema, but used to pass some metadata information like file location, row group index, etc.
+   */
+  private final List<SchemaPath> metadataColumns;
   private final boolean createNewAggregations;
   private final MetadataType metadataLevel;
 
@@ -43,7 +47,7 @@ public class MetadataAggregateContext {
     this.groupByExpressions = builder.groupByExpressions;
     this.interestingColumns = builder.interestingColumns;
     this.createNewAggregations = builder.createNewAggregations;
-    this.excludedColumns = builder.excludedColumns;
+    this.metadataColumns = builder.metadataColumns;
     this.metadataLevel = builder.metadataLevel;
   }
 
@@ -63,8 +67,8 @@ public class MetadataAggregateContext {
   }
 
   @JsonProperty
-  public List<SchemaPath> excludedColumns() {
-    return excludedColumns;
+  public List<SchemaPath> metadataColumns() {
+    return metadataColumns;
   }
 
   @JsonProperty
@@ -78,7 +82,7 @@ public class MetadataAggregateContext {
         .add("groupByExpressions=" + groupByExpressions)
         .add("interestingColumns=" + interestingColumns)
         .add("createNewAggregations=" + createNewAggregations)
-        .add("excludedColumns=" + excludedColumns)
+        .add("excludedColumns=" + metadataColumns)
         .toString();
   }
 
@@ -91,7 +95,7 @@ public class MetadataAggregateContext {
         .groupByExpressions(groupByExpressions)
         .interestingColumns(interestingColumns)
         .createNewAggregations(createNewAggregations)
-        .excludedColumns(excludedColumns)
+        .metadataColumns(metadataColumns)
         .metadataLevel(metadataLevel);
   }
 
@@ -101,7 +105,7 @@ public class MetadataAggregateContext {
     private List<SchemaPath> interestingColumns;
     private Boolean createNewAggregations;
     private MetadataType metadataLevel;
-    private List<SchemaPath> excludedColumns;
+    private List<SchemaPath> metadataColumns;
 
     public MetadataAggregateContextBuilder groupByExpressions(List<NamedExpression> groupByExpressions) {
       this.groupByExpressions = groupByExpressions;
@@ -123,15 +127,15 @@ public class MetadataAggregateContext {
       return this;
     }
 
-    public MetadataAggregateContextBuilder excludedColumns(List<SchemaPath> excludedColumns) {
-      this.excludedColumns = excludedColumns;
+    public MetadataAggregateContextBuilder metadataColumns(List<SchemaPath> metadataColumns) {
+      this.metadataColumns = metadataColumns;
       return this;
     }
 
     public MetadataAggregateContext build() {
       Objects.requireNonNull(groupByExpressions, "groupByExpressions were not set");
       Objects.requireNonNull(createNewAggregations, "createNewAggregations was not set");
-      Objects.requireNonNull(excludedColumns, "excludedColumns were not set");
+      Objects.requireNonNull(metadataColumns, "metadataColumns were not set");
       Objects.requireNonNull(metadataLevel, "metadataLevel was not set");
       return new MetadataAggregateContext(this);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index e159ec6..d1d1c48 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -29,6 +29,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.TypeHelper;
@@ -234,6 +235,21 @@ public class ScanBatch implements CloseableRecordBatch {
       logger.trace("currentReader.next return recordCount={}", recordCount);
       Preconditions.checkArgument(recordCount >= 0, "recordCount from RecordReader.next() should not be negative");
       boolean isNewSchema = mutator.isNewSchema();
+      // If scan is done for collecting metadata, additional implicit column `$project_metadata$`
+      // will be projected to handle the case when scan may return empty results (scan on empty file or row group).
+      // Scan will return single row for the case when empty file or row group is present with correct
+      // values of other implicit columns (like `fqn`, `rgi`), so this metadata will be stored to the Metastore.
+      if (implicitValues != null) {
+        String projectMetadataColumn = context.getOptions().getOption(ExecConstants.IMPLICIT_PROJECT_METADATA_COLUMN_LABEL_VALIDATOR);
+        if (recordCount > 0) {
+          // Sets the implicit value to false to signal that some results were returned and there is no need for creating an additional record.
+          implicitValues.replace(projectMetadataColumn, Boolean.FALSE.toString());
+        } else if (Boolean.parseBoolean(implicitValues.get(projectMetadataColumn))) {
+          recordCount++;
+          // Sets implicit value to null to avoid affecting resulting count value.
+          implicitValues.put(projectMetadataColumn, null);
+        }
+      }
       populateImplicitVectors();
       mutator.container.setValueCount(recordCount);
       oContext.getStats().batchReceived(0, recordCount, isNewSchema);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
index 6f00dea..9122de1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
@@ -22,7 +22,9 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.IfExpression;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.NullExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.common.logical.data.NamedExpression;
@@ -55,6 +57,7 @@ public class MetadataAggregateHelper {
   private final ColumnNamesOptions columnNamesOptions;
   private final BatchSchema schema;
   private final AggPrelBase.OperatorPhase phase;
+  private final List<SchemaPath> excludedColumns;
 
   public MetadataAggregateHelper(MetadataAggregateContext context, ColumnNamesOptions columnNamesOptions,
       BatchSchema schema, AggPrelBase.OperatorPhase phase) {
@@ -63,6 +66,8 @@ public class MetadataAggregateHelper {
     this.schema = schema;
     this.phase = phase;
     this.valueExpressions = new ArrayList<>();
+    this.excludedColumns = new ArrayList<>(context.metadataColumns());
+    excludedColumns.add(SchemaPath.getSimplePath(columnNamesOptions.projectMetadataColumn()));
     createAggregatorInternal();
   }
 
@@ -71,8 +76,6 @@ public class MetadataAggregateHelper {
   }
 
   private void createAggregatorInternal() {
-    List<SchemaPath> excludedColumns = context.excludedColumns();
-
     // Iterates through input expressions and adds aggregate calls for table fields
     // to collect required statistics (MIN, MAX, COUNT, etc.) or aggregate calls to merge incoming metadata
     getUnflattenedFileds(Lists.newArrayList(schema), null)
@@ -117,16 +120,16 @@ public class MetadataAggregateHelper {
       }
     }
 
-    for (SchemaPath excludedColumn : excludedColumns) {
-      if (excludedColumn.equals(SchemaPath.getSimplePath(columnNamesOptions.rowGroupStart()))
-          || excludedColumn.equals(SchemaPath.getSimplePath(columnNamesOptions.rowGroupLength()))) {
-        LogicalExpression lastModifiedTime = new FunctionCall("any_value",
+    for (SchemaPath metadataColumns : context.metadataColumns()) {
+      if (metadataColumns.equals(SchemaPath.getSimplePath(columnNamesOptions.rowGroupStart()))
+          || metadataColumns.equals(SchemaPath.getSimplePath(columnNamesOptions.rowGroupLength()))) {
+        LogicalExpression anyValueCall = new FunctionCall("any_value",
             Collections.singletonList(
-                FieldReference.getWithQuotedRef(excludedColumn.getRootSegmentPath())),
+                FieldReference.getWithQuotedRef(metadataColumns.getRootSegmentPath())),
             ExpressionPosition.UNKNOWN);
 
-        valueExpressions.add(new NamedExpression(lastModifiedTime,
-            FieldReference.getWithQuotedRef(excludedColumn.getRootSegmentPath())));
+        valueExpressions.add(new NamedExpression(anyValueCall,
+            FieldReference.getWithQuotedRef(metadataColumns.getRootSegmentPath())));
       }
     }
 
@@ -207,9 +210,8 @@ public class MetadataAggregateHelper {
    */
   private void addCollectListCall(List<LogicalExpression> fieldList) {
     ArrayList<LogicalExpression> collectListArguments = new ArrayList<>(fieldList);
-    List<SchemaPath> excludedColumns = context.excludedColumns();
     // populate columns which weren't included in the schema, but should be collected to the COLLECTED_MAP_FIELD
-    for (SchemaPath logicalExpressions : excludedColumns) {
+    for (SchemaPath logicalExpressions : context.metadataColumns()) {
       // adds string literal with field name to the list
       collectListArguments.add(ValueExpressions.getChar(logicalExpressions.getRootSegmentPath(),
           DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getDefaultPrecision(SqlTypeName.VARCHAR)));
@@ -254,7 +256,7 @@ public class MetadataAggregateHelper {
   private void addMetadataAggregateCalls() {
     AnalyzeColumnUtils.META_STATISTICS_FUNCTIONS.forEach((statisticsKind, sqlKind) -> {
       LogicalExpression call = new FunctionCall(sqlKind.name(),
-          Collections.singletonList(ValueExpressions.getBigInt(1)), ExpressionPosition.UNKNOWN);
+          Collections.singletonList(FieldReference.getWithQuotedRef(columnNamesOptions.projectMetadataColumn())), ExpressionPosition.UNKNOWN);
       valueExpressions.add(
           new NamedExpression(call,
               FieldReference.getWithQuotedRef(AnalyzeColumnUtils.getMetadataStatisticsFieldName(statisticsKind))));
@@ -275,7 +277,6 @@ public class MetadataAggregateHelper {
     for (MaterializedField field : fields) {
       // statistics collecting is not supported for array types
       if (field.getType().getMode() != TypeProtos.DataMode.REPEATED) {
-        List<SchemaPath> excludedColumns = context.excludedColumns();
         // excludedColumns are applied for root fields only
         if (parentFields != null || !excludedColumns.contains(SchemaPath.getSimplePath(field.getName()))) {
           List<String> currentPath;
@@ -313,8 +314,19 @@ public class MetadataAggregateHelper {
       if (interestingColumns == null || interestingColumns.contains(fieldRef)) {
         // collect statistics for all or only interesting columns if they are specified
         AnalyzeColumnUtils.COLUMN_STATISTICS_FUNCTIONS.forEach((statisticsKind, sqlKind) -> {
+          // constructs "case when is not null projectMetadataColumn then column1 else null end" call
+          // to avoid using default values for required columns when data for empty result is obtained
+          LogicalExpression caseExpr = IfExpression.newBuilder()
+              .setIfCondition(new IfExpression.IfCondition(
+                  new FunctionCall(
+                      "isnotnull",
+                      Collections.singletonList(FieldReference.getWithQuotedRef(columnNamesOptions.projectMetadataColumn())),
+                      ExpressionPosition.UNKNOWN), fieldRef))
+              .setElse(NullExpression.INSTANCE)
+              .build();
+
           LogicalExpression call = new FunctionCall(sqlKind.name(),
-              Collections.singletonList(fieldRef), ExpressionPosition.UNKNOWN);
+              Collections.singletonList(caseExpr), ExpressionPosition.UNKNOWN);
           valueExpressions.add(
               new NamedExpression(call,
                   FieldReference.getWithQuotedRef(AnalyzeColumnUtils.getColumnStatisticsFieldName(fieldName, statisticsKind))));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
index 7314961..8ee3beb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
@@ -586,9 +586,9 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
     Multimap<String, StatisticsHolder<?>> columnStatistics = ArrayListMultimap.create();
     Map<String, TypeProtos.MinorType> columnTypes = new HashMap<>();
     for (ColumnMetadata column : columnMetadata) {
-      String fieldName = AnalyzeColumnUtils.getColumnName(column.name());
 
       if (AnalyzeColumnUtils.isColumnStatisticsField(column.name())) {
+        String fieldName = AnalyzeColumnUtils.getColumnName(column.name());
         StatisticsKind<?> statisticsKind = AnalyzeColumnUtils.getStatisticsKind(column.name());
         columnStatistics.put(fieldName,
             new StatisticsHolder<>(getConvertedColumnValue(reader.column(column.name())), statisticsKind));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertMetadataAggregateToDirectScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertMetadataAggregateToDirectScanRule.java
index 80a463b..f9b266e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertMetadataAggregateToDirectScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ConvertMetadataAggregateToDirectScanRule.java
@@ -39,6 +39,7 @@ import org.apache.drill.exec.store.ColumnExplorer.ImplicitFileColumns;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FormatSelection;
 import org.apache.drill.exec.store.direct.DirectGroupScan;
+import org.apache.drill.exec.store.parquet.BaseParquetMetadataProvider;
 import org.apache.drill.exec.store.parquet.ParquetGroupScan;
 import org.apache.drill.exec.store.pojo.DynamicPojoRecordReader;
 import org.apache.drill.exec.util.ImpersonationUtil;
@@ -50,7 +51,6 @@ import org.apache.drill.metastore.statistics.ColumnStatisticsKind;
 import org.apache.drill.metastore.statistics.ExactStatisticsConstants;
 import org.apache.drill.metastore.statistics.StatisticsKind;
 import org.apache.drill.metastore.statistics.TableStatisticsKind;
-import org.apache.drill.metastore.util.SchemaPathUtils;
 import org.apache.drill.shaded.guava.com.google.common.collect.HashBasedTable;
 import org.apache.drill.shaded.guava.com.google.common.collect.Multimap;
 import org.apache.drill.shaded.guava.com.google.common.collect.Table;
@@ -94,11 +94,11 @@ import java.util.stream.IntStream;
  * </pre>
  */
 public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
+  private static final Logger logger = LoggerFactory.getLogger(ConvertMetadataAggregateToDirectScanRule.class);
+
   public static final ConvertMetadataAggregateToDirectScanRule INSTANCE =
       new ConvertMetadataAggregateToDirectScanRule();
 
-  private static final Logger logger = LoggerFactory.getLogger(ConvertMetadataAggregateToDirectScanRule.class);
-
   public ConvertMetadataAggregateToDirectScanRule() {
     super(
         RelOptHelper.some(MetadataAggRel.class, RelOptHelper.any(DrillScanRel.class)),
@@ -210,7 +210,7 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
 
         // do not gather statistics for array columns as it is not supported by Metastore
         if (containsArrayColumn(rowGroupMetadata.getSchema(), schemaPath)) {
-          return null;
+          continue;
         }
 
         if (IsPredicate.isNullOrEmpty(columnStatistics)) {
@@ -232,6 +232,8 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
                 columnStatisticsFieldName,
                 statsValue.getClass());
             recordsTable.put(columnStatisticsFieldName, rowIndex, statsValue);
+          } else {
+            recordsTable.put(columnStatisticsFieldName, rowIndex, BaseParquetMetadataProvider.NULL_VALUE);
           }
         }
       }
@@ -244,6 +246,8 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
         if (statisticsValue != null) {
           schema.putIfAbsent(metadataStatisticsFieldName, statisticsValue.getClass());
           recordsTable.put(metadataStatisticsFieldName, rowIndex, statisticsValue);
+        } else {
+          recordsTable.put(metadataStatisticsFieldName, rowIndex, BaseParquetMetadataProvider.NULL_VALUE);
         }
       }
 
@@ -258,15 +262,19 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
 
     // DynamicPojoRecordReader requires LinkedHashMap with fields order
     // which corresponds to the value position in record list.
-    LinkedHashMap<String, Class<?>> orderedSchema = recordsTable.rowKeySet().stream()
-        .collect(Collectors.toMap(
-            Function.identity(),
-            column -> schema.getOrDefault(column, Integer.class),
-            (o, n) -> n,
-            LinkedHashMap::new));
+    LinkedHashMap<String, Class<?>> orderedSchema = new LinkedHashMap<>();
+    for (String s : recordsTable.rowKeySet()) {
+      Class<?> clazz = schema.get(s);
+      if (clazz != null) {
+        orderedSchema.put(s, clazz);
+      } else {
+        return null;
+      }
+    }
 
     IntFunction<List<Object>> collectRecord = currentIndex -> orderedSchema.keySet().stream()
         .map(column -> recordsTable.get(column, currentIndex))
+        .map(value -> value != BaseParquetMetadataProvider.NULL_VALUE ? value : null)
         .collect(Collectors.toList());
 
     List<List<Object>> records = IntStream.range(0, rowIndex)
@@ -288,12 +296,11 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
    * @return {@code true} if any segment in the schema path is an array, {@code false} otherwise
    */
   private static boolean containsArrayColumn(TupleMetadata schema, SchemaPath schemaPath) {
-    ColumnMetadata columnMetadata = SchemaPathUtils.getColumnMetadata(schemaPath, schema);
     PathSegment currentPath = schemaPath.getRootSegment();
-    ColumnMetadata currentColumn = columnMetadata;
-    do {
-      if (currentColumn.isArray()) {
-        return false;
+    ColumnMetadata columnMetadata = schema.metadata(currentPath.getNameSegment().getPath());
+    while (columnMetadata != null) {
+      if (columnMetadata.isArray()) {
+        return true;
       } else if (columnMetadata.isMap()) {
         currentPath = currentPath.getChild();
         columnMetadata = columnMetadata.tupleSchema().metadata(currentPath.getNameSegment().getPath());
@@ -301,9 +308,9 @@ public class ConvertMetadataAggregateToDirectScanRule extends RelOptRule {
         currentPath = currentPath.getChild();
         columnMetadata = ((DictColumnMetadata) columnMetadata).valueColumnMetadata();
       } else {
-        return true;
+        return false;
       }
-    } while (columnMetadata != null);
-    return true;
+    }
+    return false;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java
index 36eae41..2be8dfc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/MetastoreAnalyzeTableHandler.java
@@ -406,13 +406,13 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
     SchemaPath lastModifiedTimeField =
         SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
 
-    List<SchemaPath> excludedColumns = Arrays.asList(locationField, lastModifiedTimeField);
+    List<SchemaPath> metadataColumns = Arrays.asList(locationField, lastModifiedTimeField);
 
     MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
         .groupByExpressions(Collections.emptyList())
         .interestingColumns(statisticsColumns)
         .createNewAggregations(createNewAggregations)
-        .excludedColumns(excludedColumns)
+        .metadataColumns(metadataColumns)
         .metadataLevel(MetadataType.TABLE)
         .build();
 
@@ -433,7 +433,7 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
     SchemaPath lastModifiedTimeField =
         SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
 
-    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField);
+    List<SchemaPath> metadataColumns = Arrays.asList(lastModifiedTimeField, locationField);
 
     List<NamedExpression> groupByExpressions = new ArrayList<>(segmentExpressions);
 
@@ -441,7 +441,7 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
         .groupByExpressions(groupByExpressions.subList(0, segmentLevel))
         .interestingColumns(statisticsColumns)
         .createNewAggregations(createNewAggregations)
-        .excludedColumns(excludedColumns)
+        .metadataColumns(metadataColumns)
         .metadataLevel(MetadataType.SEGMENT)
         .build();
 
@@ -461,7 +461,7 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
     SchemaPath lastModifiedTimeField =
         SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL));
 
-    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField);
+    List<SchemaPath> metadataColumns = Arrays.asList(lastModifiedTimeField, locationField);
 
     NamedExpression locationExpression =
         new NamedExpression(locationField, FieldReference.getWithQuotedRef(MetastoreAnalyzeConstants.LOCATION_FIELD));
@@ -472,7 +472,7 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
         .groupByExpressions(fileGroupByExpressions)
         .interestingColumns(statisticsColumns)
         .createNewAggregations(createNewAggregations)
-        .excludedColumns(excludedColumns)
+        .metadataColumns(metadataColumns)
         .metadataLevel(MetadataType.FILE)
         .build();
 
@@ -505,13 +505,13 @@ public class MetastoreAnalyzeTableHandler extends DefaultSqlHandler {
     SchemaPath rowGroupLengthField =
         SchemaPath.getSimplePath(config.getContext().getOptions().getString(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL));
 
-    List<SchemaPath> excludedColumns = Arrays.asList(lastModifiedTimeField, locationField, rgiField, rowGroupStartField, rowGroupLengthField);
+    List<SchemaPath> metadataColumns = Arrays.asList(lastModifiedTimeField, locationField, rgiField, rowGroupStartField, rowGroupLengthField);
 
     MetadataAggregateContext aggregateContext = MetadataAggregateContext.builder()
         .groupByExpressions(rowGroupGroupByExpressions)
         .interestingColumns(statisticsColumns)
         .createNewAggregations(createNewAggregations)
-        .excludedColumns(excludedColumns)
+        .metadataColumns(metadataColumns)
         .metadataLevel(MetadataType.ROW_GROUP)
         .build();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index e3ed2f6..430e0a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -267,6 +267,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.IMPLICIT_LAST_MODIFIED_TIME_COLUMN_LABEL_VALIDATOR),
+      new OptionDefinition(ExecConstants.IMPLICIT_PROJECT_METADATA_COLUMN_LABEL_VALIDATOR),
       new OptionDefinition(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE_VALIDATOR),
       new OptionDefinition(ExecConstants.CREATE_PREPARE_STATEMENT_TIMEOUT_MILLIS_VALIDATOR),
       new OptionDefinition(ExecConstants.DYNAMIC_UDF_SUPPORT_ENABLED_VALIDATOR,  new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, false)),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
index d4c45ae..ecb12f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
@@ -311,6 +311,9 @@ public class ColumnExplorer {
         case ROW_GROUP_LENGTH:
           implicitValues.put(key, String.valueOf(length));
           break;
+        case PROJECT_METADATA:
+          implicitValues.put(key, Boolean.TRUE.toString());
+          break;
         case LAST_MODIFIED_TIME:
           try {
             implicitValues.put(key, String.valueOf(fs.getFileStatus(filePath).getModificationTime()));
@@ -509,7 +512,9 @@ public class ColumnExplorer {
 
     ROW_GROUP_START(ExecConstants.IMPLICIT_ROW_GROUP_START_COLUMN_LABEL),
 
-    ROW_GROUP_LENGTH(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL);
+    ROW_GROUP_LENGTH(ExecConstants.IMPLICIT_ROW_GROUP_LENGTH_COLUMN_LABEL),
+
+    PROJECT_METADATA(ExecConstants.IMPLICIT_PROJECT_METADATA_COLUMN_LABEL);
 
     private final String name;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
index b535e0f..f2e177d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
@@ -77,7 +77,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   /**
    * {@link HashBasedTable} cannot contain nulls, used this object to represent null values.
    */
-  static final Object NULL_VALUE = new Object();
+  public static final Object NULL_VALUE = new Object();
 
   protected final List<ReadEntryWithPath> entries;
   protected final ParquetReaderConfig readerConfig;
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 7230426..b365047 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -483,6 +483,7 @@ drill.exec.options: {
     drill.exec.storage.implicit.row_group_start.column.label: "rgs",
     drill.exec.storage.implicit.row_group_length.column.label: "rgl",
     drill.exec.storage.implicit.last_modified_time.column.label: "lmt",
+    drill.exec.storage.implicit.project_metadata.column.label: "$project_metadata$",
     drill.exec.testing.controls: "{}",
     drill.exec.memory.operator.output_batch_size : 16777216, # 16 MB
     drill.exec.memory.operator.output_batch_size_avail_mem_factor : 0.1,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
index 51d36f2..394adca 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
@@ -3160,6 +3160,316 @@ public class TestMetastoreCommands extends ClusterTest {
     }
   }
 
+  @Test
+  public void testAnalyzeEmptyNullableParquetTable() throws Exception {
+    File table = dirTestWatcher.copyResourceToRoot(Paths.get("parquet", "empty", "simple", "empty_simple.parquet"));
+
+    String tableName = "parquet/empty/simple/empty_simple.parquet";
+
+    TableInfo tableInfo = getTableInfo(tableName, "default");
+
+    TupleMetadata schema = new SchemaBuilder()
+        .addNullable("id", TypeProtos.MinorType.BIGINT)
+        .addNullable("name", TypeProtos.MinorType.VARCHAR)
+        .build();
+
+    Map<SchemaPath, ColumnStatistics<?>> columnStatistics = ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
+        .put(SchemaPath.getSimplePath("name"),
+            getColumnStatistics(null, null, 0L, TypeProtos.MinorType.VARCHAR))
+        .put(SchemaPath.getSimplePath("id"),
+            getColumnStatistics(null, null, 0L, TypeProtos.MinorType.BIGINT))
+        .build();
+
+    BaseTableMetadata expectedTableMetadata = BaseTableMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(TABLE_META_INFO)
+        .schema(schema)
+        .location(new Path(table.toURI().getPath()))
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(Arrays.asList(new StatisticsHolder<>(0L, TableStatisticsKind.ROW_COUNT),
+            new StatisticsHolder<>(MetadataType.ALL, TableStatisticsKind.ANALYZE_METADATA_LEVEL)))
+        .partitionKeys(Collections.emptyMap())
+        .lastModifiedTime(getMaxLastModified(table))
+        .build();
+
+    try {
+      testBuilder()
+          .sqlQuery("ANALYZE TABLE dfs.`%s` REFRESH METADATA", tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Collected / refreshed metadata for table [dfs.default.%s]", tableName))
+          .go();
+
+      MetastoreTableInfo metastoreTableInfo = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .metastoreTableInfo(tableInfo);
+
+      assertTrue("table metadata wasn't found", metastoreTableInfo.isExists());
+
+      BaseTableMetadata tableMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .tableMetadata(tableInfo);
+
+      assertEquals(expectedTableMetadata, tableMetadata);
+
+      List<FileMetadata> filesMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .filesMetadata(tableInfo, null, null);
+
+      assertEquals(1, filesMetadata.size());
+
+      List<RowGroupMetadata> rowGroupsMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .rowGroupsMetadata(tableInfo, (String) null, null);
+
+      assertEquals(1, rowGroupsMetadata.size());
+    } finally {
+      run("analyze table dfs.`%s` drop metadata if exists", tableName);
+    }
+  }
+
+  @Test
+  public void testAnalyzeEmptyRequiredParquetTable() throws Exception {
+    String tableName = "analyze_empty_simple_required";
+
+    run("create table dfs.tmp.%s as select 1 as id, 'a' as name from (values(1)) where 1 = 2", tableName);
+
+    File table = new File(dirTestWatcher.getDfsTestTmpDir(), tableName);
+
+    TableInfo tableInfo = getTableInfo(tableName, "tmp");
+
+    TupleMetadata schema = new SchemaBuilder()
+        .add("id", TypeProtos.MinorType.INT)
+        .add("name", TypeProtos.MinorType.VARCHAR)
+        .build();
+
+    Map<SchemaPath, ColumnStatistics<?>> columnStatistics = ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
+        .put(SchemaPath.getSimplePath("name"),
+            getColumnStatistics(null, null, 0L, TypeProtos.MinorType.VARCHAR))
+        .put(SchemaPath.getSimplePath("id"),
+            getColumnStatistics(null, null, 0L, TypeProtos.MinorType.INT))
+        .build();
+
+    BaseTableMetadata expectedTableMetadata = BaseTableMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(TABLE_META_INFO)
+        .schema(schema)
+        .location(new Path(table.toURI().getPath()))
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(Arrays.asList(new StatisticsHolder<>(0L, TableStatisticsKind.ROW_COUNT),
+            new StatisticsHolder<>(MetadataType.ALL, TableStatisticsKind.ANALYZE_METADATA_LEVEL)))
+        .partitionKeys(Collections.emptyMap())
+        .lastModifiedTime(getMaxLastModified(table))
+        .build();
+
+    try {
+      testBuilder()
+          .sqlQuery("ANALYZE TABLE dfs.tmp.`%s` REFRESH METADATA", tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Collected / refreshed metadata for table [dfs.tmp.%s]", tableName))
+          .go();
+
+      MetastoreTableInfo metastoreTableInfo = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .metastoreTableInfo(tableInfo);
+
+      assertTrue("table metadata wasn't found", metastoreTableInfo.isExists());
+
+      BaseTableMetadata tableMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .tableMetadata(tableInfo);
+
+      assertEquals(expectedTableMetadata, tableMetadata);
+
+      List<FileMetadata> filesMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .filesMetadata(tableInfo, null, null);
+
+      assertEquals(1, filesMetadata.size());
+
+      List<RowGroupMetadata> rowGroupsMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .rowGroupsMetadata(tableInfo, (String) null, null);
+
+      assertEquals(1, rowGroupsMetadata.size());
+    } finally {
+      run("analyze table dfs.tmp.`%s` drop metadata if exists", tableName);
+      run("drop table if exists dfs.tmp.`%s`", tableName);
+    }
+  }
+
+  @Test
+  public void testAnalyzeNonEmptyTableWithEmptyFile() throws Exception {
+    String tableName = "parquet_with_empty_file";
+
+    File table = dirTestWatcher.copyResourceToTestTmp(Paths.get("parquet", "empty", "simple"), Paths.get(tableName));
+
+    TableInfo tableInfo = getTableInfo(tableName, "tmp");
+
+    TupleMetadata schema = new SchemaBuilder()
+        .addNullable("id", TypeProtos.MinorType.BIGINT)
+        .addNullable("name", TypeProtos.MinorType.VARCHAR)
+        .build();
+
+    Map<SchemaPath, ColumnStatistics<?>> columnStatistics = ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
+        .put(SchemaPath.getSimplePath("name"),
+            getColumnStatistics("Tom", "Tom", 1L, TypeProtos.MinorType.VARCHAR))
+        .put(SchemaPath.getSimplePath("id"),
+            getColumnStatistics(2L, 2L, 1L, TypeProtos.MinorType.BIGINT))
+        .build();
+
+    BaseTableMetadata expectedTableMetadata = BaseTableMetadata.builder()
+        .tableInfo(tableInfo)
+        .metadataInfo(TABLE_META_INFO)
+        .schema(schema)
+        .location(new Path(table.toURI().getPath()))
+        .columnsStatistics(columnStatistics)
+        .metadataStatistics(Arrays.asList(new StatisticsHolder<>(1L, TableStatisticsKind.ROW_COUNT),
+            new StatisticsHolder<>(MetadataType.ALL, TableStatisticsKind.ANALYZE_METADATA_LEVEL)))
+        .partitionKeys(Collections.emptyMap())
+        .lastModifiedTime(getMaxLastModified(table))
+        .build();
+
+    try {
+      testBuilder()
+          .sqlQuery("ANALYZE TABLE dfs.tmp.`%s` REFRESH METADATA", tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Collected / refreshed metadata for table [dfs.tmp.%s]", tableName))
+          .go();
+
+      MetastoreTableInfo metastoreTableInfo = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .metastoreTableInfo(tableInfo);
+
+      assertTrue("table metadata wasn't found", metastoreTableInfo.isExists());
+
+      BaseTableMetadata tableMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .tableMetadata(tableInfo);
+
+      assertEquals(expectedTableMetadata, tableMetadata);
+
+      List<FileMetadata> filesMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .filesMetadata(tableInfo, null, null);
+
+      assertEquals(2, filesMetadata.size());
+
+      List<RowGroupMetadata> rowGroupsMetadata = cluster.drillbit().getContext()
+          .getMetastoreRegistry()
+          .get()
+          .tables()
+          .basicRequests()
+          .rowGroupsMetadata(tableInfo, (String) null, null);
+
+      assertEquals(2, rowGroupsMetadata.size());
+    } finally {
+      run("analyze table dfs.tmp.`%s` drop metadata if exists", tableName);
+    }
+  }
+
+  @Test
+  public void testSelectEmptyRequiredParquetTable() throws Exception {
+    String tableName = "empty_simple_required";
+
+    run("create table dfs.tmp.%s as select 1 as id, 'a' as name from (values(1)) where 1 = 2", tableName);
+
+    try {
+      testBuilder()
+          .sqlQuery("ANALYZE TABLE dfs.tmp.`%s` REFRESH METADATA", tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Collected / refreshed metadata for table [dfs.tmp.%s]", tableName))
+          .go();
+
+      String query = "select * from dfs.tmp.`%s`";
+
+      queryBuilder()
+          .sql(query, tableName)
+          .planMatcher()
+          .include("usedMetastore=true")
+          .match();
+
+      testBuilder()
+          .sqlQuery(query, tableName)
+          .unOrdered()
+          .baselineColumns("id", "name")
+          .expectsEmptyResultSet()
+          .go();
+    } finally {
+      run("analyze table dfs.tmp.`%s` drop metadata if exists", tableName);
+      run("drop table if exists dfs.tmp.`%s`", tableName);
+    }
+  }
+
+  @Test
+  public void testSelectNonEmptyTableWithEmptyFile() throws Exception {
+    String tableName = "parquet_with_empty_file";
+
+    dirTestWatcher.copyResourceToRoot(Paths.get("parquet", "empty", "simple"), Paths.get(tableName));
+
+    try {
+      testBuilder()
+          .sqlQuery("ANALYZE TABLE dfs.`%s` REFRESH METADATA", tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Collected / refreshed metadata for table [dfs.default.%s]", tableName))
+          .go();
+
+      String query = "select * from dfs.`%s`";
+
+      queryBuilder()
+          .sql(query, tableName)
+          .planMatcher()
+          .include("usedMetastore=true")
+          .match();
+
+      testBuilder()
+          .sqlQuery(query, tableName)
+          .unOrdered()
+          .baselineColumns("id", "name")
+          .baselineValues(2L, "Tom")
+          .go();
+    } finally {
+      run("analyze table dfs.`%s` drop metadata if exists", tableName);
+    }
+  }
+
   private static <T> ColumnStatistics<T> getColumnStatistics(T minValue, T maxValue,
       long rowCount, TypeProtos.MinorType minorType) {
     return new ColumnStatistics<>(


[drill] 10/10: DRILL-7514: Update Apache POI to Latest Version

Posted by ag...@apache.org.
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 8a8e58bf8382060305162b2ca261949b91179e46
Author: Charles Givre <cg...@apache.org>
AuthorDate: Thu Feb 20 13:21:06 2020 -0500

    DRILL-7514: Update Apache POI to Latest Version
    
    closes #1991
---
 contrib/format-excel/pom.xml                                 |  2 +-
 .../org/apache/drill/exec/store/excel/ExcelBatchReader.java  | 12 ++++--------
 contrib/udfs/pom.xml                                         |  2 +-
 3 files changed, 6 insertions(+), 10 deletions(-)

diff --git a/contrib/format-excel/pom.xml b/contrib/format-excel/pom.xml
index c66a912..ca6b6ab 100644
--- a/contrib/format-excel/pom.xml
+++ b/contrib/format-excel/pom.xml
@@ -31,7 +31,7 @@
   <name>contrib/format-excel</name>
 
   <properties>
-    <poi.version>4.1.1</poi.version>
+    <poi.version>4.1.2</poi.version>
   </properties>
   <dependencies>
     <dependency>
diff --git a/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelBatchReader.java b/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelBatchReader.java
index 848fa75..32c062d 100644
--- a/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelBatchReader.java
+++ b/contrib/format-excel/src/main/java/org/apache/drill/exec/store/excel/ExcelBatchReader.java
@@ -38,6 +38,7 @@ import org.apache.poi.ss.usermodel.CellValue;
 import org.apache.poi.ss.usermodel.DateUtil;
 import org.apache.poi.ss.usermodel.FormulaEvaluator;
 import org.apache.poi.ss.usermodel.Row;
+import org.apache.poi.xssf.usermodel.XSSFRow;
 import org.apache.poi.xssf.usermodel.XSSFSheet;
 import org.apache.poi.xssf.usermodel.XSSFWorkbook;
 import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework.FileSchemaNegotiator;
@@ -270,15 +271,10 @@ public class ExcelBatchReader implements ManagedReader<FileSchemaNegotiator> {
    * @return The number of actual columns
    */
   private int getColumnCount() {
-    int columnCount;
+    int rowNumber = readerConfig.headerRow > 0 ? sheet.getFirstRowNum() : 0;
+    XSSFRow sheetRow = sheet.getRow(rowNumber);
 
-    if (readerConfig.headerRow >= 0) {
-      columnCount = sheet.getRow(sheet.getFirstRowNum()).getPhysicalNumberOfCells();
-    } else {
-      // Case for when the user defines the headerRow as -1 IE:  When there isn't a headerRow.
-      columnCount = sheet.getRow(0).getPhysicalNumberOfCells();
-    }
-    return columnCount;
+    return sheetRow != null ? sheetRow.getPhysicalNumberOfCells() : 0;
   }
 
   @Override
diff --git a/contrib/udfs/pom.xml b/contrib/udfs/pom.xml
index ed2bac2..be24c96 100644
--- a/contrib/udfs/pom.xml
+++ b/contrib/udfs/pom.xml
@@ -66,7 +66,7 @@
     <dependency>
       <groupId>nl.basjes.parse.useragent</groupId>
       <artifactId>yauaa</artifactId>
-      <version>5.11</version>
+      <version>5.13</version>
     </dependency>
 
     <!-- Test dependencies -->


[drill] 02/10: DRILL-7589: Set temporary tests folder for UDF_DIRECTORY_LOCAL, fix allocators closing in BloomFilterTest and TestWriteToDisk, fix permissions issue for TestGracefulShutdown tests

Posted by ag...@apache.org.
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 a81c49653503b7c2f8707a769095e50838c8f8fa
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Mon Feb 17 20:32:32 2020 +0200

    DRILL-7589: Set temporary tests folder for UDF_DIRECTORY_LOCAL, fix allocators closing in BloomFilterTest and TestWriteToDisk, fix permissions issue for TestGracefulShutdown tests
    
    closes #1987
---
 .../apache/drill/exec/cache/TestWriteToDisk.java   | 106 +++-----
 .../exec/udf/dynamic/TestDynamicUDFSupport.java    |  11 +-
 .../drill/exec/work/filter/BloomFilterTest.java    | 274 ++++++---------------
 .../org/apache/drill/test/BaseDirTestWatcher.java  |  11 +
 .../java/org/apache/drill/test/ClusterFixture.java |   1 +
 .../org/apache/drill/test/OperatorFixture.java     |   1 +
 .../apache/drill/test/TestGracefulShutdown.java    | 100 ++++----
 7 files changed, 186 insertions(+), 318 deletions(-)

diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
index db7f743..c6a68b6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
@@ -18,95 +18,61 @@
 package org.apache.drill.exec.cache;
 
 import java.io.File;
-import java.util.List;
 
-import org.apache.drill.shaded.guava.com.google.common.io.Files;
-import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSets;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.test.TestTools;
+import org.apache.drill.test.SubOperatorTest;
 import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VarBinaryVector;
+import org.apache.drill.test.rowSet.RowSetUtilities;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.Rule;
 import org.junit.Test;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.junit.rules.TestRule;
-
-public class TestWriteToDisk extends ExecTest {
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(90000); // 90secs
+public class TestWriteToDisk extends SubOperatorTest {
 
   @Test
-  @SuppressWarnings("static-method")
   public void test() throws Exception {
-    final List<ValueVector> vectorList = Lists.newArrayList();
-    final DrillConfig config = DrillConfig.create();
-    try (final RemoteServiceSet serviceSet = RemoteServiceSet
-        .getLocalServiceSet();
-        final Drillbit bit = new Drillbit(config, serviceSet)) {
-      bit.run();
-      final DrillbitContext context = bit.getContext();
-
-      final MaterializedField intField = MaterializedField.create("int", Types.required(TypeProtos.MinorType.INT));
-      final MaterializedField binField = MaterializedField.create("binary", Types.required(TypeProtos.MinorType.VARBINARY));
-      try (final IntVector intVector = (IntVector) TypeHelper.getNewVector(intField, context.getAllocator());
-          final VarBinaryVector binVector =
-              (VarBinaryVector) TypeHelper.getNewVector(binField, context.getAllocator())) {
-        AllocationHelper.allocate(intVector, 4, 4);
-        AllocationHelper.allocate(binVector, 4, 5);
-        vectorList.add(intVector);
-        vectorList.add(binVector);
+    VectorContainer container = expectedRowSet().container();
 
-        intVector.getMutator().setSafe(0, 0);
-        binVector.getMutator().setSafe(0, "ZERO".getBytes());
-        intVector.getMutator().setSafe(1, 1);
-        binVector.getMutator().setSafe(1, "ONE".getBytes());
-        intVector.getMutator().setSafe(2, 2);
-        binVector.getMutator().setSafe(2, "TWO".getBytes());
-        intVector.getMutator().setSafe(3, 3);
-        binVector.getMutator().setSafe(3, "THREE".getBytes());
-        intVector.getMutator().setValueCount(4);
-        binVector.getMutator().setValueCount(4);
+    WritableBatch batch = WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container, false);
 
-        VectorContainer container = new VectorContainer();
-        container.addCollection(vectorList);
-        container.setRecordCount(4);
-        WritableBatch batch = WritableBatch.getBatchNoHVWrap(
-            container.getRecordCount(), container, false);
-        VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(
-            batch, context.getAllocator());
+    VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, fixture.allocator());
 
-        final VectorAccessibleSerializable newWrap = new VectorAccessibleSerializable(
-            context.getAllocator());
-        try (final FileSystem fs = getLocalFileSystem()) {
-          final File tempDir = Files.createTempDir();
-          tempDir.deleteOnExit();
-          final Path path = new Path(tempDir.getAbsolutePath(), "drillSerializable");
-          try (final FSDataOutputStream out = fs.create(path)) {
-            wrap.writeToStream(out);
-          }
-
-          try (final FSDataInputStream in = fs.open(path)) {
-            newWrap.readFromStream(in);
-          }
-        }
+    VectorAccessibleSerializable newWrap = new VectorAccessibleSerializable(fixture.allocator());
+    try (FileSystem fs = ExecTest.getLocalFileSystem()) {
+      File tempDir = dirTestWatcher.getTmpDir();
+      tempDir.deleteOnExit();
+      Path path = new Path(tempDir.getAbsolutePath(), "drillSerializable");
+      try (FSDataOutputStream out = fs.create(path)) {
+        wrap.writeToStream(out);
+      }
 
-        newWrap.get();
+      try (FSDataInputStream in = fs.open(path)) {
+        newWrap.readFromStream(in);
       }
     }
+
+    RowSetUtilities.verify(expectedRowSet(), RowSets.wrap(newWrap.get()));
+  }
+
+  private RowSet expectedRowSet() {
+    TupleMetadata schema = new SchemaBuilder()
+        .add("int", TypeProtos.MinorType.INT)
+        .add("binary", TypeProtos.MinorType.VARBINARY)
+        .build();
+
+    return fixture.rowSetBuilder(schema)
+        .addRow(0, "ZERO".getBytes())
+        .addRow(1, "ONE".getBytes())
+        .addRow(2, "TWO".getBytes())
+        .addRow(3, "THREE".getBytes())
+        .build();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/udf/dynamic/TestDynamicUDFSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/udf/dynamic/TestDynamicUDFSupport.java
index dd9da22..c83d24a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/udf/dynamic/TestDynamicUDFSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/udf/dynamic/TestDynamicUDFSupport.java
@@ -78,7 +78,6 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
   private static final String DEFAULT_JAR_NAME = "drill-custom-lower";
   private static URI fsUri;
-  private static File udfDir;
   private static File jarsDir;
   private static File buildDirectory;
   private static JarBuilder jarBuilder;
@@ -103,9 +102,10 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
   @Before
   public void setupNewDrillbit() throws Exception {
-    udfDir = dirTestWatcher.makeSubDir(Paths.get("udf"));
+    File udfLocalDir = new File(dirTestWatcher.getUdfDir(), "local");
     Properties overrideProps = new Properties();
-    overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, udfDir.getAbsolutePath());
+    overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getUdfDir().getAbsolutePath());
+    overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_LOCAL, udfLocalDir.getAbsolutePath());
     overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_FS, FileSystem.DEFAULT_FS);
     updateTestCluster(1, DrillConfig.create(overrideProps));
 
@@ -115,7 +115,6 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   @After
   public void cleanup() throws Exception {
     closeClient();
-    FileUtils.cleanDirectory(udfDir);
     dirTestWatcher.clear();
   }
 
@@ -957,7 +956,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     return spy;
   }
 
-  private class SimpleQueryRunner implements Runnable {
+  private static class SimpleQueryRunner implements Runnable {
 
     private final String query;
 
@@ -975,7 +974,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     }
   }
 
-  private class TestBuilderRunner implements Runnable {
+  private static class TestBuilderRunner implements Runnable {
 
     private final TestBuilder testBuilder;
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
index 761a2cc..5875a40 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
@@ -17,18 +17,16 @@
  */
 package org.apache.drill.exec.work.filter;
 
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.function.CheckedFunction;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.ValueVectorReadExpression;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.fn.impl.ValueVectorHashHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.FragmentContextImpl;
-import org.apache.drill.exec.proto.BitControl;
+import org.apache.drill.exec.physical.rowSet.RowSet;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
@@ -36,26 +34,23 @@ import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.drill.test.BaseTest;
+import org.apache.drill.test.SubOperatorTest;
 import org.junit.Assert;
 import org.junit.Test;
+
+import java.io.IOException;
 import java.util.Iterator;
 
-public class BloomFilterTest extends BaseTest {
-  public static DrillConfig c = DrillConfig.create();
+public class BloomFilterTest extends SubOperatorTest {
 
-  class TestRecordBatch implements RecordBatch {
+  private static class TestRecordBatch implements RecordBatch {
     private final VectorContainer container;
 
     public TestRecordBatch(VectorContainer container) {
       this.container = container;
-
     }
 
     @Override
@@ -85,7 +80,6 @@ public class BloomFilterTest extends BaseTest {
 
     @Override
     public void kill(boolean sendUpstream) {
-
     }
 
     @Override
@@ -133,214 +127,110 @@ public class BloomFilterTest extends BaseTest {
     }
   }
 
-
   @Test
   public void testNotExist() throws Exception {
-    Drillbit bit = new Drillbit(c, RemoteServiceSet.getLocalServiceSet(), ClassPathScanner.fromPrescan(c));
-    bit.run();
-    DrillbitContext bitContext = bit.getContext();
-    FunctionImplementationRegistry registry = bitContext.getFunctionImplementationRegistry();
-    FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), null, registry);
-    BufferAllocator bufferAllocator = bitContext.getAllocator();
-    //create RecordBatch
-    VarCharVector vector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    vector.allocateNew();
-    int valueCount = 3;
-    VarCharVector.Mutator mutator = vector.getMutator();
-    mutator.setSafe(0, "a".getBytes());
-    mutator.setSafe(1, "b".getBytes());
-    mutator.setSafe(2, "c".getBytes());
-    mutator.setValueCount(valueCount);
-    VectorContainer vectorContainer = new VectorContainer();
-    TypedFieldId fieldId = vectorContainer.add(vector);
-    RecordBatch recordBatch = new TestRecordBatch(vectorContainer);
-    //construct hash64
-    ValueVectorReadExpression exp = new ValueVectorReadExpression(fieldId);
-    LogicalExpression[] expressions = new LogicalExpression[1];
-    expressions[0] = exp;
-    TypedFieldId[] fieldIds = new TypedFieldId[1];
-    fieldIds[0] = fieldId;
-    ValueVectorHashHelper valueVectorHashHelper = new ValueVectorHashHelper(recordBatch, context);
-    ValueVectorHashHelper.Hash64 hash64 = valueVectorHashHelper.getHash64(expressions, fieldIds);
-
-    //construct BloomFilter
-    int numBytes = BloomFilter.optimalNumOfBytes(3, 0.03);
-
-    BloomFilter bloomFilter = new BloomFilter(numBytes, bufferAllocator);
-    for (int i = 0; i < valueCount; i++) {
-      long hashCode = hash64.hash64Code(i, 0, 0);
-      bloomFilter.insert(hashCode);
-    }
+    RowSet.SingleRowSet probeRowSet = fixture.rowSetBuilder(getTestSchema())
+        .addRow("f")
+        .build();
 
-    //-----------------create probe side RecordBatch---------------------
-    VarCharVector probeVector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    probeVector.allocateNew();
-    int probeValueCount = 1;
-    VarCharVector.Mutator mutator1 = probeVector.getMutator();
-    mutator1.setSafe(0, "f".getBytes());
-    mutator1.setValueCount(probeValueCount);
-    VectorContainer probeVectorContainer = new VectorContainer();
-    TypedFieldId probeFieldId = probeVectorContainer.add(probeVector);
-    RecordBatch probeRecordBatch = new TestRecordBatch(probeVectorContainer);
-    ValueVectorReadExpression probExp = new ValueVectorReadExpression(probeFieldId);
-    LogicalExpression[] probExpressions = new LogicalExpression[1];
-    probExpressions[0] = probExp;
-    TypedFieldId[] probeFieldIds = new TypedFieldId[1];
-    probeFieldIds[0] = probeFieldId;
-    ValueVectorHashHelper probeValueVectorHashHelper = new ValueVectorHashHelper(probeRecordBatch, context);
-    ValueVectorHashHelper.Hash64 probeHash64 = probeValueVectorHashHelper.getHash64(probExpressions, probeFieldIds);
-    long hashCode = probeHash64.hash64Code(0, 0, 0);
-    boolean contain = bloomFilter.find(hashCode);
-    Assert.assertFalse(contain);
-    bloomFilter.getContent().close();
-    vectorContainer.clear();
-    probeVectorContainer.clear();
-    context.close();
-    bitContext.close();
-    bit.close();
+    checkBloomFilterResult(probeRowSet, BloomFilterTest::getSimpleBloomFilter, false);
   }
 
-
   @Test
   public void testExist() throws Exception {
+    RowSet.SingleRowSet probeRowSet = fixture.rowSetBuilder(getTestSchema())
+        .addRow("a")
+        .build();
 
-    Drillbit bit = new Drillbit(c, RemoteServiceSet.getLocalServiceSet(), ClassPathScanner.fromPrescan(c));
-    bit.run();
-    DrillbitContext bitContext = bit.getContext();
-    FunctionImplementationRegistry registry = bitContext.getFunctionImplementationRegistry();
-    FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), null, registry);
-    BufferAllocator bufferAllocator = bitContext.getAllocator();
-    //create RecordBatch
-    VarCharVector vector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    vector.allocateNew();
-    int valueCount = 3;
-    VarCharVector.Mutator mutator = vector.getMutator();
-    mutator.setSafe(0, "a".getBytes());
-    mutator.setSafe(1, "b".getBytes());
-    mutator.setSafe(2, "c".getBytes());
-    mutator.setValueCount(valueCount);
-    VectorContainer vectorContainer = new VectorContainer();
-    TypedFieldId fieldId = vectorContainer.add(vector);
-    RecordBatch recordBatch = new TestRecordBatch(vectorContainer);
-    //construct hash64
-    ValueVectorReadExpression exp = new ValueVectorReadExpression(fieldId);
-    LogicalExpression[] expressions = new LogicalExpression[1];
-    expressions[0] = exp;
-    TypedFieldId[] fieldIds = new TypedFieldId[1];
-    fieldIds[0] = fieldId;
-    ValueVectorHashHelper valueVectorHashHelper = new ValueVectorHashHelper(recordBatch, context);
-    ValueVectorHashHelper.Hash64 hash64 = valueVectorHashHelper.getHash64(expressions, fieldIds);
-
-    //construct BloomFilter
-    int numBytes = BloomFilter.optimalNumOfBytes(3, 0.03);
+    checkBloomFilterResult(probeRowSet, BloomFilterTest::getSimpleBloomFilter, true);
+  }
+
+  @Test
+  public void testMerged() throws Exception {
+    RowSet.SingleRowSet probeRowSet = fixture.rowSetBuilder(getTestSchema())
+        .addRow("a")
+        .build();
 
-    BloomFilter bloomFilter = new BloomFilter(numBytes, bufferAllocator);
+    checkBloomFilterResult(probeRowSet, this::getDisjunctionBloomFilter, true);
+  }
+
+  private BloomFilter getDisjunctionBloomFilter(ValueVectorHashHelper.Hash64 hash64) throws SchemaChangeException {
+    int numBytes = BloomFilter.optimalNumOfBytes(3, 0.03);
+    BloomFilter bloomFilter = new BloomFilter(numBytes, fixture.allocator());
+    int valueCount = 3;
     for (int i = 0; i < valueCount; i++) {
       long hashCode = hash64.hash64Code(i, 0, 0);
       bloomFilter.insert(hashCode);
     }
 
-    //-----------------create probe side RecordBatch---------------------
-    VarCharVector probeVector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    probeVector.allocateNew();
-    int probeValueCount = 1;
-    VarCharVector.Mutator mutator1 = probeVector.getMutator();
-    mutator1.setSafe(0, "a".getBytes());
-    mutator1.setValueCount(probeValueCount);
-    VectorContainer probeVectorContainer = new VectorContainer();
-    TypedFieldId probeFieldId = probeVectorContainer.add(probeVector);
-    RecordBatch probeRecordBatch = new TestRecordBatch(probeVectorContainer);
-    ValueVectorReadExpression probExp = new ValueVectorReadExpression(probeFieldId);
-    LogicalExpression[] probExpressions = new LogicalExpression[1];
-    probExpressions[0] = probExp;
-    TypedFieldId[] probeFieldIds = new TypedFieldId[1];
-    probeFieldIds[0] = probeFieldId;
-    ValueVectorHashHelper probeValueVectorHashHelper = new ValueVectorHashHelper(probeRecordBatch, context);
-    ValueVectorHashHelper.Hash64 probeHash64 = probeValueVectorHashHelper.getHash64(probExpressions, probeFieldIds);
-    long hashCode = probeHash64.hash64Code(0, 0, 0);
-    boolean contain = bloomFilter.find(hashCode);
-    Assert.assertTrue(contain);
+    BloomFilter disjunctionBloomFilter = getSimpleBloomFilter(hash64);
+    disjunctionBloomFilter.or(bloomFilter);
+
     bloomFilter.getContent().close();
-    vectorContainer.clear();
-    probeVectorContainer.clear();
-    context.close();
-    bitContext.close();
-    bit.close();
+
+    return disjunctionBloomFilter;
   }
 
+  private static BloomFilter getSimpleBloomFilter(ValueVectorHashHelper.Hash64 hash64) throws SchemaChangeException {
+    int numBytes = BloomFilter.optimalNumOfBytes(3, 0.03);
 
-  @Test
-  public void testMerged() throws Exception {
+    BloomFilter bloomFilter = new BloomFilter(numBytes, fixture.allocator());
 
-    Drillbit bit = new Drillbit(c, RemoteServiceSet.getLocalServiceSet(), ClassPathScanner.fromPrescan(c));
-    bit.run();
-    DrillbitContext bitContext = bit.getContext();
-    FunctionImplementationRegistry registry = bitContext.getFunctionImplementationRegistry();
-    FragmentContextImpl context = new FragmentContextImpl(bitContext, BitControl.PlanFragment.getDefaultInstance(), null, registry);
-    BufferAllocator bufferAllocator = bitContext.getAllocator();
-    //create RecordBatch
-    VarCharVector vector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    vector.allocateNew();
     int valueCount = 3;
-    VarCharVector.Mutator mutator = vector.getMutator();
-    mutator.setSafe(0, "a".getBytes());
-    mutator.setSafe(1, "b".getBytes());
-    mutator.setSafe(2, "c".getBytes());
-    mutator.setValueCount(valueCount);
-    VectorContainer vectorContainer = new VectorContainer();
-    TypedFieldId fieldId = vectorContainer.add(vector);
-    RecordBatch recordBatch = new TestRecordBatch(vectorContainer);
-    //construct hash64
-    ValueVectorReadExpression exp = new ValueVectorReadExpression(fieldId);
-    LogicalExpression[] expressions = new LogicalExpression[1];
-    expressions[0] = exp;
-    TypedFieldId[] fieldIds = new TypedFieldId[1];
-    fieldIds[0] = fieldId;
-    ValueVectorHashHelper valueVectorHashHelper = new ValueVectorHashHelper(recordBatch, context);
-    ValueVectorHashHelper.Hash64 hash64 = valueVectorHashHelper.getHash64(expressions, fieldIds);
-
-    //construct BloomFilter
-    int numBytes = BloomFilter.optimalNumOfBytes(3, 0.03);
-
-    BloomFilter bloomFilter = new BloomFilter(numBytes, bufferAllocator);
     for (int i = 0; i < valueCount; i++) {
       long hashCode = hash64.hash64Code(i, 0, 0);
       bloomFilter.insert(hashCode);
     }
+    return bloomFilter;
+  }
 
-    BloomFilter bloomFilter1 = new BloomFilter(numBytes, bufferAllocator);
-    for (int i = 0; i < valueCount; i++) {
-      long hashCode = hash64.hash64Code(i, 0, 0);
-      bloomFilter1.insert(hashCode);
+  private void checkBloomFilterResult(RowSet.SingleRowSet probeRowSet,
+      CheckedFunction<ValueVectorHashHelper.Hash64, BloomFilter, SchemaChangeException> bloomFilterProvider,
+      boolean matches) throws ClassTransformationException, IOException, SchemaChangeException {
+    try (FragmentContext context = fixture.getFragmentContext()) {
+      // create build side batch
+      RowSet.SingleRowSet batchRowSet = fixture.rowSetBuilder(getTestSchema())
+          .addRow("a")
+          .addRow("b")
+          .addRow("c")
+          .build();
+
+      // create build side Hash64
+      ValueVectorHashHelper.Hash64 hash64 = getHash64(context, batchRowSet);
+
+      // construct BloomFilter
+      BloomFilter bloomFilter = bloomFilterProvider.apply(hash64);
+
+      // create probe side Hash64
+      ValueVectorHashHelper.Hash64 probeHash64 = getHash64(context, probeRowSet);
+
+      long hashCode = probeHash64.hash64Code(0, 0, 0);
+
+      Assert.assertEquals(matches, bloomFilter.find(hashCode));
+
+      bloomFilter.getContent().close();
+      batchRowSet.clear();
+      probeRowSet.clear();
     }
+  }
 
-    bloomFilter.or(bloomFilter1);
-
-    //-----------------create probe side RecordBatch---------------------
-    VarCharVector probeVector = new VarCharVector(SchemaBuilder.columnSchema("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED), bufferAllocator);
-    probeVector.allocateNew();
-    int probeValueCount = 1;
-    VarCharVector.Mutator mutator1 = probeVector.getMutator();
-    mutator1.setSafe(0, "a".getBytes());
-    mutator1.setValueCount(probeValueCount);
-    VectorContainer probeVectorContainer = new VectorContainer();
-    TypedFieldId probeFieldId = probeVectorContainer.add(probeVector);
-    RecordBatch probeRecordBatch = new TestRecordBatch(probeVectorContainer);
+  private static TupleMetadata getTestSchema() {
+    return new SchemaBuilder()
+        .add("a", TypeProtos.MinorType.VARCHAR)
+        .build();
+  }
+
+  private static ValueVectorHashHelper.Hash64 getHash64(FragmentContext context,
+      RowSet.SingleRowSet probeRowSet) throws ClassTransformationException, IOException, SchemaChangeException {
+
+    RecordBatch probeRecordBatch = new TestRecordBatch(probeRowSet.container());
+    TypedFieldId probeFieldId = probeRecordBatch.getValueVectorId(SchemaPath.getSimplePath("a"));
     ValueVectorReadExpression probExp = new ValueVectorReadExpression(probeFieldId);
     LogicalExpression[] probExpressions = new LogicalExpression[1];
     probExpressions[0] = probExp;
     TypedFieldId[] probeFieldIds = new TypedFieldId[1];
     probeFieldIds[0] = probeFieldId;
     ValueVectorHashHelper probeValueVectorHashHelper = new ValueVectorHashHelper(probeRecordBatch, context);
-    ValueVectorHashHelper.Hash64 probeHash64 = probeValueVectorHashHelper.getHash64(probExpressions, probeFieldIds);
-    long hashCode = probeHash64.hash64Code(0, 0, 0);
-    boolean contain = bloomFilter.find(hashCode);
-    Assert.assertTrue(contain);
-    bloomFilter.getContent().close();
-    vectorContainer.clear();
-    probeVectorContainer.clear();
-    context.close();
-    bitContext.close();
-    bit.close();
+    return probeValueVectorHashHelper.getHash64(probExpressions, probeFieldIds);
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/BaseDirTestWatcher.java b/exec/java-exec/src/test/java/org/apache/drill/test/BaseDirTestWatcher.java
index 9a538d8..7f65750 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/BaseDirTestWatcher.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/BaseDirTestWatcher.java
@@ -73,6 +73,7 @@ public class BaseDirTestWatcher extends DirTestWatcher {
   private File dfsTestTmpParentDir;
   private File dfsTestTmpDir;
   private File rootDir;
+  private File udfDir;
 
   /**
    * Creates a {@link BaseDirTestWatcher} which does not delete it's temp directories at the end of tests.
@@ -103,6 +104,7 @@ public class BaseDirTestWatcher extends DirTestWatcher {
     tmpDir = makeSubDir(Paths.get("tmp"));
     storeDir = makeSubDir(Paths.get("store"));
     dfsTestTmpParentDir = makeSubDir(Paths.get("dfsTestTmp"));
+    udfDir = makeSubDir(Paths.get("udf"));
 
     newDfsTestTmpDir();
   }
@@ -118,6 +120,7 @@ public class BaseDirTestWatcher extends DirTestWatcher {
       FileUtils.cleanDirectory(tmpDir);
       FileUtils.cleanDirectory(storeDir);
       FileUtils.cleanDirectory(dfsTestTmpDir);
+      FileUtils.cleanDirectory(udfDir);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -168,6 +171,14 @@ public class BaseDirTestWatcher extends DirTestWatcher {
   }
 
   /**
+   * Gets the temp directory that should be used as base directory for dynamic UDFs.
+   * @return The temp directory that should be used as base directory for dynamic UDFs.
+   */
+  public File getUdfDir() {
+    return udfDir;
+  }
+
+  /**
    * This methods creates a new directory which can be mapped to <b>dfs.tmp</b>.
    */
   public void newDfsTestTmpDir() {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
index 735c97a..a7e8ef7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
@@ -584,6 +584,7 @@ public class ClusterFixture extends BaseFixture implements AutoCloseable {
     Properties props = new Properties();
     props.putAll(ClusterFixture.TEST_CONFIGURATIONS);
     props.setProperty(ExecConstants.DRILL_TMP_DIR, dirTestWatcher.getTmpDir().getAbsolutePath());
+    props.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getUdfDir().getAbsolutePath());
     props.setProperty(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH, dirTestWatcher.getStoreDir().getAbsolutePath());
 
     builder.configBuilder.configProps(props);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
index b9f4b43..3ded2f8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
@@ -127,6 +127,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
         configBuilder.put(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH, dirTestWatcher.getStoreDir().getAbsolutePath());
         configBuilder.put(ExecConstants.SPILL_DIRS, Lists.newArrayList(dirTestWatcher.getSpillDir().getAbsolutePath()));
         configBuilder.put(ExecConstants.HASHJOIN_SPILL_DIRS, Lists.newArrayList(dirTestWatcher.getSpillDir().getAbsolutePath()));
+        configBuilder.put(ExecConstants.UDF_DIRECTORY_ROOT, dirTestWatcher.getUdfDir().getAbsolutePath());
       }
     }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestGracefulShutdown.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestGracefulShutdown.java
index c5d508b..5e27f75 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestGracefulShutdown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestGracefulShutdown.java
@@ -31,10 +31,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
+import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
-import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.lang.reflect.Field;
 import java.net.HttpURLConnection;
@@ -50,29 +51,30 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 @Category({SlowTest.class})
-public class TestGracefulShutdown extends BaseTestQuery {
+public class TestGracefulShutdown extends ClusterTest {
 
   @Rule
   public final TestRule TIMEOUT = TestTools.getTimeoutRule(120_000);
 
   @BeforeClass
   public static void setUpTestData() throws Exception {
-    for( int i = 0; i < 300; i++) {
+    for (int i = 0; i < 300; i++) {
       setupFile(i);
     }
   }
 
-  private static void enableWebServer(ClusterFixtureBuilder builder) {
-    enableDrillPortHunting(builder);
-    builder.configBuilder.put(ExecConstants.HTTP_ENABLE, true);
-    builder.configBuilder.put(ExecConstants.HTTP_PORT_HUNT, true);
-    builder.sessionOption(ExecConstants.SLICE_TARGET, 10);
+  private static ClusterFixtureBuilder builderWithEnabledWebServer() {
+    return builderWithEnabledPortHunting()
+        .configProperty(ExecConstants.HTTP_ENABLE, true)
+        .configProperty(ExecConstants.HTTP_PORT_HUNT, true)
+        .configProperty(ExecConstants.SLICE_TARGET, 10);
   }
 
-  private static void enableDrillPortHunting(ClusterFixtureBuilder builder) {
-    builder.configBuilder.put(ExecConstants.DRILL_PORT_HUNT, true);
-    builder.configBuilder.put(ExecConstants.GRACE_PERIOD, 500);
-    builder.configBuilder.put(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true);
+  private static ClusterFixtureBuilder builderWithEnabledPortHunting() {
+    return ClusterFixture.builder(dirTestWatcher)
+        .configProperty(ExecConstants.DRILL_PORT_HUNT, true)
+        .configProperty(ExecConstants.GRACE_PERIOD, 500)
+        .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true);
   }
 
   /*
@@ -80,24 +82,25 @@ public class TestGracefulShutdown extends BaseTestQuery {
   endpoints and check if the drillbit still exists.
    */
   @Test
-  public void testOnlineEndPoints() throws  Exception {
+  public void testOnlineEndPoints() throws Exception {
 
     String[] drillbits = {"db1", "db2", "db3"};
-    ClusterFixtureBuilder builder = ClusterFixture.bareBuilder(dirTestWatcher).withLocalZk().withBits(drillbits);
-    enableDrillPortHunting(builder);
+    ClusterFixtureBuilder builder = builderWithEnabledPortHunting()
+        .withLocalZk()
+        .withBits(drillbits);
 
-    try ( ClusterFixture cluster = builder.build()) {
+    try (ClusterFixture cluster = builder.build()) {
 
       Drillbit drillbit = cluster.drillbit("db2");
       int zkRefresh = drillbit.getContext().getConfig().getInt(ExecConstants.ZK_REFRESH);
-      DrillbitEndpoint drillbitEndpoint =  drillbit.getRegistrationHandle().getEndPoint();
+      DrillbitEndpoint drillbitEndpoint = drillbit.getRegistrationHandle().getEndPoint();
       cluster.closeDrillbit("db2");
 
       while (true) {
         Collection<DrillbitEndpoint> drillbitEndpoints = cluster.drillbit()
-                .getContext()
-                .getClusterCoordinator()
-                .getOnlineEndPoints();
+            .getContext()
+            .getClusterCoordinator()
+            .getOnlineEndPoints();
 
         if (!drillbitEndpoints.contains(drillbitEndpoint)) {
           // Success
@@ -116,12 +119,14 @@ public class TestGracefulShutdown extends BaseTestQuery {
   public void testRestApi() throws Exception {
 
     String[] drillbits = {"db1", "db2", "db3"};
-    ClusterFixtureBuilder builder = ClusterFixture.bareBuilder(dirTestWatcher).withLocalZk().withBits(drillbits);
-    enableWebServer(builder);
+    ClusterFixtureBuilder builder = builderWithEnabledWebServer()
+        .withLocalZk()
+        .withBits(drillbits);
+
     QueryBuilder.QuerySummaryFuture listener;
     final String sql = "select * from dfs.root.`.`";
     try (ClusterFixture cluster = builder.build();
-         final ClientFixture client = cluster.clientFixture()) {
+         ClientFixture client = cluster.clientFixture()) {
       Drillbit drillbit = cluster.drillbit("db1");
       int port = drillbit.getWebServerPort();
       int zkRefresh = drillbit.getContext().getConfig().getInt(ExecConstants.ZK_REFRESH);
@@ -133,10 +138,7 @@ public class TestGracefulShutdown extends BaseTestQuery {
         throw new RuntimeException("Failed : HTTP error code : "
                 + conn.getResponseCode());
       }
-      while (true) {
-        if (listener.isDone()) {
-          break;
-        }
+      while (!listener.isDone()) {
         Thread.sleep(100L);
       }
 
@@ -154,8 +156,8 @@ public class TestGracefulShutdown extends BaseTestQuery {
   public void testRestApiShutdown() throws Exception {
 
     String[] drillbits = {"db1", "db2", "db3"};
-    ClusterFixtureBuilder builder = ClusterFixture.bareBuilder(dirTestWatcher).withLocalZk().withBits(drillbits);
-    enableWebServer(builder);
+    ClusterFixtureBuilder builder = builderWithEnabledWebServer().withLocalZk().withBits(drillbits);
+
     QueryBuilder.QuerySummaryFuture listener;
     final String sql = "select * from dfs.root.`.`";
     try (ClusterFixture cluster = builder.build();
@@ -163,12 +165,8 @@ public class TestGracefulShutdown extends BaseTestQuery {
       Drillbit drillbit = cluster.drillbit("db1");
       int port = drillbit.getWebServerPort();
       int zkRefresh = drillbit.getContext().getConfig().getInt(ExecConstants.ZK_REFRESH);
-      listener =  client.queryBuilder().sql(sql).futureSummary();
-      while (true) {
-        if (listener.isDone()) {
-          break;
-        }
-
+      listener = client.queryBuilder().sql(sql).futureSummary();
+      while (!listener.isDone()) {
         Thread.sleep(100L);
       }
       URL url = new URL("http://localhost:" + port + "/shutdown");
@@ -187,7 +185,8 @@ public class TestGracefulShutdown extends BaseTestQuery {
 
   @Test // DRILL-6912
   public void testDrillbitWithSamePortContainsShutdownThread() throws Exception {
-    ClusterFixtureBuilder fixtureBuilder = ClusterFixture.bareBuilder(dirTestWatcher).withLocalZk()
+    ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher)
+        .withLocalZk()
         .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
         .configProperty(ExecConstants.INITIAL_USER_PORT, QueryTestUtil.getFreePortNumber(31170, 300))
         .configProperty(ExecConstants.INITIAL_BIT_PORT, QueryTestUtil.getFreePortNumber(31180, 300));
@@ -213,14 +212,14 @@ public class TestGracefulShutdown extends BaseTestQuery {
 
   @Test // DRILL-7056
   public void testDrillbitTempDir() throws Exception {
-    File originalDrillbitTempDir = null;
-    ClusterFixtureBuilder fixtureBuilder = ClusterFixture.bareBuilder(dirTestWatcher).withLocalZk()
+    File originalDrillbitTempDir;
+    ClusterFixtureBuilder fixtureBuilder = ClusterFixture.builder(dirTestWatcher).withLocalZk()
         .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
         .configProperty(ExecConstants.INITIAL_USER_PORT, QueryTestUtil.getFreePortNumber(31170, 300))
         .configProperty(ExecConstants.INITIAL_BIT_PORT, QueryTestUtil.getFreePortNumber(31180, 300));
 
     try (ClusterFixture fixture = fixtureBuilder.build();
-        Drillbit twinDrillbitOnSamePort = new Drillbit(fixture.config(),
+         Drillbit twinDrillbitOnSamePort = new Drillbit(fixture.config(),
             fixtureBuilder.configBuilder().getDefinitions(), fixture.serviceSet())) {
       // Assert preconditions :
       //      1. First drillbit instance should be started normally
@@ -262,17 +261,18 @@ public class TestGracefulShutdown extends BaseTestQuery {
   }
 
   private static void setupFile(int file_num) throws Exception {
-    final String file = "employee"+file_num+".json";
-    final Path path = dirTestWatcher.getRootDir().toPath().resolve(file);
-    try(PrintWriter out = new PrintWriter(new BufferedWriter(new FileWriter(path.toFile(), true)))) {
-      out.println("{\"employee_id\":1,\"full_name\":\"Sheri Nowmer\",\"first_name\":\"Sheri\",\"last_name\":\"Nowmer\",\"position_id\":1,\"position_title\":\"President\",\"store_id\":0,\"department_id\":1,\"birth_date\":\"1961-08-26\",\"hire_date\":\"1994-12-01 00:00:00.0\",\"end_date\":null,\"salary\":80000.0000,\"supervisor_id\":0,\"education_level\":\"Graduate Degree\",\"marital_status\":\"S\",\"gender\":\"F\",\"management_role\":\"Senior Management\"}\n" +
-              "{\"employee_id\":2,\"full_name\":\"Derrick Whelply\",\"first_name\":\"Derrick\",\"last_name\":\"Whelply\",\"position_id\":2,\"position_title\":\"VP Country Manager\",\"store_id\":0,\"department_id\":1,\"birth_date\":\"1915-07-03\",\"hire_date\":\"1994-12-01 00:00:00.0\",\"end_date\":null,\"salary\":40000.0000,\"supervisor_id\":1,\"education_level\":\"Graduate Degree\",\"marital_status\":\"M\",\"gender\":\"M\",\"management_role\":\"Senior Management\"}\n" +
-              "{\"employee_id\":4,\"full_name\":\"Michael Spence\",\"first_name\":\"Michael\",\"last_name\":\"Spence\",\"position_id\":2,\"position_title\":\"VP Country Manager\",\"store_id\":0,\"department_id\":1,\"birth_date\":\"1969-06-20\",\"hire_date\":\"1998-01-01 00:00:00.0\",\"end_date\":null,\"salary\":40000.0000,\"supervisor_id\":1,\"education_level\":\"Graduate Degree\",\"marital_status\":\"S\",\"gender\":\"M\",\"management_role\":\"Senior Management\"}\n" +
-              "{\"employee_id\":5,\"full_name\":\"Maya Gutierrez\",\"first_name\":\"Maya\",\"last_name\":\"Gutierrez\",\"position_id\":2,\"position_title\":\"VP Country Manager\",\"store_id\":0,\"department_id\":1,\"birth_date\":\"1951-05-10\",\"hire_date\":\"1998-01-01 00:00:00.0\",\"end_date\":null,\"salary\":35000.0000,\"supervisor_id\":1,\"education_level\":\"Bachelors Degree\",\"marital_status\":\"M\",\"gender\":\"F\",\"management_role\":\"Senior Management\"}\n" +
-              "{\"employee_id\":6,\"full_name\":\"Roberta Damstra\",\"first_name\":\"Roberta\",\"last_name\":\"Damstra\",\"position_id\":3,\"position_title\":\"VP Information Systems\",\"store_id\":0,\"department_id\":2,\"birth_date\":\"1942-10-08\",\"hire_date\":\"1994-12-01 00:00:00.0\",\"end_date\":null,\"salary\":25000.0000,\"supervisor_id\":1,\"education_level\":\"Bachelors Degree\",\"marital_status\":\"M\",\"gender\":\"F\",\"management_role\":\"Senior Management\"}\n" +
-              "{\"employee_id\":7,\"full_name\":\"Rebecca Kanagaki\",\"first_name\":\"Rebecca\",\"last_name\":\"Kanagaki\",\"position_id\":4,\"position_title\":\"VP Human Resources\",\"store_id\":0,\"department_id\":3,\"birth_date\":\"1949-03-27\",\"hire_date\":\"1994-12-01 00:00:00.0\",\"end_date\":null,\"salary\":15000.0000,\"supervisor_id\":1,\"education_level\":\"Bachelors Degree\",\"marital_status\":\"M\",\"gender\":\"F\",\"management_role\":\"Senior Management\"}\n");
-    } catch (IOException e) {
-      fail(e.getMessage());
+    String file = "employee" + file_num + ".json";
+    Path path = dirTestWatcher.getRootDir().toPath().resolve(file);
+    StringBuilder stringBuilder = new StringBuilder();
+    int rowsCount = 7;
+    try (BufferedReader reader = new BufferedReader(new InputStreamReader(TestGracefulShutdown.class.getResourceAsStream("/employee.json")))) {
+      for (int i = 0; i < rowsCount; i++) {
+        stringBuilder.append(reader.readLine());
+      }
+    }
+    String content = stringBuilder.toString();
+    try (PrintWriter out = new PrintWriter(new BufferedWriter(new FileWriter(path.toFile(), true)))) {
+      out.println(content);
     }
   }
 


[drill] 05/10: DRILL-7574: Generalize the projection parser

Posted by ag...@apache.org.
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 b0ab3a6cf8552c4f4e6b3256d4c70f0fbe698343
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Fri Feb 7 18:11:02 2020 -0800

    DRILL-7574: Generalize the projection parser
    
    Adds support for multi-dimensional arrays, and columns
    projected as both an array and a map.
    
    closes #1974
---
 .../impl/scan/columns/ColumnsArrayParser.java      |  17 +-
 .../scan/columns/ResolvedColumnsArrayColumn.java   |   2 +-
 .../scan/columns/UnresolvedColumnsArrayColumn.java |   2 +-
 .../impl/scan/file/FileMetadataColumnsParser.java  |  49 +--
 .../impl/scan/file/FileMetadataManager.java        |  23 +-
 .../scan/project/AbstractUnresolvedColumn.java     |   2 +-
 .../scan/project/ExplicitSchemaProjection.java     |  12 +-
 .../scan/project/ReaderSchemaOrchestrator.java     |   9 -
 .../impl/scan/project/ScanLevelProjection.java     |  67 ++-
 .../impl/scan/project/ScanSchemaOrchestrator.java  |  12 +-
 .../scan/project/projSet/AbstractReadColProj.java  |   4 -
 .../scan/project/projSet/EmptyProjectionSet.java   |   3 +
 .../project/projSet/ExplicitProjectionSet.java     |  40 +-
 .../scan/project/projSet/ProjectedDictColumn.java  |  10 +-
 .../scan/project/projSet/ProjectedMapColumn.java   |   2 +-
 .../scan/project/projSet/ProjectedReadColumn.java  |   9 +-
 .../scan/project/projSet/ProjectionChecker.java    | 141 +++++++
 .../scan/project/projSet/ProjectionSetBuilder.java |  26 +-
 .../scan/project/projSet/ProjectionSetFactory.java |   4 +-
 .../project/projSet/WildcardProjectionSet.java     |   3 +
 .../exec/physical/resultSet/ProjectionSet.java     |  13 +-
 .../physical/resultSet/impl/ContainerState.java    |  12 -
 .../resultSet/impl/ResultSetLoaderImpl.java        |  36 +-
 .../exec/physical/resultSet/impl/TupleState.java   |  18 -
 .../project/BaseRequestedColumn.java}              |  40 +-
 .../resultSet/project/ImpliedTupleRequest.java     |  28 +-
 .../physical/resultSet/project/ProjectionType.java | 187 ---------
 .../physical/resultSet/project/Projections.java    | 159 +++++++
 .../exec/physical/resultSet/project/Qualifier.java | 162 ++++++++
 .../project/QualifierContainer.java}               |  15 +-
 .../resultSet/project/RequestedColumn.java         | 239 +++++++++++
 .../resultSet/project/RequestedColumnImpl.java     | 217 ++--------
 .../physical/resultSet/project/RequestedTuple.java |  54 +--
 .../resultSet/project/RequestedTupleImpl.java      | 277 ++++---------
 .../resultSet/project/RequestedWildcardColumn.java |  61 +++
 .../exec/physical/impl/scan/TestColumnsArray.java  |   1 -
 .../impl/scan/TestColumnsArrayFramework.java       |   3 -
 .../physical/impl/scan/TestColumnsArrayParser.java |  29 +-
 .../impl/scan/TestFileMetadataColumnParser.java    | 100 -----
 .../impl/scan/TestScanOrchestratorEarlySchema.java | 110 +----
 .../impl/scan/project/TestScanLevelProjection.java |  64 +--
 .../impl/scan/project/TestSchemaSmoothing.java     |   1 -
 .../scan/project/projSet/TestProjectionSet.java    |  22 -
 .../physical/resultSet/impl/RowSetTestUtils.java   |   7 +-
 .../resultSet/project/TestProjectedPath.java       | 325 +++++++++++++++
 .../resultSet/project/TestProjectedTuple.java      | 457 ++++++++++++---------
 .../resultSet/project/TestProjectionType.java      | 155 -------
 .../easy/text/compliant/TestCsvWithHeaders.java    |  18 +-
 .../easy/text/compliant/TestCsvWithoutHeaders.java |  10 +-
 49 files changed, 1663 insertions(+), 1594 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
index ffd69e6..4a25426 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ColumnsArrayParser.java
@@ -22,8 +22,8 @@ import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedColumnImpl;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.store.easy.text.TextFormatPlugin;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -115,7 +115,7 @@ public class ColumnsArrayParser implements ScanProjectionParser {
 
   @Override
   public boolean parse(RequestedColumn inCol) {
-    if (! requireColumnsArray && ! allowOtherCols) {
+    if (!requireColumnsArray && !allowOtherCols) {
 
       // If we do not require the columns array, then we presume that
       // the reader does not provide arrays, so any use of the columns[x]
@@ -137,14 +137,14 @@ public class ColumnsArrayParser implements ScanProjectionParser {
           new RequestedColumnImpl(builder.rootProjection(), ColumnsScanFramework.COLUMNS_COL));
       return true;
     }
-    if (! inCol.nameEquals(ColumnsScanFramework.COLUMNS_COL)) {
+    if (!inCol.nameEquals(ColumnsScanFramework.COLUMNS_COL)) {
       return false;
     }
 
     // The columns column cannot be a map. That is, the following is
     // not allowed: columns.foo.
 
-    if (inCol.isTuple() && ! allowOtherCols) {
+    if (inCol.isTuple() && !allowOtherCols) {
       throw UserException
         .validationError()
         .message("Column `%s` has map elements, but must be an array", inCol.name())
@@ -175,11 +175,10 @@ public class ColumnsArrayParser implements ScanProjectionParser {
     // Special `columns` array column. Allow multiple, but
     // project only one.
 
-    if (columnsArrayCol != null) {
-      return;
+    if (columnsArrayCol == null) {
+      columnsArrayCol = new UnresolvedColumnsArrayColumn(inCol);
+      builder.addTableColumn(columnsArrayCol);
     }
-    columnsArrayCol = new UnresolvedColumnsArrayColumn(inCol);
-    builder.addTableColumn(columnsArrayCol);
   }
 
   @Override
@@ -197,7 +196,7 @@ public class ColumnsArrayParser implements ScanProjectionParser {
           .addContext(builder.context())
           .build(logger);
       }
-      if (requireColumnsArray && ! allowOtherCols) {
+      if (requireColumnsArray && !allowOtherCols) {
         throw UserException
           .validationError()
           .message("Only `columns` column is allowed. Found: " + col.name())
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
index 2c83065..33a4ccb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/ResolvedColumnsArrayColumn.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.scan.columns;
 
 import org.apache.drill.exec.physical.impl.scan.project.ResolvedTableColumn;
 import org.apache.drill.exec.physical.impl.scan.project.VectorSource;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.MaterializedField;
 
 public class ResolvedColumnsArrayColumn extends ResolvedTableColumn {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
index aba6926..9eb378e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.columns;
 
 import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 
 public class UnresolvedColumnsArrayColumn extends AbstractUnresolvedColumn {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
index b3cf55c..02d4da9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataColumnsParser.java
@@ -25,7 +25,9 @@ import java.util.regex.Pattern;
 import org.apache.drill.exec.physical.impl.scan.project.ColumnProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection;
 import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.ScanProjectionParser;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Parses the implicit file metadata columns out of a project list,
@@ -33,7 +35,7 @@ import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.Requested
  */
 
 public class FileMetadataColumnsParser implements ScanProjectionParser {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileMetadataColumnsParser.class);
+  private static final Logger logger = LoggerFactory.getLogger(FileMetadataColumnsParser.class);
 
   // Internal
 
@@ -46,8 +48,6 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
   private boolean hasImplicitCols;
 
-  private boolean expandPartitionsAtEnd;
-
   public FileMetadataColumnsParser(FileMetadataManager metadataManager) {
     this.metadataManager = metadataManager;
     partitionPattern = Pattern.compile(metadataManager.partitionDesignator + "(\\d+)", Pattern.CASE_INSENSITIVE);
@@ -69,11 +69,6 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
     if (defn != null) {
       return buildMetadataColumn(defn, inCol);
     }
-    if (inCol.isWildcard()) {
-      buildWildcard();
-
-      // Don't consider this a match.
-    }
     return false;
   }
 
@@ -81,15 +76,13 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
     // If the projected column is a map or array, then it shadows the
     // partition column. Example: dir0.x, dir0[2].
-
     if (! inCol.isSimple()) {
       logger.warn("Partition column {} is shadowed by a projected {}",
-          inCol.name(), inCol.summary());
+          inCol.name(), inCol.toString());
       return false;
     }
 
     // Partition column
-
     int partitionIndex = Integer.parseInt(m.group(1));
     if (! referencedPartitions.contains(partitionIndex)) {
       builder.addMetadataColumn(
@@ -98,7 +91,6 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
             partitionIndex));
 
       // Remember the partition for later wildcard expansion
-
       referencedPartitions.add(partitionIndex);
       hasImplicitCols = true;
     }
@@ -110,47 +102,24 @@ public class FileMetadataColumnsParser implements ScanProjectionParser {
 
     // If the projected column is a map or array, then it shadows the
     // metadata column. Example: filename.x, filename[2].
-
     if (! inCol.isSimple()) {
       logger.warn("File metadata column {} is shadowed by a projected {}",
-          inCol.name(), inCol.summary());
+          inCol.name(), inCol.toString());
       return false;
     }
 
     // File metadata (implicit) column
-
     builder.addMetadataColumn(new FileMetadataColumn(inCol.name(), defn));
     hasImplicitCols = true;
     return true;
   }
 
-  private void buildWildcard() {
-    if (!metadataManager.options().useLegacyWildcardExpansion) {
-      return;
-    }
-    if (metadataManager.options().useLegacyExpansionLocation) {
-
-      // Star column: this is a SELECT * query.
-
-      // Old-style wildcard handling inserts all partition columns in
-      // the scanner, removes them in Project.
-      // Fill in the file metadata columns. Can do here because the
-      // set is constant across all files.
-
-      expandPartitions();
-    } else {
-      expandPartitionsAtEnd = true;
-    }
-  }
-
   @Override
   public void validate() {
 
-    // Expand partitions if using a wildcard appears, if using the
-    // feature to expand partitions for wildcards, and we want the
-    // partitions after data columns.
-
-    if (expandPartitionsAtEnd) {
+    // Expand partitions when the projection includes a wildcard
+    // and when "legacy" partition expansion is enabled.
+    if (builder.projectAll() && metadataManager.options().useLegacyWildcardExpansion) {
       expandPartitions();
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
index 330a2ab..e859c46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/file/FileMetadataManager.java
@@ -60,14 +60,12 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * <p>
  * This is the successor to {@link org.apache.drill.exec.store.ColumnExplorer}.
  */
-
 public class FileMetadataManager implements MetadataManager, ReaderProjectionResolver, VectorSource {
 
   /**
    * Automatically compute partition depth from files. Use only
    * for testing!
    */
-
   public static final int AUTO_PARTITION_DEPTH = -1;
 
   public static class FileMetadataOptions {
@@ -75,8 +73,12 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
     private Path rootDir;
     private int partitionCount = AUTO_PARTITION_DEPTH;
     private List<Path> files;
+
+    /**
+     * Historically Drill will expand parition columns (dir0, dir1, ...)
+     * when the project list includes a wildcard.
+     */
     protected boolean useLegacyWildcardExpansion = true;
-    protected boolean useLegacyExpansionLocation;
 
     /**
       * Specify the selection root for a directory scan, if any.
@@ -113,21 +115,6 @@ public class FileMetadataManager implements MetadataManager, ReaderProjectionRes
      public void useLegacyWildcardExpansion(boolean flag) {
        useLegacyWildcardExpansion = flag;
      }
-
-     /**
-      * In legacy mode, above, Drill expands partition columns whenever the
-      * wildcard appears. Drill 1.1 - 1.11 put expanded partition columns after
-      * data columns. This is actually a better position as it minimizes changes
-      * the row layout for files at different depths. Drill 1.12 moved them before
-      * data columns: at the location of the wildcard.
-      * <p>
-      * This flag, when set, uses the Drill 1.12 position. Later enhancements
-      * can unset this flag to go back to the future: use the preferred location
-      * after other columns.
-      */
-     public void useLegacyExpansionLocation(boolean flag) {
-       useLegacyExpansionLocation = flag;
-     }
   }
 
   // Input
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
index 9e40659..78bc388 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/AbstractUnresolvedColumn.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.scan.project;
 
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
index 8975a0b..fc91b38 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ExplicitSchemaProjection.java
@@ -22,12 +22,14 @@ import java.util.List;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.complex.DictVector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Perform a schema projection for the case of an explicit list of
@@ -43,7 +45,7 @@ import org.apache.drill.exec.vector.complex.DictVector;
  */
 
 public class ExplicitSchemaProjection extends ReaderLevelProjection {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExplicitSchemaProjection.class);
+  private static final Logger logger = LoggerFactory.getLogger(ExplicitSchemaProjection.class);
 
   private final ScanLevelProjection scanProj;
 
@@ -148,7 +150,7 @@ public class ExplicitSchemaProjection extends ReaderLevelProjection {
 
     ResolvedMapColumn mapCol = new ResolvedMapColumn(outputTuple,
         column.schema(), sourceIndex);
-    resolveTuple(mapCol.members(), requestedCol.mapProjection(),
+    resolveTuple(mapCol.members(), requestedCol.tuple(),
         column.tupleSchema());
 
     // If the projection is simple, then just project the map column
@@ -193,7 +195,7 @@ public class ExplicitSchemaProjection extends ReaderLevelProjection {
     }
 
     ResolvedDictColumn dictColumn = new ResolvedDictColumn(outputTuple, column.schema(), sourceIndex);
-    resolveDictTuple(dictColumn.members(), requestedCol.mapProjection(), column.tupleSchema());
+    resolveDictTuple(dictColumn.members(), requestedCol.tuple(), column.tupleSchema());
 
     // The same as for Map
     if (dictColumn.members().isSimpleProjection()) {
@@ -301,7 +303,7 @@ public class ExplicitSchemaProjection extends ReaderLevelProjection {
   private ResolvedColumn resolveMapMembers(ResolvedTuple outputTuple, RequestedColumn col) {
     ResolvedMapColumn mapCol = new ResolvedMapColumn(outputTuple, col.name());
     ResolvedTuple members = mapCol.members();
-    for (RequestedColumn child : col.mapProjection().projections()) {
+    for (RequestedColumn child : col.tuple().projections()) {
       if (child.isTuple()) {
         members.add(resolveMapMembers(members, child));
       } else {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
index 1460a85..99356e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ReaderSchemaOrchestrator.java
@@ -34,7 +34,6 @@ import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTes
  * scan operator manages. Vectors are reused across readers, but via a vector
  * cache. All other state is distinct between readers.
  */
-
 public class ReaderSchemaOrchestrator implements VectorSource {
 
   private final ScanSchemaOrchestrator scanOrchestrator;
@@ -48,7 +47,6 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    * schema changes in this output batch by absorbing trivial schema changes
    * that occur across readers.
    */
-
   private ResolvedRow rootTuple;
   private VectorContainer tableContainer;
 
@@ -88,7 +86,6 @@ public class ReaderSchemaOrchestrator implements VectorSource {
     options.setSchema(readerSchema);
 
     // Create the table loader
-
     tableLoader = new ResultSetLoaderImpl(scanOrchestrator.allocator, options.build());
     return tableLoader;
   }
@@ -111,22 +108,18 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    * to the output batch. First, build the metadata and/or null columns for the
    * table row count. Then, merge the sources.
    */
-
   public void endBatch() {
 
     // Get the batch results in a container.
-
     tableContainer = tableLoader.harvest();
 
     // If the schema changed, set up the final projection based on
     // the new (or first) schema.
-
     if (prevTableSchemaVersion < tableLoader.schemaVersion()) {
       reviseOutputProjection();
     } else {
 
       // Fill in the null and metadata columns.
-
       populateNonDataColumns();
     }
     rootTuple.setRowCount(tableContainer.getRecordCount());
@@ -147,7 +140,6 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    * only need be done if null columns were created when mapping from a prior
    * schema.
    */
-
   private void reviseOutputProjection() {
 
     // Do the table-schema level projection; the final matching
@@ -223,7 +215,6 @@ public class ReaderSchemaOrchestrator implements VectorSource {
    *
    * @param tableSchema newly arrived schema
    */
-
   private void doExplicitProjection(TupleMetadata tableSchema) {
     rootTuple = newRootTuple();
     new ExplicitSchemaProjection(scanOrchestrator.scanProj,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
index b4be33b..a117e7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanLevelProjection.java
@@ -26,9 +26,9 @@ import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn
 import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn.UnresolvedWildcardColumn;
 import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionSetBuilder;
 import org.apache.drill.exec.physical.resultSet.project.ImpliedTupleRequest;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTupleImpl;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
@@ -211,7 +211,6 @@ public class ScanLevelProjection {
    * the columns are table columns. The add-on parser can tag
    * columns as special, such as to hold metadata.
    */
-
   public interface ScanProjectionParser {
     void bind(ScanLevelProjection builder);
     boolean parse(RequestedColumn inCol);
@@ -222,8 +221,9 @@ public class ScanLevelProjection {
 
   public static class Builder {
     private List<SchemaPath> projectionList;
-    private List<ScanProjectionParser> parsers = new ArrayList<>();
+    private final List<ScanProjectionParser> parsers = new ArrayList<>();
     private TupleMetadata outputSchema;
+
     /**
      * Context used with error messages.
      */
@@ -301,16 +301,14 @@ public class ScanLevelProjection {
    * Projection definition for the scan a whole. Parsed form of the input
    * projection list.
    */
-
   protected RequestedTuple outputProjection;
 
   /**
    * Projection definition passed to each reader. This is the set of
    * columns that the reader is asked to provide.
    */
-
   protected RequestedTuple readerProjection;
-  protected ScanProjectionType projectionType = ScanProjectionType.EMPTY;
+  protected ScanProjectionType projectionType;
 
   private ScanLevelProjection(Builder builder) {
     this.projectionList = builder.projectionList();
@@ -351,23 +349,25 @@ public class ScanLevelProjection {
   }
 
   private void doParse() {
-    outputProjection = RequestedTupleImpl.parse(projectionList);
-
-    for (ScanProjectionParser parser : parsers) {
-      parser.bind(this);
-    }
-
-    // First pass: check if a wildcard exists.
-
-    for (RequestedColumn inCol : outputProjection.projections()) {
-      if (inCol.isWildcard()) {
+    outputProjection = Projections.parse(projectionList);
+    switch (outputProjection.type()) {
+      case ALL:
         includesWildcard = true;
+        projectionType = ScanProjectionType.WILDCARD;
+        break;
+      case NONE:
+        projectionType = ScanProjectionType.EMPTY;
+        break;
+      default:
+        projectionType = ScanProjectionType.EXPLICIT;
         break;
-      }
     }
 
-    // Second pass: process projected columns.
+    for (ScanProjectionParser parser : parsers) {
+      parser.bind(this);
+    }
 
+    // Process projected columns.
     for (RequestedColumn inCol : outputProjection.projections()) {
       if (inCol.isWildcard()) {
         mapWildcard(inCol);
@@ -393,19 +393,18 @@ public class ScanLevelProjection {
     // projection. With a schema, we want the schema columns (which may
     // or may not correspond to reader columns.)
 
-    if (projectionType != ScanProjectionType.EMPTY &&
-        projectionType != ScanProjectionType.EXPLICIT) {
-
+    if (projectionType == ScanProjectionType.EMPTY) {
+      readerProjection = ImpliedTupleRequest.NO_MEMBERS;
+    } else if (projectionType != ScanProjectionType.EXPLICIT) {
       readerProjection = ImpliedTupleRequest.ALL_MEMBERS;
     } else {
-
       List<RequestedColumn> outputProj = new ArrayList<>();
       for (ColumnProjection col : outputCols) {
         if (col instanceof AbstractUnresolvedColumn) {
           outputProj.add(((AbstractUnresolvedColumn) col).element());
         }
       }
-      readerProjection = RequestedTupleImpl.build(outputProj);
+      readerProjection = Projections.build(outputProj);
     }
   }
 
@@ -414,29 +413,25 @@ public class ScanLevelProjection {
    * columns that are needed. The order is important: we want custom
    * columns to follow table columns.
    */
-
   private void mapWildcard(RequestedColumn inCol) {
 
     // Wildcard column: this is a SELECT * query.
-
     assert includesWildcard;
     if (sawWildcard) {
       throw new IllegalArgumentException("Duplicate * entry in project list");
     }
 
     // Expand strict schema columns, if provided
-
+    assert projectionType == ScanProjectionType.WILDCARD;
     boolean expanded = expandOutputSchema();
 
     // Remember the wildcard position, if we need to insert it.
     // Ensures that the main wildcard expansion occurs before add-on
     // columns.
-
     int wildcardPosn = outputCols.size();
 
     // Parsers can consume the wildcard. But, all parsers must
     // have visibility to the wildcard column.
-
     for (ScanProjectionParser parser : parsers) {
       if (parser.parse(inCol)) {
         wildcardPosn = -1;
@@ -444,12 +439,10 @@ public class ScanLevelProjection {
     }
 
     // Set this flag only after the parser checks.
-
     sawWildcard = true;
 
     // If not consumed, put the wildcard column into the projection list as a
     // placeholder to be filled in later with actual table columns.
-
     if (expanded) {
       projectionType =
           outputSchema.booleanProperty(TupleMetadata.IS_STRICT_SCHEMA_PROP)
@@ -457,7 +450,6 @@ public class ScanLevelProjection {
           : ScanProjectionType.SCHEMA_WILDCARD;
     } else if (wildcardPosn != -1) {
       outputCols.add(wildcardPosn, new UnresolvedWildcardColumn(inCol));
-      projectionType = ScanProjectionType.WILDCARD;
     }
   }
 
@@ -476,7 +468,6 @@ public class ScanLevelProjection {
 
       // Skip columns tagged as "special"; those that should not expand
       // automatically.
-
       if (col.booleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD)) {
         continue;
       }
@@ -502,7 +493,6 @@ public class ScanLevelProjection {
    *
    * @param inCol the SELECT column
    */
-
   private void mapColumn(RequestedColumn inCol) {
 
     // Give the extensions first crack at each column.
@@ -525,7 +515,6 @@ public class ScanLevelProjection {
     }
 
     // This is a desired table column.
-
     addTableColumn(inCol);
   }
 
@@ -539,7 +528,6 @@ public class ScanLevelProjection {
 
   public void addTableColumn(ColumnProjection outCol) {
     outputCols.add(outCol);
-    projectionType = ScanProjectionType.EXPLICIT;
   }
 
   public void addMetadataColumn(ColumnProjection outCol) {
@@ -552,17 +540,14 @@ public class ScanLevelProjection {
    * add-on parser is given an opportunity to do its own
    * validation.
    */
-
   private void verify() {
 
     // Let parsers do overall validation.
-
     for (ScanProjectionParser parser : parsers) {
       parser.validate();
     }
 
     // Validate column-by-column.
-
     for (ColumnProjection outCol : outputCols) {
       for (ScanProjectionParser parser : parsers) {
         parser.validateColumn(outCol);
@@ -576,7 +561,6 @@ public class ScanLevelProjection {
    * Return the set of columns from the SELECT list
    * @return the SELECT list columns, in SELECT list order
    */
-
   public List<SchemaPath> requestedCols() { return projectionList; }
 
   /**
@@ -585,7 +569,6 @@ public class ScanLevelProjection {
    * table order (for SELECT * queries).
    * @return the set of output columns in output order
    */
-
   public List<ColumnProjection> columns() { return outputCols; }
 
   public ScanProjectionType projectionType() { return projectionType; }
@@ -594,7 +577,6 @@ public class ScanLevelProjection {
    * Return whether this is a SELECT * query
    * @return true if this is a SELECT * query
    */
-
   public boolean projectAll() { return projectionType.isWildcard(); }
 
   /**
@@ -607,7 +589,6 @@ public class ScanLevelProjection {
    * if at least one column is projected (or the query contained
    * the wildcard)
    */
-
   public boolean isEmptyProjection() { return projectionType == ScanProjectionType.EMPTY; }
 
   public RequestedTuple rootProjection() { return outputProjection; }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
index edc6acf..5626d1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/ScanSchemaOrchestrator.java
@@ -165,12 +165,12 @@ public class ScanSchemaOrchestrator {
     private MetadataManager metadataManager;
     private int scanBatchRecordLimit = DEFAULT_BATCH_ROW_COUNT;
     private int scanBatchByteLimit = DEFAULT_BATCH_BYTE_COUNT;
-    private List<ScanProjectionParser> parsers = new ArrayList<>();
-    private List<ReaderProjectionResolver> schemaResolvers = new ArrayList<>();
+    private final List<ScanProjectionParser> parsers = new ArrayList<>();
+    private final List<ReaderProjectionResolver> schemaResolvers = new ArrayList<>();
     private boolean useSchemaSmoothing;
     private boolean allowRequiredNullColumns;
     private List<SchemaPath> projection;
-    private TypeConverter.Builder typeConverterBuilder = TypeConverter.builder();
+    private final TypeConverter.Builder typeConverterBuilder = TypeConverter.builder();
 
     /**
      * Option that enables whether the scan operator starts with an empty
@@ -422,9 +422,9 @@ public class ScanSchemaOrchestrator {
 
     ScanProjectionParser parser = metadataManager.projectionParser();
     if (parser != null) {
-      // Insert in first position so that it is ensured to see
-      // any wildcard that exists
-      options.parsers.add(0, parser);
+      // Insert in last position to expand wildcards at
+      // the end of the tuple.
+      options.parsers.add(parser);
     }
 
     // Parse the projection list.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java
index d7bde8f..d3fc2ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/AbstractReadColProj.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet.ColumnReadProjection;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 
@@ -44,7 +43,4 @@ public abstract class AbstractReadColProj implements ColumnReadProjection {
 
   @Override
   public ProjectionSet mapProjection() { return ProjectionSetFactory.projectAll(); }
-
-  @Override
-  public ProjectionType projectionType() { return null; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java
index 7fdec52..d02a04c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/EmptyProjectionSet.java
@@ -45,4 +45,7 @@ public class EmptyProjectionSet implements ProjectionSet {
 
   @Override
   public boolean isEmpty() { return true; }
+
+  @Override
+  public boolean isProjected(String colName) { return false; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java
index daf2f1a..aace7e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ExplicitProjectionSet.java
@@ -17,19 +17,14 @@
  */
 package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedColumnImpl;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 import org.apache.drill.exec.vector.complex.DictVector;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Projection set based on an explicit set of columns provided
@@ -38,7 +33,6 @@ import org.slf4j.LoggerFactory;
  */
 
 public class ExplicitProjectionSet extends AbstractProjectionSet {
-  private static final Logger logger = LoggerFactory.getLogger(ExplicitProjectionSet.class);
 
   private final RequestedTuple requestedProj;
 
@@ -48,6 +42,11 @@ public class ExplicitProjectionSet extends AbstractProjectionSet {
   }
 
   @Override
+  public boolean isProjected(String colName) {
+    return requestedProj.get(colName) != null;
+  }
+
+  @Override
   public ColumnReadProjection readProjection(ColumnMetadata col) {
     RequestedColumn reqCol = requestedProj.get(col.name());
     if (reqCol == null) {
@@ -59,7 +58,7 @@ public class ExplicitProjectionSet extends AbstractProjectionSet {
 
   private ColumnReadProjection getReadProjection(ColumnMetadata col, RequestedColumn reqCol) {
     ColumnMetadata outputSchema = outputSchema(col);
-    validateProjection(reqCol, outputSchema == null ? col : outputSchema);
+    ProjectionChecker.validateProjection(reqCol, outputSchema == null ? col : outputSchema, errorContext);
     if (!col.isMap() && !col.isDict()) {
 
       // Non-map column.
@@ -73,7 +72,7 @@ public class ExplicitProjectionSet extends AbstractProjectionSet {
 
       TypeConverter childConverter = childConverter(outputSchema);
       ProjectionSet mapProjection;
-      if (! reqCol.type().isTuple() || reqCol.mapProjection().type() == TupleProjectionType.ALL) {
+      if (! reqCol.isTuple() || reqCol.tuple().type() == TupleProjectionType.ALL) {
 
         // Projection is simple: "m". This is equivalent to
         // (non-SQL) m.*
@@ -88,7 +87,7 @@ public class ExplicitProjectionSet extends AbstractProjectionSet {
         // projected; that case, while allowed in the RequestedTuple
         // implementation, can never occur in a SELECT list.)
 
-        mapProjection = new ExplicitProjectionSet(reqCol.mapProjection(), childConverter);
+        mapProjection = new ExplicitProjectionSet(reqCol.tuple(), childConverter);
       }
       if (col.isMap()) {
         return new ProjectedMapColumn(col, reqCol, outputSchema, mapProjection);
@@ -98,27 +97,6 @@ public class ExplicitProjectionSet extends AbstractProjectionSet {
     }
   }
 
-  public void validateProjection(RequestedColumn colReq, ColumnMetadata readCol) {
-    if (colReq == null || readCol == null) {
-      return;
-    }
-    ProjectionType type = colReq.type();
-    if (type == null) {
-      return;
-    }
-    ProjectionType neededType = ProjectionType.typeFor(readCol.majorType());
-    if (type.isCompatible(neededType)) {
-      return;
-    }
-    throw UserException.validationError()
-      .message("Column type not compatible with projection specification")
-      .addContext("Column:", readCol.name())
-      .addContext("Projection type:", type.label())
-      .addContext("Column type:", Types.getSqlTypeName(readCol.majorType()))
-      .addContext(errorContext)
-      .build(logger);
-  }
-
   @Override
   public ColumnReadProjection readDictProjection(ColumnMetadata col) {
     // Unlike for a MAP, requestedProj contains a key value, rather than nested field's name:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java
index 9f4eecf..6805a0e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedDictColumn.java
@@ -18,8 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 public class ProjectedDictColumn extends ProjectedReadColumn {
@@ -27,7 +26,7 @@ public class ProjectedDictColumn extends ProjectedReadColumn {
   private final ProjectionSet tupleProjection;
 
   public ProjectedDictColumn(ColumnMetadata readSchema,
-                            RequestedTuple.RequestedColumn requestedCol, ColumnMetadata outputSchema,
+                            RequestedColumn requestedCol, ColumnMetadata outputSchema,
                             ProjectionSet tupleProjection) {
     super(readSchema, requestedCol, outputSchema, null);
     this.tupleProjection = tupleProjection;
@@ -37,9 +36,4 @@ public class ProjectedDictColumn extends ProjectedReadColumn {
   public ProjectionSet mapProjection() {
     return tupleProjection;
   }
-
-  @Override
-  public ProjectionType projectionType() {
-    return super.projectionType().isArray() ? ProjectionType.DICT_ARRAY : ProjectionType.ARRAY;
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
index da38550..f29ca31 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 
 public class ProjectedMapColumn extends ProjectedReadColumn {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java
index ddc9002..834b657 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedReadColumn.java
@@ -18,8 +18,7 @@
 package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 
@@ -30,6 +29,7 @@ import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
  */
 
 public class ProjectedReadColumn extends AbstractReadColProj {
+  @SuppressWarnings("unused")
   private final RequestedColumn requestedCol;
   private final ColumnMetadata outputSchema;
   private final ColumnConversionFactory conversionFactory;
@@ -69,10 +69,5 @@ public class ProjectedReadColumn extends AbstractReadColProj {
   }
 
   @Override
-  public ProjectionType projectionType() {
-    return requestedCol == null ? null : requestedCol.type();
-  }
-
-  @Override
   public ColumnConversionFactory conversionFactory() { return conversionFactory; }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java
new file mode 100644
index 0000000..8757167
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionChecker.java
@@ -0,0 +1,141 @@
+/*
+ * 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.physical.impl.scan.project.projSet;
+
+import org.apache.drill.common.exceptions.CustomErrorContext;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to check if a column is consistent with the projection
+ * requested for a query. Used for scans: the reader offers certain columns
+ * and the scan operator must decide whether to accept them, and if so,
+ * if the column that has actually appeared is consistent with the projection
+ * schema path provided by the planner. An obvious example is if projection
+ * asks for {@code a[0]} (and array), but the reader offer up {@code a}
+ * as a non-array column.
+ * <p>
+ * Checks are reasonable, but not complete. Particularly in the {@code DICT}
+ * case, projection depends on multiple factors, such as the type of the
+ * key and values. This class does not (yet) handle that complexity.
+ * Instead, the goal is no false negatives for the complex cases, while
+ * catching the simple cases.
+ * <p>
+ * The Project operator or other consuming operator is the final arbitrator
+ * of whether a particular column satisfies a particular projection. This
+ * class tries to catch those errors early to provide better error
+ * messages.
+ */
+public class ProjectionChecker {
+  private static final Logger logger = LoggerFactory.getLogger(ProjectionChecker.class);
+
+  private ProjectionChecker() { }
+
+  /**
+   * Check if the given read column is consistent with the projection requested for
+   * a tuple. This form handles wildcard projection and unprojected columns; cases
+   * where there is no column-level projection information.
+   *
+   * @param tuple the tuple-level projection description
+   * @param readCol metadata for the column which the reader has actually
+   * produced
+   * @return {@code true} if the column is consistent with projection (or if the
+   * column is too complex to check), {@code false} if the column is not
+   * consistent and represents an error case. Also returns {@code true} if
+   * the column is not projected, as any type of column can be ignored
+   */
+  public static boolean isConsistent(RequestedTuple tuple, ColumnMetadata readCol) {
+    if (tuple == null || !tuple.isProjected(readCol.name())) {
+      return true;
+    }
+    // If the column is projected, it may be projected implicitly.
+    // Only check explicit projection.
+    RequestedColumn col = tuple.get(readCol.name());
+    if (col == null) {
+      return true;
+    } else {
+      return isConsistent(col, readCol);
+    }
+  }
+
+  /**
+   * Check if the given read column is consistent with the projection requested for
+   * that column. Does not handle subtleties such as DICT key types, actual types
+   * in a UNION, etc.
+   *
+   * @param colReq the column-level projection description
+   * @param readCol metadata for the column which the reader has actually
+   * produced
+   * @return {@code true} if the column is consistent with projection (or if the
+   * column is too complex to check), {@code false} if the column is not
+   * consistent and represents an error case. Also returns {@code true} if
+   * the column is not projected, as any type of column can be ignored
+   */
+  public static boolean isConsistent(RequestedColumn colReq, ColumnMetadata readCol) {
+    if (colReq == null || readCol == null) {
+      return true;
+    }
+    if (colReq.isTuple() && !(readCol.isMap() || readCol.isDict() || readCol.isVariant())) {
+      return false;
+    }
+    if (colReq.isArray()) {
+      if (colReq.arrayDims() == 1) {
+        return readCol.isArray() || readCol.isDict() || readCol.isVariant();
+      } else {
+        return readCol.type() == MinorType.LIST || readCol.isDict() || readCol.isVariant();
+      }
+    }
+    return true;
+  }
+
+  public static void validateProjection(RequestedColumn colReq, ColumnMetadata readCol) {
+    validateProjection(colReq, readCol, null);
+  }
+
+  /**
+   * Perform the column-level projection as described in
+   * {@link #isConsistent(RequestedColumn, ColumnMetadata)}, and raise a
+   * {@code UserException} if the column is not consistent with projection.
+   *
+   * @param colReq the column-level projection description
+   * @param readCol metadata for the column which the reader has actually
+   * produced
+   * @param errorContext additional error context to pass along in the
+   * exception
+   * @throws UserException if the read column is not consistent with the
+   * projection description for the column
+   */
+  public static void validateProjection(RequestedColumn colReq, ColumnMetadata readCol,
+      CustomErrorContext errorContext) {
+    if (!isConsistent(colReq, readCol)) {
+      throw UserException.validationError()
+        .message("Column type not compatible with projection specification")
+        .addContext("Column:", readCol.name())
+        .addContext("Projection type:", colReq.toString())
+        .addContext("Column type:", Types.getSqlTypeName(readCol.majorType()))
+        .addContext(errorContext)
+        .build(logger);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java
index f1a48d6..6f788b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetBuilder.java
@@ -22,9 +22,9 @@ import java.util.Collection;
 import org.apache.drill.common.exceptions.CustomErrorContext;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTupleImpl;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
 public class ProjectionSetBuilder {
@@ -52,7 +52,7 @@ public class ProjectionSetBuilder {
     if (projection == null) {
       parsedProjection = null;
     } else {
-      parsedProjection = RequestedTupleImpl.parse(projection);
+      parsedProjection = Projections.parse(projection);
     }
     return this;
   }
@@ -83,17 +83,17 @@ public class ProjectionSetBuilder {
 
     ProjectionSet projSet;
     switch (projType) {
-    case ALL:
-      projSet = new WildcardProjectionSet(typeConverter);
-      break;
-    case NONE:
-      projSet = ProjectionSetFactory.projectNone();
-      break;
-    case SOME:
-      projSet = new ExplicitProjectionSet(parsedProjection, typeConverter);
-      break;
-    default:
-      throw new IllegalStateException("Unexpected projection type: " + projType.toString());
+      case ALL:
+        projSet = new WildcardProjectionSet(typeConverter);
+        break;
+      case NONE:
+        projSet = ProjectionSetFactory.projectNone();
+        break;
+      case SOME:
+        projSet = new ExplicitProjectionSet(parsedProjection, typeConverter);
+        break;
+      default:
+        throw new IllegalStateException("Unexpected projection type: " + projType.toString());
     }
     projSet.setErrorContext(errorContext);
     return projSet;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java
index f5642cb..46e1b94 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectionSetFactory.java
@@ -23,8 +23,8 @@ import java.util.Map;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.impl.scan.project.projSet.TypeConverter.CustomTypeTransform;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
+import org.apache.drill.exec.physical.resultSet.project.Projections;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTupleImpl;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
 import org.apache.drill.exec.vector.accessor.convert.StandardConversions.ConversionDefn;
@@ -70,7 +70,7 @@ public class ProjectionSetFactory {
     if (selection == null) {
       return projectAll();
     }
-    return wrap(RequestedTupleImpl.parse(selection));
+    return wrap(Projections.parse(selection));
   }
 
   public static CustomTypeTransform simpleTransform(ColumnConversionFactory colFactory) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java
index 1da5a2b..f7d216f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/WildcardProjectionSet.java
@@ -31,6 +31,9 @@ public class WildcardProjectionSet extends AbstractProjectionSet {
   }
 
   @Override
+  public boolean isProjected(String colName) { return true; }
+
+  @Override
   public ColumnReadProjection readProjection(ColumnMetadata col) {
     if (isSpecial(col)) {
       return new UnprojectedReadColumn(col);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java
index 208defd..81b6a24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/ProjectionSet.java
@@ -18,10 +18,8 @@
 package org.apache.drill.exec.physical.resultSet;
 
 import org.apache.drill.common.exceptions.CustomErrorContext;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.convert.ColumnConversionFactory;
-import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Provides a dynamic, run-time view of a projection set. Used by
@@ -74,7 +72,6 @@ public interface ProjectionSet {
    * indicates whether a reader column is projected, and if so, the attributes
    * of that projection.
    */
-
   public interface ColumnReadProjection {
 
     /**
@@ -83,23 +80,15 @@ public interface ProjectionSet {
      * from an explicit projection, or columns within a wildcard projection
      * where the column is "special" and is not expanded in the wildcard.
      */
-
     boolean isProjected();
 
     ColumnMetadata readSchema();
     ColumnMetadata providedSchema();
     ColumnConversionFactory conversionFactory();
     ProjectionSet mapProjection();
-
-    /**
-     * The projection type from the parse of the projection list,
-     * if available. Used for testing only. Don't use this in production
-     * code, let this class do the checks itself.
-     */
-    @VisibleForTesting
-    ProjectionType projectionType();
   }
 
+  boolean isProjected(String colName);
   void setErrorContext(CustomErrorContext errorContext);
   ColumnReadProjection readProjection(ColumnMetadata col);
   ColumnReadProjection readDictProjection(ColumnMetadata col);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
index 3881891..fde2844 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ContainerState.java
@@ -40,7 +40,6 @@ import org.apache.drill.exec.record.metadata.ColumnMetadata;
  * <li>A column state which orchestrates the above three items.</li>
  * <ul>
  */
-
 public abstract class ContainerState {
 
   protected final LoaderInternals loader;
@@ -51,7 +50,6 @@ public abstract class ContainerState {
    * Vector cache for this loader.
    * @see {@link OptionBuilder#setVectorCache()}.
    */
-
   protected final ResultVectorCache vectorCache;
 
   public ContainerState(LoaderInternals loader, ResultVectorCache vectorCache, ProjectionSet projectionSet) {
@@ -79,7 +77,6 @@ public abstract class ContainerState {
    *
    * @return <tt>true</tt> if versioned
    */
-
   protected abstract boolean isVersioned();
 
   protected LoaderInternals loader() { return loader; }
@@ -89,19 +86,15 @@ public abstract class ContainerState {
   public ColumnState addColumn(ColumnMetadata columnSchema) {
 
     // Create the vector, writer and column state
-
     ColumnState colState = loader.columnBuilder().buildColumn(this, columnSchema);
 
     // Add the column to this container
-
     addColumn(colState);
 
     // Set initial cardinality
-
     colState.updateCardinality(innerCardinality());
 
     // Allocate vectors if a batch is in progress.
-
     if (loader().writeable()) {
       colState.allocateVectors();
     }
@@ -116,7 +109,6 @@ public abstract class ContainerState {
    * this value is recursively pushed downward to compute the cardinality
    * of lists of maps that contains lists of maps, and so on.
    */
-
   public void updateCardinality() {
     int innerCardinality = innerCardinality();
     assert innerCardinality > 0;
@@ -129,7 +121,6 @@ public abstract class ContainerState {
    * Start a new batch by shifting the overflow buffers back into the main
    * write vectors and updating the writers.
    */
-
   public void startBatch(boolean schemaOnly) {
     for (ColumnState colState : columnStates()) {
       colState.startBatch(schemaOnly);
@@ -143,7 +134,6 @@ public abstract class ContainerState {
    * for some previous row, depending on exactly when and where the overflow
    * occurs.
    */
-
   public void rollover() {
     for (ColumnState colState : columnStates()) {
       colState.rollover();
@@ -155,7 +145,6 @@ public abstract class ContainerState {
    * vector for harvesting to send downstream. Set aside the look-ahead vector
    * and put the full vector buffer back into the active vector.
    */
-
   public void harvestWithLookAhead() {
     for (ColumnState colState : columnStates()) {
       colState.harvestWithLookAhead();
@@ -166,7 +155,6 @@ public abstract class ContainerState {
    * Clean up state (such as backup vectors) associated with the state
    * for each vector.
    */
-
   public void close() {
     for (ColumnState colState : columnStates()) {
       colState.close();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
index 6c27706..c0695fa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/ResultSetLoaderImpl.java
@@ -39,13 +39,11 @@ import org.slf4j.LoggerFactory;
  *
  * @see {@link ResultSetLoader}
  */
-
 public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
 
   /**
    * Read-only set of options for the result set loader.
    */
-
   public static class ResultSetOptions {
     protected final int vectorSizeLimit;
     protected final int rowCountLimit;
@@ -95,13 +93,11 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
     /**
      * Before the first batch.
      */
-
     START,
 
     /**
      * Writing to a batch normally.
      */
-
     ACTIVE,
 
     /**
@@ -109,14 +105,12 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
      * to write to a temporary "overflow" batch until the
      * end of the current row.
      */
-
     OVERFLOW,
 
     /**
      * Temporary state to avoid batch-size related overflow while
      * an overflow is in progress.
      */
-
     IN_OVERFLOW,
 
     /**
@@ -124,14 +118,12 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
      * when saving a row.
      * No more writes allowed until harvesting the current batch.
      */
-
     FULL_BATCH,
 
     /**
      * Current batch was harvested: data is gone. No lookahead
      * batch exists.
      */
-
     HARVESTED,
 
     /**
@@ -155,13 +147,11 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
      * fine. The correct buffers are restored once a new batch is started
      * and the state moves to ACTIVE.
      */
-
     LOOK_AHEAD,
 
     /**
      * Mutator is closed: no more operations are allowed.
      */
-
     CLOSED
   }
 
@@ -170,26 +160,22 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   /**
    * Options provided to this loader.
    */
-
   private final ResultSetOptions options;
 
   /**
    * Allocator for vectors created by this loader.
    */
-
   private final BufferAllocator allocator;
 
   /**
    * Builds columns (vector, writer, state).
    */
-
   private final ColumnBuilder columnBuilder;
 
   /**
    * Internal structure used to work with the vectors (real or dummy) used
    * by this loader.
    */
-
   private final RowState rootState;
 
   /**
@@ -200,21 +186,18 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * within the writer structure that points to the current position within
    * an array column.
    */
-
   private final WriterIndexImpl writerIndex;
 
   /**
    * The row-level writer for stepping through rows as they are written,
    * and for accessing top-level columns.
    */
-
   private final RowSetLoaderImpl rootWriter;
 
   /**
    * Tracks the state of the row set loader. Handling vector overflow requires
    * careful stepping through a variety of states as the write proceeds.
    */
-
   private State state = State.START;
 
   /**
@@ -223,7 +206,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * This allows very easy checks for schema changes: save the prior version number
    * and compare it against the current version number.
    */
-
   private int activeSchemaVersion;
 
   /**
@@ -234,21 +216,18 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * sees the schema as it existed at a prior version: the harvest schema
    * version.
    */
-
   private int harvestSchemaVersion;
 
   /**
    * Counts the batches harvested (sent downstream) from this loader. Does
    * not include the current, in-flight batch.
    */
-
   private int harvestBatchCount;
 
   /**
    * Counts the rows included in previously-harvested batches. Does not
    * include the number of rows in the current batch.
    */
-
   private int previousRowCount;
 
   /**
@@ -258,7 +237,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * overflow row is in effect, then this number is undefined (and should be
    * zero.)
    */
-
   private int pendingRowCount;
 
   /**
@@ -266,13 +244,11 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * adjusted between batches, perhaps based on the actual observed size of
    * input data.
    */
-
   private int targetRowCount;
 
   /**
    * Total bytes allocated to the current batch.
    */
-
   protected int accumulatedBatchSize;
 
   protected final ProjectionSet projectionSet;
@@ -399,7 +375,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
   /**
    * Start a batch to report only schema without data.
    */
-
   public void startEmptyBatch() {
     startBatch(true);
   }
@@ -484,7 +459,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * Called before writing a new row. Implementation of
    * {@link RowSetLoader#start()}.
    */
-
   protected void startRow() {
     switch (state) {
     case ACTIVE:
@@ -504,13 +478,12 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * Finalize the current row. Implementation of
    * {@link RowSetLoader#save()}.
    */
-
   protected void saveRow() {
     switch (state) {
     case ACTIVE:
       rootWriter.endArrayValue();
       rootWriter.saveRow();
-      if (! writerIndex.next()) {
+      if (!writerIndex.next()) {
         state = State.FULL_BATCH;
       }
 
@@ -550,11 +523,10 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * @return true if the batch is full (reached vector capacity or the
    * row count limit), false if more rows can be added
    */
-
   protected boolean isFull() {
     switch (state) {
     case ACTIVE:
-      return ! writerIndex.valid();
+      return !writerIndex.valid();
     case OVERFLOW:
     case FULL_BATCH:
       return true;
@@ -579,7 +551,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * @return the number of rows to be sent downstream for this
    * batch. Does not include the overflow row.
    */
-
   protected int rowCount() {
     switch (state) {
     case ACTIVE:
@@ -621,7 +592,7 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
 
   @Override
   public boolean isProjectionEmpty() {
-    return ! rootState.hasProjections();
+    return !rootState.hasProjections();
   }
 
   @Override
@@ -809,7 +780,6 @@ public class ResultSetLoaderImpl implements ResultSetLoader, LoaderInternals {
    * This will occur if the target row count is incorrect for the
    * data size.
    */
-
   private void checkInitialAllocation() {
     if (options.maxBatchSize < 0) {
       logger.debug("Initial vector allocation: {}, no batch limit specified",
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
index 2e5881b..6138283 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
@@ -90,7 +90,6 @@ import org.apache.drill.exec.vector.complex.RepeatedDictVector;
  * either one list of columns or another, the internal and external maps must
  * differ. The set of child vectors (except for child maps) are shared.
  */
-
 public abstract class TupleState extends ContainerState
   implements AbstractTupleWriter.TupleWriterListener {
 
@@ -125,7 +124,6 @@ public abstract class TupleState extends ContainerState
    * a structured: an ordered, named list of columns.) When looking for newly
    * added columns, they will always be at the end.
    */
-
   public static class MapColumnState extends BaseContainerColumnState {
     protected final MapState mapState;
     protected boolean isVersioned;
@@ -169,7 +167,6 @@ public abstract class TupleState extends ContainerState
      * </ul>
      * @return <tt>true</tt> if this map is versioned as described above
      */
-
     public boolean isVersioned() { return isVersioned; }
 
     @Override
@@ -181,7 +178,6 @@ public abstract class TupleState extends ContainerState
    * vector. The map vector itself is a pseudo-vector that is simply a
    * container for other vectors, and so needs no management itself.
    */
-
   public static class MapVectorState implements VectorState {
 
     private final AbstractMapVector mapVector;
@@ -243,14 +239,12 @@ public abstract class TupleState extends ContainerState
    * Note that by "row" we mean the set of vectors that define the
    * set of rows.
    */
-
   public static class RowState extends TupleState {
 
     /**
      * The row-level writer for stepping through rows as they are written,
      * and for accessing top-level columns.
      */
-
     private final RowSetLoaderImpl writer;
 
     /**
@@ -259,7 +253,6 @@ public abstract class TupleState extends ContainerState
      * consumer of the writers. Also excludes columns if added during
      * an overflow row.
      */
-
     private final VectorContainer outputContainer;
 
     public RowState(ResultSetLoaderImpl rsLoader, ResultVectorCache vectorCache) {
@@ -283,7 +276,6 @@ public abstract class TupleState extends ContainerState
      *
      * @return <tt>true</tt>
      */
-
     @Override
     protected boolean isVersioned() { return true; }
 
@@ -314,7 +306,6 @@ public abstract class TupleState extends ContainerState
    * The map state is associated with a map vector. This vector is built
    * either during harvest time (normal maps) or on the fly (union maps.)
    */
-
   public static abstract class MapState extends TupleState {
 
     public MapState(LoaderInternals events,
@@ -366,7 +357,6 @@ public abstract class TupleState extends ContainerState
      * that maps are materialized regardless of nesting depth within
      * a union.
      */
-
     @Override
     protected boolean isVersioned() {
       return ((MapColumnState) parentColumn).isVersioned();
@@ -400,7 +390,6 @@ public abstract class TupleState extends ContainerState
      * map, then it is the writer itself. If this is a map array,
      * then the tuple is nested inside the array.
      */
-
     @Override
     public AbstractTupleWriter writer() {
       return (AbstractTupleWriter) parentColumn.writer().tuple();
@@ -420,7 +409,6 @@ public abstract class TupleState extends ContainerState
      * map, then it is the writer itself. If this is a map array,
      * then the tuple is nested inside the array.
      */
-
     @Override
     public AbstractTupleWriter writer() {
       return (AbstractTupleWriter) parentColumn.writer().array().tuple();
@@ -433,13 +421,11 @@ public abstract class TupleState extends ContainerState
    * query does not project; the result set loader creates a dummy column
    * and dummy writer, then does not project the column to the output.)
    */
-
   protected final List<ColumnState> columns = new ArrayList<>();
 
   /**
    * Internal writer schema that matches the column list.
    */
-
   protected final TupleMetadata schema = new TupleSchema();
 
   /**
@@ -455,7 +441,6 @@ public abstract class TupleState extends ContainerState
    * not defer columns because of the muddy semantics (and infrequent use)
    * of unions.
    */
-
   protected TupleMetadata outputSchema;
 
   private int prevHarvestIndex = -1;
@@ -479,7 +464,6 @@ public abstract class TupleState extends ContainerState
    * @return ordered list of column states for the columns within
    * this tuple
    */
-
   public List<ColumnState> columns() { return columns; }
 
   public TupleMetadata schema() { return writer().tupleSchema(); }
@@ -518,12 +502,10 @@ public abstract class TupleState extends ContainerState
   protected void updateOutput(int curSchemaVersion) {
 
     // Scan all columns
-
     for (int i = 0; i < columns.size(); i++) {
       final ColumnState colState = columns.get(i);
 
       // Ignore unprojected columns
-
       if (! colState.writer().isProjected()) {
         continue;
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/BaseRequestedColumn.java
similarity index 51%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/BaseRequestedColumn.java
index da38550..1363631 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/project/projSet/ProjectedMapColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/BaseRequestedColumn.java
@@ -15,25 +15,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.scan.project.projSet;
+package org.apache.drill.exec.physical.resultSet.project;
 
-import org.apache.drill.exec.physical.resultSet.ProjectionSet;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
-import org.apache.drill.exec.record.metadata.ColumnMetadata;
+public abstract class BaseRequestedColumn implements RequestedColumn {
 
-public class ProjectedMapColumn extends ProjectedReadColumn {
+  private final RequestedTuple parent;
+  private final String name;
 
-  private final ProjectionSet mapProjection;
+  public BaseRequestedColumn(RequestedTuple parent, String name) {
+    this.parent = parent;
+    this.name = name;
+  }
+
+  public boolean isRoot() { return parent == null; }
+
+  @Override
+  public String name() { return name; }
+
+  @Override
+  public String fullName() {
+    final StringBuilder buf = new StringBuilder();
+    buildName(buf);
+    return buf.toString();
+  }
 
-  public ProjectedMapColumn(ColumnMetadata readSchema,
-      RequestedColumn requestedCol, ColumnMetadata outputSchema,
-      ProjectionSet mapProjection) {
-    super(readSchema, requestedCol, outputSchema, null);
-    this.mapProjection = mapProjection;
+  protected void buildName(StringBuilder buf) {
+    parent.buildName(buf);
+    buf.append('`')
+       .append(name)
+       .append('`');
   }
 
   @Override
-  public ProjectionSet mapProjection() {
-    return mapProjection;
+  public boolean nameEquals(String target) {
+    return name.equalsIgnoreCase(target);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
index d44f8bf..3b76d7e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ImpliedTupleRequest.java
@@ -20,14 +20,11 @@ package org.apache.drill.exec.physical.resultSet.project;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.drill.common.expression.PathSegment;
-
 /**
  * Represents a wildcard: SELECT * when used at the root tuple.
  * When used with maps, means selection of all map columns, either
  * implicitly, or because the map itself is selected.
  */
-
 public class ImpliedTupleRequest implements RequestedTuple {
 
   public static final RequestedTuple ALL_MEMBERS =
@@ -43,21 +40,11 @@ public class ImpliedTupleRequest implements RequestedTuple {
   }
 
   @Override
-  public ProjectionType projectionType(String colName) {
-    return allProjected
-      ? ProjectionType.GENERAL
-      : ProjectionType.UNPROJECTED;
-  }
-
-  @Override
   public RequestedTuple mapProjection(String colName) {
     return allProjected ? ALL_MEMBERS : NO_MEMBERS;
   }
 
   @Override
-  public void parseSegment(PathSegment child) { }
-
-  @Override
   public RequestedColumn get(String colName) { return null; }
 
   @Override
@@ -70,4 +57,19 @@ public class ImpliedTupleRequest implements RequestedTuple {
   public TupleProjectionType type() {
     return allProjected ? TupleProjectionType.ALL : TupleProjectionType.NONE;
   }
+
+  @Override
+  public boolean isProjected(String colName) {
+    return allProjected;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder()
+        .append("{");
+    if (allProjected) {
+      buf.append("*");
+    }
+    return buf.append("}").toString();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionType.java
deleted file mode 100644
index 363f51d..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/ProjectionType.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.resultSet.project;
-
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-
-/**
- * Specifies the type of projection obtained by parsing the
- * projection list. The type is returned from a query of the
- * form "how is this column projected, if at all?"
- * <p>
- * The projection type allows the scan framework to catch
- * inconsistencies, such as projecting an array as a map,
- * and so on.
- */
-
-public enum ProjectionType {
-
-  /**
-   * The column is not projected in the query.
-   */
-
-  UNPROJECTED,
-
-  /**
-   * Projection is a wildcard.
-   */
-  WILDCARD,     // *
-
-  /**
-   * Projection is by simple name. "General" means that
-   * we have no hints about the type of the column from
-   * the projection.
-   */
-
-  GENERAL,      // x
-
-  /**
-   * The column is projected as a scalar. This state
-   * requires metadata beyond the projection list and
-   * is returned only when that metadata is available.
-   */
-
-  SCALAR,       // x (from schema)
-
-  /**
-   * Applies to the parent of an x.y pair in projection: the
-   * existence of a dotted-member tells us that the parent
-   * must be a tuple (e.g. a Map.)
-   */
-
-  TUPLE,        // x.y
-
-  /**
-   * The projection includes an array suffix, so the column
-   * must be an array.
-   */
-
-  ARRAY,        // x[0]
-
-  /**
-   * Combination of array and map hints.
-   */
-
-  TUPLE_ARRAY,  // x[0].y
-
-  DICT, // x[0] or x['key'] (depends on key type)
-
-  DICT_ARRAY; // x[0][42] or x[0]['key'] (depends on key type)
-
-  public boolean isTuple() {
-    return this == ProjectionType.TUPLE || this == ProjectionType.TUPLE_ARRAY;
-  }
-
-  public boolean isArray() {
-    return this == ProjectionType.ARRAY || this == ProjectionType.TUPLE_ARRAY || this == DICT_ARRAY;
-  }
-
-  public boolean isDict() {
-    return this == DICT || this == DICT_ARRAY;
-  }
-
-  /**
-   * We can't tell, just from the project list, if a column must
-   * be scalar. A column of the form "a" could be a scalar, but
-   * that form is also consistent with maps and arrays.
-   */
-  public boolean isMaybeScalar() {
-    return this == GENERAL || this == SCALAR;
-  }
-
-  public static ProjectionType typeFor(MajorType majorType) {
-    boolean repeated = Types.isRepeated(majorType);
-    if (majorType.getMinorType() == MinorType.MAP) {
-      return repeated ? TUPLE_ARRAY : TUPLE;
-    } else if (majorType.getMinorType() == MinorType.DICT) {
-      return repeated ? DICT_ARRAY : DICT;
-    } else if (repeated || majorType.getMinorType() == MinorType.LIST) {
-      return ARRAY;
-    }
-    return SCALAR;
-  }
-
-  /**
-   * Reports if this type (representing an item in a projection list)
-   * is compatible with the projection type representing an actual
-   * column produced by an operator. The check is not symmetric.
-   * <p>
-   * For example, a column of type map array is compatible with a
-   * projection of type map "m.a" (project all a members of the map array),
-   * but a projection type of map array "m[1].a" is not compatible with
-   * a (non-array) map column.
-   *
-   * @param readType projection type, from {@link #typeFor(MajorType)},
-   * for an actual column
-   * @return true if this projection type is compatible with the
-   * column's projection type
-   */
-
-  public boolean isCompatible(ProjectionType readType) {
-    switch (readType) {
-    case UNPROJECTED:
-    case GENERAL:
-    case WILDCARD:
-      return true;
-    default:
-      break;
-    }
-
-    switch (this) {
-    case ARRAY:
-      return readType == ARRAY || readType == TUPLE_ARRAY
-          || readType == DICT // the actual key type should be validated later
-          || readType == DICT_ARRAY;
-    case TUPLE_ARRAY:
-      return readType == TUPLE_ARRAY || readType == DICT_ARRAY;
-    case SCALAR:
-      return readType == SCALAR;
-    case TUPLE:
-      return readType == TUPLE || readType == TUPLE_ARRAY || readType == DICT || readType == DICT_ARRAY;
-    case DICT:
-      return readType == DICT || readType == DICT_ARRAY;
-    case UNPROJECTED:
-    case GENERAL:
-    case WILDCARD:
-      return true;
-    default:
-      throw new IllegalStateException(toString());
-    }
-  }
-
-  public String label() {
-    switch (this) {
-    case SCALAR:
-      return "scalar (a)";
-    case ARRAY:
-      return "array (a[n])";
-    case TUPLE:
-      return "tuple (a.x)";
-    case TUPLE_ARRAY:
-      return "tuple array (a[n].x)";
-    case DICT:
-      return "dict (a['key'])";
-    case WILDCARD:
-      return "wildcard (*)";
-    default:
-      return name();
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java
new file mode 100644
index 0000000..a14a68d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Projections.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.resultSet.project;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.ArraySegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.common.expression.SchemaPath;
+
+/**
+ * Converts a projection list passed to an operator into a scan projection list,
+ * coalescing multiple references to the same column into a single reference.
+ */
+public class Projections {
+
+  private Projections() { }
+
+  public static RequestedTuple projectAll() {
+    return ImpliedTupleRequest.ALL_MEMBERS;
+  }
+
+  public static RequestedTuple projectNone() {
+    return ImpliedTupleRequest.NO_MEMBERS;
+  }
+
+  /**
+   * Parse a projection list. The list should consist of a list of column names;
+   * or wildcards. An empty list means
+   * nothing is projected. A null list means everything is projected (that is, a
+   * null list here is equivalent to a wildcard in the SELECT statement.)
+   * <p>
+   * The projection list may include both a wildcard and column names (as in
+   * the case of implicit columns.) This results in a final list that both
+   * says that everything is projected, and provides the list of columns.
+   * <p>
+   * Parsing is used at two different times. First, to parse the list from
+   * the physical operator. This has the case above: an explicit wildcard
+   * and/or additional columns. Then, this class is used again to prepare the
+   * physical projection used when reading. In this case, wildcards should
+   * be removed, implicit columns pulled out, and just the list of read-level
+   * columns should remain.
+   *
+   * @param projList
+   *          the list of projected columns, or null if no projection is to be
+   *          done
+   * @return a projection set that implements the specified projection
+   */
+  public static RequestedTuple parse(Collection<SchemaPath> projList) {
+    if (projList == null) {
+      return projectAll();
+    }
+    if (projList.isEmpty()) {
+      return projectNone();
+    }
+    RequestedTupleImpl tupleProj = new RequestedTupleImpl();
+    for (SchemaPath col : projList) {
+      parseMember(tupleProj, col.getRootSegment());
+    }
+    return tupleProj;
+  }
+
+  private static void parseMember(RequestedTupleImpl tuple, NameSegment nameSeg) {
+    RequestedColumn col = tuple.project(nameSeg.getPath());
+    if (!col.isWildcard()) {
+      RequestedColumnImpl colImpl = (RequestedColumnImpl) col;
+      parseChildSeg(colImpl, colImpl, nameSeg);
+    }
+  }
+
+  private static void parseChildSeg(RequestedColumnImpl column, QualifierContainer parent, PathSegment parentPath) {
+    if (parentPath.isLastPath()) {
+      parseLeaf(parent);
+    } else {
+      PathSegment seg = parentPath.getChild();
+      if (seg.isArray()) {
+        parseArraySeg(column, parent, (ArraySegment) seg);
+      } else {
+        parseMemberSeg(column, parent, (NameSegment) seg);
+      }
+    }
+  }
+
+  /**
+   * Parse a projection of the form {@code a}: that is, just a bare column.
+   */
+  private static void parseLeaf(QualifierContainer parent) {
+    Qualifier qual = parent.qualifier();
+    if (qual == null) {
+      // Nothing to do
+    } else if (qual.isArray()) {
+      qual.projectAllElements();
+    } else if (qual.isTuple()) {
+      qual.projectAllMembers();
+    }
+  }
+
+  private static void parseArraySeg(RequestedColumnImpl column, QualifierContainer parent, ArraySegment arraySeg) {
+    Qualifier prevQualifier = parent.qualifier();
+    Qualifier qualifier = parent.requireQualifier();
+    if (column.refCount() > 1 && (prevQualifier == null || !prevQualifier.isArray())) {
+      qualifier.projectAllElements();
+    } else {
+      qualifier.addIndex(arraySeg.getIndex());
+    }
+    parseChildSeg(column, qualifier, arraySeg);
+  }
+
+  private static void parseMemberSeg(RequestedColumnImpl column, QualifierContainer parent, NameSegment memberSeg) {
+    Qualifier prevQualifier = parent.qualifier();
+    Qualifier qualifier = parent.requireQualifier();
+    if (column.refCount() > 1 && (prevQualifier == null || !prevQualifier.isTuple())) {
+      qualifier.projectAllMembers();
+    } else {
+      RequestedTupleImpl tuple = qualifier.explicitMembers();
+      if (tuple != null) {
+        parseMember(tuple, memberSeg);
+      }
+    }
+  }
+
+  /**
+   * Create a requested tuple projection from a rewritten top-level
+   * projection list. The columns within the list have already been parsed to
+   * pick out arrays, maps and scalars. The list must not include the
+   * wildcard: a wildcard list must be passed in as a null list. An
+   * empty list means project nothing. Null list means project all, else
+   * project only the columns in the list.
+   *
+   * @param projList top-level, parsed columns
+   * @return the tuple projection for the top-level row
+   */
+  public static RequestedTuple build(List<RequestedColumn> projList) {
+    if (projList == null) {
+      return new ImpliedTupleRequest(true);
+    }
+    if (projList.isEmpty()) {
+      return projectAll();
+    }
+    return new RequestedTupleImpl(projList);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Qualifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Qualifier.java
new file mode 100644
index 0000000..976f968
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/Qualifier.java
@@ -0,0 +1,162 @@
+/*
+ * 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.physical.resultSet.project;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
+
+/**
+ * Represents one level of qualifier for a column. Analogous to
+ * a {@code SchemaPath}, but represents the result of coalescing
+ * multiple occurrences of the same column.
+ */
+public class Qualifier implements QualifierContainer {
+  /**
+   * Marker to indicate that that a) the item is an
+   * array, and b) that all indexes are to be projected.
+   * Used when seeing both a and a[x].
+   */
+  private static final Set<Integer> ALL_INDEXES = new HashSet<>();
+
+  private Set<Integer> indexes;
+  private RequestedTuple members;
+  private Qualifier child;
+
+  @Override
+  public Qualifier qualifier() { return child; }
+
+  @Override
+  public Qualifier requireQualifier() {
+    if (child == null) {
+      child = new Qualifier();
+    }
+    return child;
+  }
+
+  public boolean isArray() {
+    return indexes != null;
+  }
+
+  public boolean hasIndexes() {
+    return isArray() && indexes != ALL_INDEXES;
+  }
+
+  public boolean hasIndex(int index) {
+    return hasIndexes() && indexes.contains(index);
+  }
+
+  public int maxIndex() {
+    if (! hasIndexes()) {
+      return 0;
+    }
+    int max = 0;
+    for (final Integer index : indexes) {
+      max = Math.max(max, index);
+    }
+    return max;
+  }
+
+  public boolean[] indexArray() {
+    if (! hasIndexes()) {
+      return null;
+    }
+    final int max = maxIndex();
+    final boolean map[] = new boolean[max+1];
+    for (final Integer index : indexes) {
+      map[index] = true;
+    }
+    return map;
+  }
+
+  public boolean isTuple() {
+    return members != null || (child != null && child.isTuple());
+  }
+
+  public RequestedTuple tuple() {
+    if (members != null) {
+      return members;
+    } if (child != null) {
+      return child.tuple();
+    } else {
+      return null;
+    }
+  }
+
+  protected void addIndex(int index) {
+    if (indexes == null) {
+      indexes = new HashSet<>();
+    }
+    if (indexes != ALL_INDEXES) {
+      indexes.add(index);
+    }
+  }
+
+  protected void projectAllElements() {
+    indexes = ALL_INDEXES;
+  }
+
+  public int arrayDims() {
+    if (!isArray()) {
+      return 0;
+    } else if (child == null) {
+      return 1;
+    } else {
+      return 1 + child.arrayDims();
+    }
+  }
+
+  public void projectAllMembers() {
+    if (members == null || members.type() != TupleProjectionType.ALL) {
+      members = ImpliedTupleRequest.ALL_MEMBERS;
+    }
+  }
+
+  public RequestedTupleImpl explicitMembers() {
+    if (members == null) {
+      members = new RequestedTupleImpl();
+    }
+    if (members.type() == TupleProjectionType.SOME) {
+      return (RequestedTupleImpl) members;
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    if (isArray()) {
+      buf.append("[");
+      if (indexes == ALL_INDEXES) {
+        buf.append("*");
+      } else {
+        List<String> idxs = indexes.stream().sorted().map(i -> Integer.toString(i)).collect(Collectors.toList());
+        buf.append(String.join(", ", idxs));
+      }
+      buf.append("]");
+    }
+    if (members != null) {
+      buf.append(members.toString());
+    }
+    return buf.toString();
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/QualifierContainer.java
similarity index 63%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/QualifierContainer.java
index aba6926..ce8d485 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/scan/columns/UnresolvedColumnsArrayColumn.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/QualifierContainer.java
@@ -15,16 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.scan.columns;
+package org.apache.drill.exec.physical.resultSet.project;
 
-import org.apache.drill.exec.physical.impl.scan.project.AbstractUnresolvedColumn;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
-
-public class UnresolvedColumnsArrayColumn extends AbstractUnresolvedColumn {
-
-  public UnresolvedColumnsArrayColumn(RequestedColumn inCol) {
-    super(inCol);
-  }
-
-  public boolean[] selectedIndexes() { return inCol.indexes(); }
+public interface QualifierContainer {
+  Qualifier qualifier();
+  Qualifier requireQualifier();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumn.java
new file mode 100644
index 0000000..c074551
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumn.java
@@ -0,0 +1,239 @@
+/*
+ * 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.physical.resultSet.project;
+
+/**
+ * Plan-time properties of a requested column. Represents
+ * a consolidated view of the set of references to a column.
+ * For example, the project list might contain:
+ * <ul>
+ * <li>{@code SELECT *}</li>
+ * <li>{@code SELECT filename, *, dir0}</li>
+ * <li>{@code SELECT a, b, c}</li>
+ * <li>{@code SELECT columns[4], columns[8]}</li>
+ * <li>{@code SELECT a.b, a.c}</li>
+ * <li>{@code SELECT columns, columns[1]}</li>
+ * <li>{@code SELECT a, a.b}</li>
+ * </ul>
+ *
+ * In each case, the same column is referenced in different
+ * forms which are consolidated into this abstraction.
+ * <p>
+ * The resulting information is a "pattern": a form of reference.
+ * Given the requested column, code can check if some concrete
+ * reader-provided column is consistent with the requested
+ * projection or not. The project
+ * list does not contain sufficient information to definitively pick
+ * a type; it only excludes certain types.
+ * <p>
+ * Even for complex types, we cannot definitively know the type.
+ * For example, the projection {@code a[0]} could either refer to an
+ * array (of any type), <b>or</b> a {@code DICT} with integer keys.
+ * Similarly, a projection of the form {@code a.b} can either refer
+ * to a member of a map, or the {@code "b"} string key of a
+ * {@code DICT} column.
+ *
+ * <h4>Compatibility Rules</h4>
+ *
+ * The pattern given by projection is consistent with certain concrete types
+ * as follows. + means any number of additional qualifiers. Note that the
+ * following list is conceptual based on observed practice; the actual
+ * implementation may be more restrictive.
+ * <p>
+ * <table>
+ * <tr><th>Type</th><th>Consistent with</th></tr>
+ * <tr><td>Non-repeated MAP</td>
+ *     <td>{@code a}, {@code a.b}</td></tr>
+ * <tr><td>Repeated MAP</td>
+ *     <td>{@code a}, {@code a.b}, {@code a[n].b}</td></tr>
+ * <tr><td>Non-repeated Scalar</td>
+ *     <td>{@code a}</td></tr>
+ * <tr><td>Repeated Scalar</td>
+ *     <td>{@code a}, {@code a[n]}</td></tr>
+ * <tr><td>Non-repeated DICT</td>
+ *     <td>{@code a}, {@code a[n]}, {@code a['key']}</td></tr>
+ * <tr><td>Repeated DICT</td>
+ *     <td>{@code a}, {@code a[n]}, {@code a['key']}, {@code a[n][m]}, {@code a[n]['key']}</td></tr>
+ * <tr><td>Non-repeated LIST</td>
+ *     <td>{@code a}, {@code a[n]}</td></tr>
+ * <tr><td>Repeated LIST</td>
+ *     <td>{@code a}, {@code a[n]}, {@code a[n][n]}</td></tr>
+ * </table>
+ * <p>
+ * MAP, DICT, UNION and LIST are structured types: projection can reach
+ * into the structure to any number of levels. In such a case, when sufficient
+ * schema information is available, the above rules can be applied recursively
+ * to each level of structure. The recursion can be done in the class for a
+ * DICT (since there is only one child type), but must be external for other
+ * complex types. For MAP, the column can report which specific members
+ * are projected.
+ * <p>
+ * The Text reader allows the {@code columns} column, which allows the
+ * user to specify indexes. This class reports which indexes were actually
+ * selected. Index information is available only at the top level, but
+ * not for 2+ dimensions.
+ */
+public interface RequestedColumn {
+
+  /**
+   * The column name as projected. If the same column appears multiple
+   * times (as in {@code a[1], A[2]}, then the case of the first appearance
+   * is used.
+   *
+   * @return the column name as observed in the project list
+   */
+  String name();
+
+  /**
+   * Returns the fully-qualified column name. If the column is in the
+   * top-level tuple, this is the same as {@code name()}. If the column
+   * is nested in an array, then this name includes the enclosing
+   * columns: {@code a.b.c}.
+   *
+   * @return the full name with enclosing map prefixes, if any
+   */
+  String fullName();
+
+  /**
+   * Case-insensitive comparison of the column name.
+   */
+  boolean nameEquals(String target);
+
+  /**
+   * Several consumers of this this mechanism process the "raw" projection list
+   * which can contain a combination of wildcard and otehr columns. For example:
+   * {@code filename, *, dir0}. The requested tuple preserves the wildcard
+   * within the projection list so that, say, the projection mechanism can insert
+   * the actual data columns between the two implicit columns in the example.
+   * <p>
+   * If a column is a wildcard, then none of the other methods apply, since
+   * this projected column represents any number or actual columns.
+   *
+   * @return if this column is the wildcard placeholder
+   */
+  boolean isWildcard();
+
+  /**
+   * @return true if this column has no qualifiers. Example:
+   * {@code a}.
+   */
+  boolean isSimple();
+
+  /**
+   * Report whether the projection implies a tuple. Example:
+   * {@code a.b}. Not that this method, and others can only tell
+   * if the projection implies a tuple; the actual column may
+   * be a tuple (MAP), but be projected simply. The map
+   * format also describes a DICT with a VARCHAR key.
+   *
+   * @return true if the column has a map-like projection.
+   */
+  boolean isTuple();
+
+  /**
+   * Return projection information for the column as a tuple. If
+   * projection included references to nested columns (such as
+   * {@code a.b, a.c}, then the tuple projection will list only
+   * the referenced columns. However, if projection is generic
+   * ({@code m}), then we presume all columns of the map are projected
+   * and the returned object assumes all members are projected.
+   *
+   * @return projection information for a (presumed) map column
+   */
+  RequestedTuple tuple();
+
+  /**
+   * Report whether the first qualifier is an array.
+   * Example: {@code a[1]}. The array format also describes
+   * a DICT with an integer key.
+   * @return true if the column must be an array.
+   */
+  boolean isArray();
+
+  /**
+   * If {@code isArray()} returns true, reports the number of dimensions
+   * observed in projection. That is if projection is {@code a[0][1]},
+   * then this method returns 2.
+   * <p>
+   * Note that, as with all projection-level information, this number
+   * reflects only what was in the project list; not what might be
+   * the number of dimensions in the actual input source.
+   *
+   * @return the maximum number of array dimensions observed in the
+   * projection list, or 0 if this column was not observed to be an
+   * array (if {@code isArray()} returns {@code false}.
+   */
+  int arrayDims();
+
+  /**
+   * Reports if the projection list included (only) specific element
+   * indexes. For example: {@code a[2], a[5]}. The user could also project
+   * both indexes and the array: {@code a[0], a}. In this case
+   * {@code isArray()} is {code true}, but {@code hasIndexes()} is {@code false}.
+   *
+   * @return {@code true} if the column has enumerated indexes, {@code false}
+   * if the column was also projected as a whole, or if this column
+   * was not observed to be an array
+   */
+  boolean hasIndexes();
+
+  /**
+   * Return the maximum index value, if only explicit indexes were given.
+   * Valid if {@code hasIndexes()} returns true.
+   *
+   * @return the maximum array index value known to the projection, or
+   * 0 if {@code isArray()} is {@code false}. Also returns 0 if
+   * {@code hasIndexe()} returns {@code false}, meaning that either
+   * the column was not observed to be an array, or was projected
+   * with both indexes and by itself: {@code a[0], a}.
+   */
+  int maxIndex();
+
+  /**
+    * Return a bitmap of the selected indexes. Only valid if
+    * {@code hasIndexes()} returns {@code true}.
+    * @return a bitmap of the selected array indexes, or {@code null}
+    * if {@code hasIndexes()} returns {@code false}.
+    */
+  boolean[] indexes();
+
+  /**
+   * Report is a specific index was selected. Short cut for the other
+   * array methods. Used in cases such as the {@code columns} column where
+   * the user can select specific elements (column) but not others.
+   *
+   * @param index the array index to check
+   * @return {@code true} if the array element was projected, either
+   * explicitly ({@code a[3]}) or implicitly ({@code a}). Returns
+   * {@code false} <i>only</i> if {@code hasIndexes()} returns
+   * {@code true} (the user listed only explicit indexes) and the
+   * requested index is not among those requested ({@code index >=
+   * maxIndex() || !indexes()[index]})
+   */
+  boolean hasIndex(int index);
+
+  /**
+   * The internal qualifier information for the column. Generally not
+   * needed by clients; use the other informations to interpret the
+   * qualifier for you.
+   *
+   * @return detailed column qualifier information, if the column was
+   * seen to be complex in the project list
+   */
+  Qualifier qualifier();
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumnImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumnImpl.java
index 55a252d..a764843 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumnImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedColumnImpl.java
@@ -17,226 +17,99 @@
  */
 package org.apache.drill.exec.physical.resultSet.project;
 
-import java.util.HashSet;
-import java.util.Set;
-
 import org.apache.drill.common.expression.PathSegment.NameSegment;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 
 /**
  * Represents one name element. Like a {@link NameSegment}, except that this
  * version is an aggregate. If the projection list contains `a.b` and `a.c`,
  * then one name segment exists for a, and contains segments for both b and c.
  */
+public class RequestedColumnImpl extends BaseRequestedColumn implements QualifierContainer {
 
-public class RequestedColumnImpl implements RequestedColumn {
-
-  /**
-   * Special marker to indicate that that a) the item is an
-   * array, and b) that all indexes are to be projected.
-   * Used when seeing both a and a[x].
-   */
-
-  private static final Set<Integer> ALL_INDEXES = new HashSet<>();
-
-  private final RequestedTuple parent;
-  private final String name;
-  private RequestedTuple members;
-  private Set<Integer> indexes;
-  private ProjectionType type;
+  private int refCount = 1;
+  private Qualifier qualifier;
 
   public RequestedColumnImpl(RequestedTuple parent, String name) {
-    this.parent = parent;
-    this.name = name;
-    setType();
-  }
-
-  public RequestedColumnImpl(RequestedTuple parent, String name, ProjectionType type) {
-    this.parent = parent;
-    this.name = name;
-    this.type = type;
+    super(parent, name);
   }
 
-  @Override
-  public String name() { return name; }
-  @Override
-  public ProjectionType type() { return type; }
-  @Override
-  public boolean isWildcard() { return type == ProjectionType.WILDCARD; }
-  @Override
-  public boolean isSimple() { return type == ProjectionType.GENERAL; }
+  protected void bumpRefCount() { refCount++; }
 
-  @Override
-  public boolean isArray() { return type.isArray(); }
+  public int refCount() { return refCount; }
 
   @Override
-  public boolean isTuple() { return type.isTuple(); }
+  public Qualifier qualifier() { return qualifier; }
 
   @Override
-  public boolean isDict() {
-    return type.isDict();
-  }
-
-  public RequestedTuple asTuple() {
-    if (members == null) {
-      members = new RequestedTupleImpl(this);
-      setType();
-    }
-    return members;
-  }
-
-  public RequestedTuple projectAllMembers(boolean projectAll) {
-    members = projectAll ? ImpliedTupleRequest.ALL_MEMBERS : ImpliedTupleRequest.NO_MEMBERS;
-    setType();
-    return members;
-  }
-
-  public void addIndex(int index) {
-    if (indexes == null) {
-      indexes = new HashSet<>();
-    }
-    if (indexes != ALL_INDEXES) {
-      indexes.add(index);
+  public Qualifier requireQualifier() {
+    if (qualifier == null) {
+      qualifier = new Qualifier();
     }
-    setType();
-  }
-
-  public void projectAllElements() {
-    indexes = ALL_INDEXES;
-    setType();
+    return qualifier;
   }
 
   @Override
-  public boolean hasIndexes() {
-    return indexes != null && indexes != ALL_INDEXES;
-  }
+  public boolean isWildcard() { return false; }
 
   @Override
-  public boolean hasIndex(int index) {
-    return hasIndexes() ? indexes.contains(index) : false;
-  }
+  public boolean isSimple() { return qualifier == null; }
 
   @Override
-  public int maxIndex() {
-    if (! hasIndexes()) {
-      return 0;
-    }
-    int max = 0;
-    for (final Integer index : indexes) {
-      max = Math.max(max, index);
-    }
-    return max;
+  public boolean isTuple() {
+    return qualifier != null && qualifier.isTuple();
   }
 
   @Override
-  public boolean[] indexes() {
-    if (! hasIndexes()) {
-      return null;
-    }
-    final int max = maxIndex();
-    final boolean map[] = new boolean[max+1];
-    for (final Integer index : indexes) {
-      map[index] = true;
+  public RequestedTuple tuple() {
+    if (!isTuple()) {
+      return ImpliedTupleRequest.ALL_MEMBERS;
     }
-    return map;
+    return qualifier.tuple();
   }
 
   @Override
-  public String fullName() {
-    final StringBuilder buf = new StringBuilder();
-    buildName(buf);
-    return buf.toString();
+  public boolean isArray() {
+    return qualifier != null && qualifier.isArray();
   }
 
-  public boolean isRoot() { return parent == null; }
-
-  private void setType() {
-    if (name.equals(SchemaPath.DYNAMIC_STAR)) {
-      type = ProjectionType.WILDCARD;
-    } else if (indexes != null && members != null) {
-      type = ProjectionType.TUPLE_ARRAY;
-    }
-    else if (indexes != null) {
-      type = ProjectionType.ARRAY;
-    } else if (members != null) {
-      type = ProjectionType.TUPLE;
-    } else {
-      type = ProjectionType.GENERAL;
-    }
-  }
-
-  protected void buildName(StringBuilder buf) {
-    parent.buildName(buf);
-    buf.append('`')
-       .append(name)
-       .append('`');
+  @Override
+  public boolean hasIndexes() {
+    return qualifier != null && qualifier.hasIndexes();
   }
 
   @Override
-  public String summary() {
-    switch (type) {
-    case ARRAY:
-      return "array column";
-    case TUPLE:
-      return "map column";
-    case TUPLE_ARRAY:
-      return "repeated map";
-    case DICT:
-      return "dict column";
-    case DICT_ARRAY:
-      return "repeated dict column";
-    case WILDCARD:
-      return "wildcard";
-    default:
-      return "column";
-    }
+  public boolean hasIndex(int index) {
+    return qualifier != null && qualifier.hasIndex(index);
   }
 
   @Override
-  public boolean nameEquals(String target) {
-    return name.equalsIgnoreCase(target);
+  public int maxIndex() {
+    return qualifier == null ? 0 : qualifier.maxIndex();
   }
 
   @Override
-  public RequestedTuple mapProjection() {
-    switch (type) {
-    case ARRAY:
-    case GENERAL:
-      // Don't know if the target is a tuple or not.
-
-      return ImpliedTupleRequest.ALL_MEMBERS;
-    case TUPLE:
-    case TUPLE_ARRAY:
-      return members == null ? ImpliedTupleRequest.ALL_MEMBERS : members;
-    case UNPROJECTED:
-      return ImpliedTupleRequest.NO_MEMBERS;
-    default:
-      return null;
-    }
+  public boolean[] indexes() {
+    return qualifier == null ? null : qualifier.indexArray();
   }
 
+  /**
+   * Convert the projection to a string of the form:
+   * {@code a[0,1,4]['*']{b, c d}}.
+   * The information here s insufficient to specify a type,
+   * it only specifies a pattern to which types are compatible.
+   */
   @Override
   public String toString() {
-    final StringBuilder buf = new StringBuilder();
-    buf
-      .append("[")
-      .append(getClass().getSimpleName())
-      .append(" name=")
-      .append(name())
-      .append(", type=")
-      .append(summary());
-    if (isArray()) {
-      buf
-        .append(", array=")
-        .append(indexes);
-    }
-    if (isTuple()) {
-      buf
-        .append(", tuple=")
-        .append(members);
+    final StringBuilder buf = new StringBuilder()
+        .append(name());
+    if (qualifier != null) {
+      buf.append(qualifier.toString());
     }
-    buf.append("]");
     return buf.toString();
   }
+
+  @Override
+  public int arrayDims() {
+    return qualifier == null ? 0 : qualifier.arrayDims();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
index d9b3e1a..1279299 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTuple.java
@@ -19,13 +19,10 @@ package org.apache.drill.exec.physical.resultSet.project;
 
 import java.util.List;
 
-import org.apache.drill.common.expression.PathSegment;
-
 /**
  * Represents the set of columns projected for a tuple (row or map.)
- * The projected columns might themselves be columns, so returns a
- * projection set for such columns. Represents the set of requested
- * columns and tuples as expressed in the physical plan.
+ * Each column may have structure: a set of referenced names or
+ * array indices.
  * <p>
  * Three variations exist:
  * <ul>
@@ -45,55 +42,22 @@ import org.apache.drill.common.expression.PathSegment;
  * projection set which the code can query to determine if a newly
  * added column is wanted (and so should have a backing vector) or
  * is unwanted (and can just receive a dummy writer.)
+ * <p>
+ * Wildcards will set the projection type to {@code ALL}, and will
+ * be retained in the projection list. Retaining the wildcard
+ * is important because multiple consumers insert columns at the
+ * wildcard position. For example:<br>
+ * {@code SELECT filename, *, filepath FROM ...}
  */
-
 public interface RequestedTuple {
 
-  /**
-   * Plan-time properties of a requested column. Represents
-   * a consolidated view of the set of references to a column.
-   * For example, the project list might contain:<br>
-   * <tt>SELECT columns[4], columns[8]</tt><br>
-   * <tt>SELECT a.b, a.c</tt><br>
-   * <tt>SELECT columns, columns[1]</tt><br>
-   * <tt>SELECT a, a.b</tt><br>
-   * In each case, the same column is referenced in different
-   * forms which are consolidated in to this abstraction.
-   * <p>
-   * Depending on the syntax, we can infer if a column must
-   * be an array or map. This is definitive: though we know that
-   * columns of the form above must be an array or a map,
-   * we cannot know if a simple column reference might refer
-   * to an array or map.
-   */
-
-  public interface RequestedColumn {
-
-    String name();
-    ProjectionType type();
-    boolean isWildcard();
-    boolean isSimple();
-    boolean isArray();
-    boolean isTuple();
-    boolean isDict();
-    String fullName();
-    RequestedTuple mapProjection();
-    boolean nameEquals(String target);
-    int maxIndex();
-    boolean[] indexes();
-    boolean hasIndexes();
-    boolean hasIndex(int index);
-    String summary();
-  }
-
   public enum TupleProjectionType {
     ALL, NONE, SOME
   }
 
   TupleProjectionType type();
-  void parseSegment(PathSegment child);
   RequestedColumn get(String colName);
-  ProjectionType projectionType(String colName);
+  boolean isProjected(String colName);
   RequestedTuple mapProjection(String colName);
   List<RequestedColumn> projections();
   void buildName(StringBuilder buf);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
index 3e91466..dc0c8e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedTupleImpl.java
@@ -17,19 +17,14 @@
  */
 package org.apache.drill.exec.physical.resultSet.project;
 
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.PathSegment;
-import org.apache.drill.common.expression.PathSegment.ArraySegment;
-import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.metadata.TupleNameSpace;
 
 /**
- * Represents an explicit projection at some tuple level.
+ * Represents an explicit projection at some tuple level. A tuple is the
+ * top-level row or a map.
  * <p>
  * A column is projected if it is explicitly listed in the selection list.
  * <p>
@@ -66,22 +61,41 @@ import org.apache.drill.exec.record.metadata.TupleNameSpace;
  * <li><tt>ArraySegment</tt> is the other kind of name part and represents
  * an array index such as the "[1]" in `columns`[1].</li>
  * <ul>
- * The parser here consumes only names, this mechanism does not consider
- * array indexes. As a result, there may be multiple projected columns that
- * map to the same projection here: `columns`[1] and `columns`[2] both map to
- * the name `columns`, for example.
+ * The parser considers names and array indexes. Example:<pre><code>
+ * a
+ * a.b
+ * a[2]
+ * a[2].b
+ * a[1][2][3]
+ * a[1][2][3].b.c
+ * a['foo'][0].b['bar']
+ * </code></pre>
+ *
+ * <h4>Usage</h4>
+ * The projection information is a <i>pattern</i> which supports queries of the
+ * form "is this column projected", and "if projected, is the projection consistent
+ * with such-and-so concrete type?" Clients should not try to work out the
+ * meaning of the pattern: doing so is very complex. Instead, do the following:
+ *
+ * <pre><code>
+ * String colName = ...;
+ * ColumnMetadata colDef = ...;
+ * InputTupleProjection tupleProj = ...
+ * if (tupleProj.isProjected(colName)) {
+ *   if (!tupleProj.isComsistentWith(colDef)) {
+ *     // Raise an error
+ *   }
+ *   // Handle a projected column.
+ * }</code></pre>
  */
-
 public class RequestedTupleImpl implements RequestedTuple {
 
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RequestedTupleImpl.class);
-  private static final Collection<SchemaPath> PROJECT_ALL = Collections.singletonList(SchemaPath.STAR_COLUMN);
-
   private final RequestedColumnImpl parent;
+  protected TupleProjectionType projectionType = TupleProjectionType.SOME;
   private final TupleNameSpace<RequestedColumn> projection = new TupleNameSpace<>();
 
   public RequestedTupleImpl() {
-    parent = null;
+    this.parent = null;
   }
 
   public RequestedTupleImpl(RequestedColumnImpl parent) {
@@ -100,194 +114,26 @@ public class RequestedTupleImpl implements RequestedTuple {
     return projection.get(colName.toLowerCase());
   }
 
-  private RequestedColumnImpl getImpl(String colName) {
+  protected RequestedColumnImpl getImpl(String colName) {
     return (RequestedColumnImpl) get(colName);
   }
 
-  @Override
-  public ProjectionType projectionType(String colName) {
+  protected RequestedColumn project(String colName) {
     RequestedColumn col = get(colName);
-    return col == null ? ProjectionType.UNPROJECTED : col.type();
-  }
-
-  @Override
-  public RequestedTuple mapProjection(String colName) {
-    RequestedColumnImpl col = getImpl(colName);
-    RequestedTuple mapProj = (col == null) ? null : col.mapProjection();
-    if (mapProj != null) {
-      return mapProj;
-    }
-
-    // No explicit information for the map. Members inherit the
-    // same projection as the map itself.
-
     if (col != null) {
-      return col.projectAllMembers(true);
-    }
-    return ImpliedTupleRequest.NO_MEMBERS;
-  }
-
-  /**
-   * Create a requested tuple projection from a rewritten top-level
-   * projection list. The columns within the list have already been parsed to
-   * pick out arrays, maps and scalars. The list must not include the
-   * wildcard: a wildcard list must be passed in as a null list. An
-   * empty list means project nothing. Null list means project all, else
-   * project only the columns in the list.
-   *
-   * @param projList top-level, parsed columns
-   * @return the tuple projection for the top-leel row
-   */
-
-  public static RequestedTuple build(List<RequestedColumn> projList) {
-    if (projList == null) {
-      return new ImpliedTupleRequest(true);
-    }
-    if (projList.isEmpty()) {
-      return ImpliedTupleRequest.NO_MEMBERS;
-    }
-    return new RequestedTupleImpl(projList);
-  }
-
-  /**
-   * Parse a projection list. The list should consist of a list of column names;
-   * or wildcards. An empty list means
-   * nothing is projected. A null list means everything is projected (that is, a
-   * null list here is equivalent to a wildcard in the SELECT statement.)
-   * <p>
-   * The projection list may include both a wildcard and column names (as in
-   * the case of implicit columns.) This results in a final list that both
-   * says that everything is projected, and provides the list of columns.
-   * <p>
-   * Parsing is used at two different times. First, to parse the list from
-   * the physical operator. This has the case above: an explicit wildcard
-   * and/or additional columns. Then, this class is used again to prepare the
-   * physical projection used when reading. In this case, wildcards should
-   * be removed, implicit columns pulled out, and just the list of read-level
-   * columns should remain.
-   *
-   * @param projList
-   *          the list of projected columns, or null if no projection is to be
-   *          done
-   * @return a projection set that implements the specified projection
-   */
-
-  public static RequestedTuple parse(Collection<SchemaPath> projList) {
-    if (projList == null) {
-      projList = PROJECT_ALL;
-    }
-    else if (projList.isEmpty()) {
-      return ImpliedTupleRequest.NO_MEMBERS;
-    }
-    RequestedTupleImpl projSet = new RequestedTupleImpl();
-    for (SchemaPath col : projList) {
-      projSet.parseSegment(col.getRootSegment());
-    }
-    return projSet;
-  }
-
-  @Override
-  public void parseSegment(PathSegment pathSeg) {
-    if (pathSeg.isLastPath()) {
-      parseLeaf((NameSegment) pathSeg);
-    } else if (pathSeg.getChild().isArray()) {
-      parseArray((NameSegment) pathSeg);
-    } else {
-      parseInternal((NameSegment) pathSeg);
-    }
-  }
-
-  private void parseLeaf(NameSegment nameSeg) {
-    String name = nameSeg.getPath();
-    RequestedColumnImpl member = getImpl(name);
-    if (member == null) {
-      projection.add(name, new RequestedColumnImpl(this, name));
-      return;
-    }
-    if (member.isSimple() || member.isWildcard()) {
-      throw UserException
-        .validationError()
-        .message("Duplicate column in project list: %s",
-            member.fullName())
-        .build(logger);
-    }
-    if (member.isArray()) {
-
-      // Saw both a and a[x]. Occurs in project list.
-      // Project all elements.
-
-      member.projectAllElements();
-      return;
-    }
-
-    // Else the column is a known map.
-
-    assert member.isTuple();
-
-    // Allow both a.b (existing) and a (this column)
-    // Since we we know a is a map, and we've projected the
-    // whole map, modify the projection of the column to
-    // project the entire map.
-
-    member.projectAllMembers(true);
-  }
-
-  private void parseInternal(NameSegment nameSeg) {
-    String name = nameSeg.getPath();
-    RequestedColumnImpl member = getImpl(name);
-    RequestedTuple map;
-    if (member == null) {
-      // New member. Since this is internal, this new member
-      // must be a map.
-
-      member = new RequestedColumnImpl(this, name);
-      projection.add(name, member);
-      map = member.asTuple();
-    } else if (member.isTuple()) {
-
-      // Known map. Add to it.
-
-      map = member.asTuple();
+      if (col instanceof RequestedColumnImpl) {
+        ((RequestedColumnImpl) col).bumpRefCount();
+      }
     } else {
-
-      // Member was previously projected by itself. We now
-      // know it is a map. So, project entire map. (Earlier
-      // we saw `a`. Now we see `a`.`b`.)
-
-      map = member.projectAllMembers(true);
-    }
-    map.parseSegment(nameSeg.getChild());
-  }
-
-  private void parseArray(NameSegment nameSeg) {
-    String name = nameSeg.getPath();
-    ArraySegment arraySeg = ((ArraySegment) nameSeg.getChild());
-    int index = arraySeg.getIndex();
-    RequestedColumnImpl member = getImpl(name);
-    if (member == null) {
-      member = new RequestedColumnImpl(this, name);
-      projection.add(name, member);
-    } else if (member.isSimple()) {
-
-      // Saw both a and a[x]. Occurs in project list.
-      // Project all elements.
-
-      member.projectAllElements();
-      return;
-    }
-
-    // Allow duplicate indexes. Example: z[0], z[0]['orange']
-    if (!member.hasIndex(index)) {
-      member.addIndex(index);
-    }
-
-    // Drills SQL parser does not support map arrays: a[0].c
-    // But, the SchemaPath does support them, so no harm in
-    // parsing them here.
-
-    if (! arraySeg.isLastPath()) {
-      parseInternal(nameSeg);
+      if (colName.equals(SchemaPath.DYNAMIC_STAR)) {
+        projectionType = TupleProjectionType.ALL;
+        col = new RequestedWildcardColumn(this, colName);
+      } else {
+        col = new RequestedColumnImpl(this, colName);
+      }
+      projection.add(colName, col);
     }
+    return col;
   }
 
   @Override
@@ -309,14 +155,39 @@ public class RequestedTupleImpl implements RequestedTuple {
    */
   @Override
   public TupleProjectionType type() {
-    if (projection.isEmpty()) {
-      return TupleProjectionType.NONE;
+    return projectionType;
+  }
+
+  @Override
+  public boolean isProjected(String colName) {
+    return projectionType == TupleProjectionType.ALL ? true : get(colName) != null;
+  }
+
+  @Override
+  public RequestedTuple mapProjection(String colName) {
+    switch (projectionType) {
+      case ALL:
+        return ImpliedTupleRequest.ALL_MEMBERS;
+      case NONE:
+        return ImpliedTupleRequest.NO_MEMBERS;
+      default:
+        RequestedColumnImpl colProj = getImpl(colName);
+        return colProj == null ? ImpliedTupleRequest.NO_MEMBERS : colProj.tuple();
     }
-    for (RequestedColumn col : projection) {
-      if (col.isWildcard()) {
-        return TupleProjectionType.ALL;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buf = new StringBuilder()
+        .append("{");
+    boolean first = true;
+    for (RequestedColumn col : projections()) {
+      if (!first) {
+        buf.append(", ");
       }
+      first = false;
+      buf.append(col.toString());
     }
-    return TupleProjectionType.SOME;
+    return buf.append("}").toString();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedWildcardColumn.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedWildcardColumn.java
new file mode 100644
index 0000000..179ac56
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/project/RequestedWildcardColumn.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.resultSet.project;
+
+public class RequestedWildcardColumn extends BaseRequestedColumn {
+
+  public RequestedWildcardColumn(RequestedTuple parent, String name) {
+    super(parent, name);
+  }
+
+  @Override
+  public boolean isWildcard() { return true; }
+
+  @Override
+  public boolean isSimple() { return true; }
+
+  @Override
+  public boolean isTuple() { return false; }
+
+  @Override
+  public RequestedTuple tuple() { return null; }
+
+  @Override
+  public boolean isArray() { return false; }
+
+  @Override
+  public boolean hasIndexes() { return false; }
+
+  @Override
+  public int maxIndex() { return 0; }
+
+  @Override
+  public boolean[] indexes() { return null; }
+
+  @Override
+  public boolean hasIndex(int index) { return false; }
+
+  @Override
+  public String toString() { return name(); }
+
+  @Override
+  public int arrayDims() { return 0; }
+
+  @Override
+  public Qualifier qualifier() { return null; }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
index 61f1cca..74f20e3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArray.java
@@ -50,7 +50,6 @@ import org.junit.experimental.categories.Category;
  * Test the "columns" array mechanism integrated with the scan schema
  * orchestrator including simulating reading data.
  */
-
 @Category(RowSetTests.class)
 public class TestColumnsArray extends SubOperatorTest {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
index e6b897d..af81aaf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayFramework.java
@@ -55,7 +55,6 @@ import static org.junit.Assert.assertTrue;
 /**
  * Test the columns-array specific behavior in the columns scan framework.
  */
-
 @Category(RowSetTests.class)
 public class TestColumnsArrayFramework extends SubOperatorTest {
 
@@ -145,7 +144,6 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
    * Test including a column other than "columns". Occurs when
    * using implicit columns.
    */
-
   @Test
   public void testNonColumnsProjection() {
 
@@ -178,7 +176,6 @@ public class TestColumnsArrayFramework extends SubOperatorTest {
   /**
    * Test projecting just the `columns` column.
    */
-
   @Test
   public void testColumnsProjection() {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
index 6d955be..1adf539 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestColumnsArrayParser.java
@@ -48,7 +48,6 @@ public class TestColumnsArrayParser extends SubOperatorTest {
    * as an array. No need for early schema. This case is special: it actually
    * creates the one and only table column to match the desired output column.
    */
-
   @Test
   public void testColumnsArray() {
     ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -89,7 +88,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
         RowSetTestUtils.projectAll(),
         ScanTestUtils.parsers(new ColumnsArrayParser(true)));
 
-    assertFalse(scanProj.projectAll());
+    assertTrue(scanProj.projectAll());
     assertEquals(1, scanProj.requestedCols().size());
 
     assertEquals(1, scanProj.columns().size());
@@ -154,7 +153,6 @@ public class TestColumnsArrayParser extends SubOperatorTest {
    * <p>
    * TODO: This should only be true for text readers, make this an option.
    */
-
   @Test
   public void testErrorColumnsArrayAndColumn() {
     try {
@@ -170,7 +168,6 @@ public class TestColumnsArrayParser extends SubOperatorTest {
   /**
    * Exclude a column and `columns` (reversed order of previous test).
    */
-
   @Test
   public void testErrorColumnAndColumnsArray() {
     try {
@@ -184,19 +181,17 @@ public class TestColumnsArrayParser extends SubOperatorTest {
   }
 
   /**
-   * Can't request `columns` twice.
+   * Requesting `columns` twice: second is ignored.
    */
-
   @Test
-  public void testErrorTwoColumnsArray() {
-    try {
-      ScanLevelProjection.build(
-          RowSetTestUtils.projectList(ColumnsScanFramework.COLUMNS_COL, ColumnsScanFramework.COLUMNS_COL),
-          ScanTestUtils.parsers(new ColumnsArrayParser(false)));
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
+  public void testTwoColumnsArray() {
+    ScanLevelProjection scanProj = ScanLevelProjection.build(
+        RowSetTestUtils.projectList(ColumnsScanFramework.COLUMNS_COL, ColumnsScanFramework.COLUMNS_COL),
+        ScanTestUtils.parsers(new ColumnsArrayParser(false)));
+    assertFalse(scanProj.projectAll());
+    assertEquals(2, scanProj.requestedCols().size());
+    assertEquals(1, scanProj.columns().size());
+    assertEquals(ColumnsScanFramework.COLUMNS_COL, scanProj.columns().get(0).name());
   }
 
   @Test
@@ -235,7 +230,6 @@ public class TestColumnsArrayParser extends SubOperatorTest {
    * The `columns` column is special: can't be used with other column names.
    * Make sure that the rule <i>does not</i> apply to implicit columns.
    */
-
   @Test
   public void testMetadataColumnsWithColumnsArray() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
@@ -274,7 +268,6 @@ public class TestColumnsArrayParser extends SubOperatorTest {
    * includes both the wildcard and the `columns` array.
    * We can ignore one of them.
    */
-
   @Test
   public void testWildcardAndColumns() {
     ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -283,7 +276,7 @@ public class TestColumnsArrayParser extends SubOperatorTest {
             ColumnsScanFramework.COLUMNS_COL),
         ScanTestUtils.parsers(new ColumnsArrayParser(true)));
 
-    assertFalse(scanProj.projectAll());
+    assertTrue(scanProj.projectAll());
     assertEquals(2, scanProj.requestedCols().size());
 
     assertEquals(1, scanProj.columns().size());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
index e115ad9..dfb4f08 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestFileMetadataColumnParser.java
@@ -78,7 +78,6 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
    * Test including file metadata (AKA "implicit columns") in the project
    * list.
    */
-
   @Test
   public void testFileMetadataColumnSelection() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
@@ -121,7 +120,6 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   /**
    * Verify that partition columns, in any case, work.
    */
-
   @Test
   public void testPartitionColumnSelection() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
@@ -152,7 +150,6 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   /**
    * Test wildcard expansion.
    */
-
   @Test
   public void testRevisedWildcard() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
@@ -170,48 +167,14 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   }
 
   /**
-   * Legacy (prior version) wildcard expansion always expands partition
-   * columns.
-   */
-
-  @Test
-  public void testLegacyWildcard() {
-    Path filePath = new Path("hdfs:///w/x/y/z.csv");
-    FileMetadataOptions options = standardOptions(filePath);
-    options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(true);
-    // Max partition depth is 3, though this "scan" sees only 2
-    options.setPartitionDepth(3);
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        options);
-
-    ScanLevelProjection scanProj = ScanLevelProjection.build(
-        RowSetTestUtils.projectAll(),
-        Lists.newArrayList(metadataManager.projectionParser()));
-
-    List<ColumnProjection> cols = scanProj.columns();
-    assertEquals(4, cols.size());
-    assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
-    assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
-    assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-    assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
-    assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
-    assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
-    assertEquals(2, ((PartitionColumn) cols.get(3)).partition());
-  }
-
-  /**
    * Combine wildcard and file metadata columns. The wildcard expands
    * table columns but not metadata columns.
    */
-
   @Test
   public void testLegacyWildcardAndFileMetadata() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataOptions options = standardOptions(filePath);
     options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
         options);
@@ -236,13 +199,11 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
    * As above, but include implicit columns before and after the
    * wildcard.
    */
-
   @Test
   public void testLegacyWildcardAndFileMetadataMixed() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataOptions options = standardOptions(filePath);
     options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
         options);
@@ -271,7 +232,6 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
    * Tests proposed functionality: included only requested partition
    * columns.
    */
-
   @Test
   public void testRevisedWildcardAndPartition() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
@@ -291,37 +251,10 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
   }
 
   @Test
-  public void testLegacyWildcardAndPartition() {
-    Path filePath = new Path("hdfs:///w/x/y/z.csv");
-    FileMetadataOptions options = standardOptions(filePath);
-    options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(true);
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        options);
-
-    ScanLevelProjection scanProj = ScanLevelProjection.build(
-        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
-            ScanTestUtils.partitionColName(8)),
-        Lists.newArrayList(metadataManager.projectionParser()));
-
-      List<ColumnProjection> cols = scanProj.columns();
-      assertEquals(4, cols.size());
-      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
-      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
-      assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
-      assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
-      assertTrue(scanProj.columns().get(3) instanceof PartitionColumn);
-      assertEquals(8, ((PartitionColumn) cols.get(3)).partition());
-  }
-
-  @Test
   public void testPreferredPartitionExpansion() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataOptions options = standardOptions(filePath);
     options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
         options);
@@ -342,43 +275,11 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
       assertEquals(1, ((PartitionColumn) cols.get(3)).partition());
   }
 
-  /**
-   * Test a case like:<br>
-   * <code>SELECT *, dir1 FROM ...</code><br>
-   * The projection list includes "dir1". The wildcard will
-   * fill in "dir0".
-   */
-
-  @Test
-  public void testLegacyWildcardAndPartitionWithOverlap() {
-    Path filePath = new Path("hdfs:///w/x/y/z.csv");
-    FileMetadataOptions options = standardOptions(filePath);
-    options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(true);
-    FileMetadataManager metadataManager = new FileMetadataManager(
-        fixture.getOptionManager(),
-        options);
-
-    ScanLevelProjection scanProj = ScanLevelProjection.build(
-        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR,
-            ScanTestUtils.partitionColName(1)),
-        Lists.newArrayList(metadataManager.projectionParser()));
-
-      List<ColumnProjection> cols = scanProj.columns();
-      assertEquals(3, cols.size());
-      assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
-      assertTrue(scanProj.columns().get(1) instanceof PartitionColumn);
-      assertEquals(0, ((PartitionColumn) cols.get(1)).partition());
-      assertTrue(scanProj.columns().get(2) instanceof PartitionColumn);
-      assertEquals(1, ((PartitionColumn) cols.get(2)).partition());
-  }
-
   @Test
   public void testPreferedWildcardExpansionWithOverlap() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
     FileMetadataOptions options = standardOptions(filePath);
     options.useLegacyWildcardExpansion(true);
-    options.useLegacyExpansionLocation(false);
     FileMetadataManager metadataManager = new FileMetadataManager(
         fixture.getOptionManager(),
         options);
@@ -402,7 +303,6 @@ public class TestFileMetadataColumnParser extends SubOperatorTest {
    * to be maps or arrays, are not interpreted as metadata. That is,
    * the projected table map or array "shadows" the metadata column.
    */
-
   @Test
   public void testShadowed() {
     Path filePath = new Path("hdfs:///w/x/y/z.csv");
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
index 8fd14b7..c3a2243 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/TestScanOrchestratorEarlySchema.java
@@ -53,41 +53,34 @@ import org.junit.experimental.categories.Category;
  * The tests here focus on the scan orchestrator itself; the tests assume
  * that tests for lower-level components have already passed.
  */
-
 @Category(RowSetTests.class)
 public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
   /**
    * Test SELECT * from an early-schema table of (a, b)
    */
-
   @Test
   public void testEarlySchemaWildcard() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT * ...
-
     builder.setProjection(RowSetTestUtils.projectAll());
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     // Simulate a first reader in a scan that can provide an
     // empty batch to define schema.
-
     {
       reader.defineSchema();
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
@@ -99,7 +92,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
 
     // Create a batch of data.
-
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -108,7 +100,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     {
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
           .addRow(1, "fred")
@@ -120,7 +111,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
 
     // Second batch.
-
     reader.startBatch();
     loader.writer()
       .addRow(3, "barney")
@@ -129,7 +119,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     {
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
           .addRow(3, "barney")
@@ -142,7 +131,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Explicit reader close. (All other tests are lazy, they
     // use an implicit close.)
-
     scanner.closeReader();
 
     scanner.close();
@@ -151,36 +139,30 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
   /**
    * Test SELECT a, b FROM table(a, b)
    */
-
   @Test
   public void testEarlySchemaSelectAll() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT a, b ...
-
     builder.setProjection(RowSetTestUtils.projectList("a", "b"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     // Don't bother with an empty batch here or in other tests.
     // Simulates the second reader in a scan.
 
     // Create a batch of data.
-
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -188,7 +170,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
         .addRow(1, "fred")
         .addRow(2, "wilma")
@@ -203,29 +184,24 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
   /**
    * Test SELECT b, a FROM table(a, b)
    */
-
   @Test
   public void testEarlySchemaSelectAllReorder() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT b, a ...
-
     builder.setProjection(RowSetTestUtils.projectList("b", "a"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     TupleMetadata expectedSchema = new SchemaBuilder()
@@ -234,7 +210,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .buildSchema();
 
     // Create a batch of data.
-
    reader.startBatch();
    loader.writer()
      .addRow(1, "fred")
@@ -242,7 +217,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    reader.endBatch();
 
     // Verify
-
    SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
       .addRow("fred", 1)
       .addRow("wilma", 2)
@@ -258,29 +232,24 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * Test SELECT a, b, c FROM table(a, b)
    * c will be null
    */
-
   @Test
   public void testEarlySchemaSelectExtra() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT a, b, c ...
-
     builder.setProjection(RowSetTestUtils.projectList("a", "b", "c"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     TupleMetadata expectedSchema = new SchemaBuilder()
@@ -289,8 +258,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .addNullable("c", MinorType.INT)
         .buildSchema();
 
-   // Create a batch of data.
-
+    // Create a batch of data.
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -314,13 +282,11 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * Test SELECT a, b, c FROM table(a, b)
    * c will be null of type VARCHAR
    */
-
   @Test
   public void testEarlySchemaSelectExtraCustomType() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // Null columns of type VARCHAR
-
     MajorType nullType = MajorType.newBuilder()
         .setMinorType(MinorType.VARCHAR)
         .setMode(DataMode.OPTIONAL)
@@ -328,23 +294,19 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     builder.setNullType(nullType);
 
     // SELECT a, b, c ...
-
     builder.setProjection(RowSetTestUtils.projectList("a", "b", "c"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table ...
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     TupleMetadata expectedSchema = new SchemaBuilder()
@@ -354,7 +316,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .buildSchema();
 
     // Create a batch of data.
-
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -362,7 +323,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
       .addRow(1, "fred", null)
       .addRow(2, "wilma", null)
@@ -377,22 +337,18 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
   /**
    * Test SELECT a FROM table(a, b)
    */
-
   @Test
   public void testEarlySchemaSelectSubset() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT a ...
-
     builder.setProjection(RowSetTestUtils.projectList("a"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
@@ -404,7 +360,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Verify that unprojected column is unprojected in the
     // table loader.
-
     assertFalse(loader.writer().column("b").isProjected());
 
     TupleMetadata expectedSchema = new SchemaBuilder()
@@ -412,7 +367,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
         .buildSchema();
 
     // Create a batch of data.
-
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -420,7 +374,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
       .addRow(1)
       .addRow(2)
@@ -435,47 +388,39 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
   /**
    * Test SELECT - FROM table(a, b)
    */
-
   @Test
   public void testEarlySchemaSelectNone() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT ...
     // (Like SELECT COUNT(*) ...
-
     builder.setProjection(RowSetTestUtils.projectList());
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema (a, b)
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.VARCHAR)
         .buildSchema();
 
     // Create the table loader
-
     ResultSetLoader loader = reader.makeTableLoader(tableSchema);
 
     // Verify that unprojected column is unprojected in the
     // table loader.
-
     assertTrue(loader.isProjectionEmpty());
     assertFalse(loader.writer().column("a").isProjected());
     assertFalse(loader.writer().column("b").isProjected());
 
     // Verify empty batch.
-
     BatchSchema expectedSchema = new BatchSchemaBuilder()
         .withSchemaBuilder(new SchemaBuilder())
         .build();
 
     // Create a batch of data.
-
     reader.startBatch();
     loader.writer()
       .addRow(1, "fred")
@@ -483,10 +428,8 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     reader.endBatch();
 
     // Verify
-
     {
       // Two rows, no data.
-
       SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
         .addRow()
         .addRow()
@@ -497,13 +440,11 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
 
     // Fast path to fill in empty rows
-
     reader.startBatch();
     loader.skipRows(10);
     reader.endBatch();
 
     // Verify
-
     {
       VectorContainer output = scanner.output();
       assertEquals(10, output.getRecordCount());
@@ -517,37 +458,30 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * Test SELECT * from an early-schema table of () (that is,
    * a schema that consists of zero columns.
    */
-
   @Test
   public void testEmptySchema() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT * ...
-
     builder.setProjection(RowSetTestUtils.projectAll());
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema ()
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .buildSchema();
 
     // Create the table loader
-
     reader.makeTableLoader(tableSchema);
 
     // Create a batch of data. Because there are no columns, it does
     // not make sense to ready any rows.
-
     reader.startBatch();
     reader.endBatch();
 
     // Verify
-
     {
       SingleRowSet expected = fixture.rowSetBuilder(tableSchema)
           .build();
@@ -559,32 +493,26 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     scanner.close();
   }
 
-
   /**
    * Test SELECT a from an early-schema table of () (that is,
    * a schema that consists of zero columns.
    */
-
   @Test
   public void testEmptySchemaExtra() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT * ...
-
     builder.setProjection(RowSetTestUtils.projectList("a"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // ... FROM table
-
     ReaderSchemaOrchestrator reader = scanner.startReader();
 
     // file schema ()
-
     TupleMetadata tableSchema = new SchemaBuilder()
         .buildSchema();
 
     // Create the table loader
-
     reader.makeTableLoader(tableSchema);
 
     TupleMetadata expectedSchema = new SchemaBuilder()
@@ -593,12 +521,10 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
 
     // Create a batch of data. Because there are no columns, it does
     // not make sense to ready any rows.
-
     reader.startBatch();
     reader.endBatch();
 
     // Verify
-
     SingleRowSet expected = fixture.rowSetBuilder(expectedSchema)
         .build();
     RowSetUtilities.verify(expected,
@@ -621,7 +547,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * The result in all cases should be
    * <tt>(a : BIGINT, b: VARCHAR)</tt>
    */
-
   @Test
   public void testTypeSmoothingExplicit() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
@@ -634,14 +559,12 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     SchemaTracker tracker = new SchemaTracker();
 
     // SELECT * ...
-
     builder.setProjection(RowSetTestUtils.projectList("a", "b", "c"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     int schemaVersion;
     {
       // ... FROM table1(a, b, c)
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
       reader.makeTableLoader(table1Schema);
       reader.defineSchema();
@@ -657,7 +580,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       //
       // B is dropped. But, it is nullable, so the vector cache
       // can supply the proper type to ensure continuity.
-
       TupleMetadata table2Schema = new SchemaBuilder()
           .add("A", MinorType.BIGINT)
           .addArray("C", MinorType.INT)
@@ -677,7 +599,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       //
       // C is dropped. But, it is an array, which uses zero-elements
       // to indicate null, so the vector cache can fill in the type.
-
       TupleMetadata table3Schema = new SchemaBuilder()
           .add("A", MinorType.BIGINT)
           .addNullable("B", MinorType.VARCHAR)
@@ -698,7 +619,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       // This version carries over a non-nullable BIGINT, but that
       // can't become a null column, so nullable BIGINT is substituted,
       // result in a schema change.
-
       TupleMetadata table2Schema = new SchemaBuilder()
           .addNullable("B", MinorType.VARCHAR)
           .addArray("C", MinorType.INT)
@@ -734,18 +654,15 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * As a side effect, makes sure that two identical tables (in this case,
    * separated by a different table) results in no schema change.
    */
-
   @Test
   public void testTypeSmoothing() {
     ScanOrchestratorBuilder builder = new MockScanBuilder();
 
     // SELECT a, b ...
-
     builder.setProjection(RowSetTestUtils.projectList("a", "b"));
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // file schema (a, b)
-
     TupleMetadata twoColSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .addNullable("b", MinorType.VARCHAR, 10)
@@ -755,12 +672,10 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     int schemaVersion;
     {
       // ... FROM table 1
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
       ResultSetLoader loader = reader.makeTableLoader(twoColSchema);
 
       // Projection of (a, b) to (a, b)
-
       reader.startBatch();
       loader.writer()
           .addRow(10, "fred")
@@ -779,17 +694,14 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
     {
       // ... FROM table 2
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
 
       // File schema (a)
-
       TupleMetadata oneColSchema = new SchemaBuilder()
           .add("a", MinorType.INT)
           .buildSchema();
 
       // Projection of (a) to (a, b), reusing b from above.
-
       ResultSetLoader loader = reader.makeTableLoader(oneColSchema);
 
       reader.startBatch();
@@ -810,11 +722,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
     {
       // ... FROM table 3
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
 
       // Projection of (a, b), to (a, b), reusing b yet again
-
       ResultSetLoader loader = reader.makeTableLoader(twoColSchema);
 
       reader.startBatch();
@@ -845,7 +755,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     ScanSchemaOrchestrator scanner = new ScanSchemaOrchestrator(fixture.allocator(), builder);
 
     // Most general schema: nullable, with precision.
-
     TupleMetadata schema1 = new SchemaBuilder()
         .addNullable("a", MinorType.VARCHAR, 10)
         .buildSchema();
@@ -859,7 +768,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ResultSetLoader loader = reader.makeTableLoader(schema1);
 
       // Create a batch
-
       reader.startBatch();
       loader.writer()
         .addRow("fred")
@@ -869,7 +777,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       schemaVersion = tracker.schemaVersion();
 
       // Verify
-
       SingleRowSet expected = fixture.rowSetBuilder(schema1)
         .addRow("fred")
         .addRow("wilma")
@@ -883,7 +790,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       // Table 2: required, use nullable
 
       // Required version.
-
       TupleMetadata schema2 = new SchemaBuilder()
           .add("a", MinorType.VARCHAR, 10)
           .buildSchema();
@@ -891,9 +797,7 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ReaderSchemaOrchestrator reader = scanner.startReader();
       ResultSetLoader loader = reader.makeTableLoader(schema2);
 
-
       // Create a batch
-
       reader.startBatch();
       loader.writer()
         .addRow("barney")
@@ -901,7 +805,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       reader.endBatch();
 
       // Verify, using persistent schema
-
       tracker.trackSchema(scanner.output());
       assertEquals(schemaVersion, tracker.schemaVersion());
       SingleRowSet expected = fixture.rowSetBuilder(schema1)
@@ -917,7 +820,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       // Table 3: narrower precision, use wider
 
       // Required version with narrower precision.
-
       TupleMetadata schema3 = new SchemaBuilder()
           .add("a", MinorType.VARCHAR, 5)
           .buildSchema();
@@ -926,7 +828,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       ResultSetLoader loader = reader.makeTableLoader(schema3);
 
       // Create a batch
-
       reader.startBatch();
       loader.writer()
         .addRow("bam-bam")
@@ -934,7 +835,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
       reader.endBatch();
 
       // Verify, using persistent schema
-
       tracker.trackSchema(scanner.output());
       assertEquals(schemaVersion, tracker.schemaVersion());
 
@@ -956,7 +856,6 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
    * SELECT order, preserving vectors, so no schema change for column
    * reordering.
    */
-
   @Test
   public void testColumnReordering() {
 
@@ -985,11 +884,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     int schemaVersion;
     {
       // ... FROM table 1
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
 
       // Projection of (a, b, c) to (a, b, c)
-
       ResultSetLoader loader = reader.makeTableLoader(schema1);
 
       reader.startBatch();
@@ -1012,11 +909,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
     {
       // ... FROM table 2
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
 
       // Projection of (c, a, b) to (a, b, c)
-
       ResultSetLoader loader = reader.makeTableLoader(schema2);
 
       reader.startBatch();
@@ -1037,11 +932,9 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
     }
     {
       // ... FROM table 3
-
       ReaderSchemaOrchestrator reader = scanner.startReader();
 
       // Projection of (a, c, b) to (a, b, c)
-
       ResultSetLoader loader = reader.makeTableLoader(schema3);
 
       reader.startBatch();
@@ -1065,5 +958,4 @@ public class TestScanOrchestratorEarlySchema extends SubOperatorTest {
   }
 
   // TODO: Start with early schema, but add columns
-
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
index 1eff25f..cc24cb7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestScanLevelProjection.java
@@ -22,10 +22,8 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.physical.impl.scan.ScanTestUtils;
@@ -35,9 +33,8 @@ import org.apache.drill.exec.physical.impl.scan.project.ScanLevelProjection.Scan
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet.ColumnReadProjection;
 import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
+import org.apache.drill.exec.physical.resultSet.project.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -49,7 +46,6 @@ import org.junit.experimental.categories.Category;
  * Test the level of projection done at the level of the scan as a whole;
  * before knowledge of table "implicit" columns or the specific table schema.
  */
-
 @Category(RowSetTests.class)
 public class TestScanLevelProjection extends SubOperatorTest {
 
@@ -58,7 +54,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
    * data source has an early schema of (a, c, d). (a, c) are
    * projected, (d) is null.
    */
-
   @Test
   public void testBasics() {
 
@@ -83,18 +78,15 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals("c", scanProj.columns().get(2).name());
 
     // Verify column type
-
     assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumn);
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(3, outputProj.projections().size());
     assertNotNull(outputProj.get("a"));
     assertTrue(outputProj.get("a").isSimple());
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.INT)
@@ -112,13 +104,11 @@ public class TestScanLevelProjection extends SubOperatorTest {
    * a dot, such as "a.b". We may not know the type of "b", but have
    * just learned that "a" must be a map.
    */
-
   @Test
   public void testMap() {
 
     // SELECT a.x, b.x, a.y, b.y, c
     // We infer a and b are maps.
-
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
         RowSetTestUtils.projectList("a.x", "b.x", "a.y", "b.y", "c"),
         ScanTestUtils.parsers());
@@ -132,29 +122,25 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals("c", scanProj.columns().get(2).name());
 
     // Verify column type
-
     assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumn);
 
     // Inferred map structure
-
     final RequestedColumn a = ((UnresolvedColumn) scanProj.columns().get(0)).element();
     assertTrue(a.isTuple());
-    assertEquals(ProjectionType.GENERAL, a.mapProjection().projectionType("x"));
-    assertEquals(ProjectionType.GENERAL, a.mapProjection().projectionType("y"));
-    assertEquals(ProjectionType.UNPROJECTED,  a.mapProjection().projectionType("z"));
+    assertTrue(a.tuple().isProjected("x"));
+    assertTrue(a.tuple().isProjected("y"));
+    assertFalse(a.tuple().isProjected("z"));
 
     final RequestedColumn c = ((UnresolvedColumn) scanProj.columns().get(2)).element();
     assertTrue(c.isSimple());
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(3, outputProj.projections().size());
     assertNotNull(outputProj.get("a"));
     assertTrue(outputProj.get("a").isTuple());
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .addMap("a")
           .add("x", MinorType.INT)
@@ -173,20 +159,19 @@ public class TestScanLevelProjection extends SubOperatorTest {
     // an actual reader.
 
     ProjectionSet projSet = scanProj.projectionSet().build();
+    assertTrue(projSet.isProjected("a"));
     ColumnReadProjection aProj = projSet.readProjection(readerSchema.metadata("a"));
     assertTrue(aProj.isProjected());
-    assertEquals(ProjectionType.TUPLE, aProj.projectionType());
     ColumnReadProjection cProj = projSet.readProjection(readerSchema.metadata("c"));
     assertTrue(cProj.isProjected());
-    assertEquals(ProjectionType.GENERAL, cProj.projectionType());
     assertFalse(projSet.readProjection(readerSchema.metadata("d")).isProjected());
+    assertFalse(projSet.isProjected("d"));
   }
 
   /**
    * Similar to maps, if the project list contains "a[1]" then we've learned that
    * a is an array, but we don't know what type.
    */
-
   @Test
   public void testArray() {
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -200,11 +185,9 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals("a", scanProj.columns().get(0).name());
 
     // Verify column type
-
     assertTrue(scanProj.columns().get(0) instanceof UnresolvedColumn);
 
     // Map structure
-
     final RequestedColumn a = ((UnresolvedColumn) scanProj.columns().get(0)).element();
     assertTrue(a.isArray());
     assertFalse(a.hasIndex(0));
@@ -213,14 +196,12 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertTrue(a.hasIndex(3));
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(1, outputProj.projections().size());
     assertNotNull(outputProj.get("a"));
     assertTrue(outputProj.get("a").isArray());
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .addArray("a", MinorType.INT)
         .add("c", MinorType.INT)
@@ -229,7 +210,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
     ProjectionSet projSet = scanProj.projectionSet().build();
     ColumnReadProjection aProj = projSet.readProjection(readerSchema.metadata("a"));
     assertTrue(aProj.isProjected());
-    assertEquals(ProjectionType.ARRAY, aProj.projectionType());
     assertFalse(projSet.readProjection(readerSchema.metadata("c")).isProjected());
   }
 
@@ -237,7 +217,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
    * Simulate a SELECT * query by passing "**" (Drill's internal representation
    * of the wildcard) as a column name.
    */
-
   @Test
   public void testWildcard() {
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -253,22 +232,18 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals(SchemaPath.DYNAMIC_STAR, scanProj.columns().get(0).name());
 
     // Verify bindings
-
     assertEquals(scanProj.columns().get(0).name(), scanProj.requestedCols().get(0).rootName());
 
     // Verify column type
-
     assertTrue(scanProj.columns().get(0) instanceof UnresolvedWildcardColumn);
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(1, outputProj.projections().size());
     assertNotNull(outputProj.get(SchemaPath.DYNAMIC_STAR));
     assertTrue(outputProj.get(SchemaPath.DYNAMIC_STAR).isWildcard());
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("c", MinorType.INT)
@@ -283,7 +258,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
    * Test an empty projection which occurs in a
    * SELECT COUNT(*) query.
    */
-
   @Test
   public void testEmptyProjection() {
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -295,12 +269,10 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals(0, scanProj.requestedCols().size());
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(0, outputProj.projections().size());
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .buildSchema();
@@ -314,7 +286,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
    * operator will fill in the column, the scan framework just ignores
    * the extra column.
    */
-
   @Test
   public void testWildcardAndColumns() {
     ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -327,7 +298,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertEquals(1, scanProj.columns().size());
 
     // Verify tuple projection
-
     RequestedTuple outputProj = scanProj.rootProjection();
     assertEquals(2, outputProj.projections().size());
     assertNotNull(outputProj.get(SchemaPath.DYNAMIC_STAR));
@@ -335,7 +305,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertNotNull(outputProj.get("a"));
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("c", MinorType.INT)
@@ -349,7 +318,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
   /**
    * Test a column name and a wildcard.
    */
-
   @Test
   public void testColumnAndWildcard() {
     ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -363,22 +331,17 @@ public class TestScanLevelProjection extends SubOperatorTest {
   }
 
   /**
-   * Can't include a wildcard twice.
+   * Wildcard included twice is benign
    * <p>
    * Note: Drill actually allows this, but the work should be done
    * in the project operator; scan should see at most one wildcard.
    */
-
   @Test
-  public void testErrorTwoWildcards() {
-    try {
-      ScanLevelProjection.build(
-          RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR, SchemaPath.DYNAMIC_STAR),
-          ScanTestUtils.parsers());
-      fail();
-    } catch (final UserException e) {
-      // Expected
-    }
+  public void testTwoWildcards() {
+    ScanLevelProjection scanProj = ScanLevelProjection.build(
+        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR, SchemaPath.DYNAMIC_STAR),
+        ScanTestUtils.parsers());
+    assertTrue(scanProj.projectAll());
   }
 
   @Test
@@ -386,7 +349,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
     TupleMetadata outputSchema = new SchemaBuilder().buildSchema();
 
     // Simulate SELECT a
-
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
         RowSetTestUtils.projectList("a"),
         ScanTestUtils.parsers(),
@@ -437,7 +399,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
         .buildSchema();
 
     // Mark b as special; not expanded in wildcard.
-
     outputSchema.metadata("b").setBooleanProperty(ColumnMetadata.EXCLUDE_FROM_WILDCARD, true);
 
     final ScanLevelProjection scanProj = ScanLevelProjection.build(
@@ -478,7 +439,6 @@ public class TestScanLevelProjection extends SubOperatorTest {
     assertTrue(scanProj.columns().get(1) instanceof UnresolvedColumn);
 
     // Make up a reader schema and test the projection set.
-
     TupleMetadata readerSchema = new SchemaBuilder()
         .add("a", MinorType.INT)
         .add("b", MinorType.INT)
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
index 1b711eb..b299511 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/TestSchemaSmoothing.java
@@ -93,7 +93,6 @@ public class TestSchemaSmoothing extends SubOperatorTest {
 
   private FileMetadataOptions standardOptions(List<Path> files) {
     FileMetadataOptions options = new FileMetadataOptions();
-    options.useLegacyWildcardExpansion(false); // Don't expand partition columns for wildcard
     options.setSelectionRoot(new Path("hdfs:///w"));
     options.setFiles(files);
     return options;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TestProjectionSet.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TestProjectionSet.java
index a59065d..caa1d25 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TestProjectionSet.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/scan/project/projSet/TestProjectionSet.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl.scan.project.projSet;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -32,7 +31,6 @@ import org.apache.drill.exec.physical.impl.scan.project.projSet.TypeConverter.Cu
 import org.apache.drill.exec.physical.resultSet.ProjectionSet;
 import org.apache.drill.exec.physical.resultSet.ProjectionSet.ColumnReadProjection;
 import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.physical.resultSet.project.ProjectionType;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -56,14 +54,12 @@ import org.junit.experimental.categories.Category;
  * of projection should be fully tested here, then just sanity tested
  * in the result set loader.
  */
-
 @Category(RowSetTests.class)
 public class TestProjectionSet extends BaseTest {
 
   /**
    * Empty projection, no schema
    */
-
   @Test
   public void testEmptyProjection() {
     ProjectionSet projSet = ProjectionSetFactory.projectNone();
@@ -85,7 +81,6 @@ public class TestProjectionSet extends BaseTest {
     assertSame(aSchema, aCol.providedSchema());
     assertNull(aCol.conversionFactory());
     assertSame(EmptyProjectionSet.PROJECT_NONE, aCol.mapProjection());
-    assertNull(aCol.projectionType());
 
     ColumnReadProjection mCol = projSet.readProjection(readSchema.metadata("m"));
     assertFalse(mCol.isProjected());
@@ -98,7 +93,6 @@ public class TestProjectionSet extends BaseTest {
   /**
    * Wildcard projection, no schema
    */
-
   @Test
   public void testWildcardProjection() {
     ProjectionSet projSet = ProjectionSetFactory.projectAll();
@@ -114,13 +108,11 @@ public class TestProjectionSet extends BaseTest {
     assertSame(aSchema, aCol.providedSchema());
     assertNull(aCol.conversionFactory());
     assertNull(aCol.mapProjection());
-    assertNull(aCol.projectionType());
   }
 
   /**
    * Wildcard projection, no schema
    */
-
   @Test
   public void testWildcardMapProjection() {
     ProjectionSet projSet = ProjectionSetFactory.projectAll();
@@ -143,7 +135,6 @@ public class TestProjectionSet extends BaseTest {
    * Wildcard projection, with schema. Some columns marked
    * as special; not expanded by wildcard.
    */
-
   @Test
   public void testWildcardAndSchemaProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -198,7 +189,6 @@ public class TestProjectionSet extends BaseTest {
    * Wildcard projection, with schema. Some columns marked
    * as special; not expanded by wildcard.
    */
-
   @Test
   public void testWildcardAndSchemaMapProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -265,7 +255,6 @@ public class TestProjectionSet extends BaseTest {
   /**
    * Wildcard and strict schema
    */
-
   @Test
   public void testWildcardAndStrictSchemaProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -303,7 +292,6 @@ public class TestProjectionSet extends BaseTest {
   /**
    * Wildcard and strict schema
    */
-
   @Test
   public void testWildcardAndStrictMapSchemaProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -369,7 +357,6 @@ public class TestProjectionSet extends BaseTest {
    * Also, sanity test of the builder for the project all,
    * project none cases.
    */
-
   @Test
   public void testExplicitProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -388,7 +375,6 @@ public class TestProjectionSet extends BaseTest {
     assertSame(aSchema, aCol.providedSchema());
     assertNull(aCol.conversionFactory());
     assertNull(aCol.mapProjection());
-    assertEquals(ProjectionType.GENERAL, aCol.projectionType());
 
     ColumnReadProjection bCol = projSet.readProjection(readSchema.metadata("b"));
     assertFalse(bCol.isProjected());
@@ -431,13 +417,11 @@ public class TestProjectionSet extends BaseTest {
     assertSame(m1Schema, m1Col.readSchema());
     assertSame(m1Schema, m1Col.providedSchema());
     assertNull(m1Col.conversionFactory());
-    assertEquals(ProjectionType.TUPLE, m1Col.projectionType());
 
     // m1.c is projected
 
     ColumnReadProjection cCol = m1Col.mapProjection().readProjection(m1ReadSchema.metadata("c"));
     assertTrue(cCol.isProjected());
-    assertEquals(ProjectionType.GENERAL, cCol.projectionType());
 
     // but m1.d is not projected
 
@@ -446,13 +430,11 @@ public class TestProjectionSet extends BaseTest {
     // m2 is entirely projected
 
     ColumnReadProjection m2Col = projSet.readProjection(m2Schema);
-    assertEquals(ProjectionType.GENERAL, m2Col.projectionType());
     assertTrue(m2Col.isProjected());
     assertSame(m2Schema, m2Col.readSchema());
     assertSame(m2Schema, m2Col.providedSchema());
     assertNull(m2Col.conversionFactory());
     assertTrue(m2Col.mapProjection() instanceof WildcardProjectionSet);
-    assertEquals(ProjectionType.GENERAL, m2Col.projectionType());
     assertTrue(m2Col.mapProjection().readProjection(m2ReadSchema.metadata("e")).isProjected());
 
     // m3 is not projected at all
@@ -487,7 +469,6 @@ public class TestProjectionSet extends BaseTest {
 
     ColumnReadProjection m1Col = projSet.readProjection(m1Schema);
     assertTrue(m1Col.isProjected());
-    assertEquals(ProjectionType.TUPLE, m1Col.projectionType());
 
     // M1.c is projected
 
@@ -504,7 +485,6 @@ public class TestProjectionSet extends BaseTest {
    * That is, SELECT m is logically equivalent to SELECT m.*
    * and is subject to the strict schema projection rule.
    */
-
   @Test
   public void testImpliedWildcardWithStrictSchema() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -544,7 +524,6 @@ public class TestProjectionSet extends BaseTest {
    * Wildcard and none already tested above, here we test the
    * builder. With schema.
    */
-
   @Test
   public void testExplicitSchemaProjection() {
     TupleMetadata readSchema = new SchemaBuilder()
@@ -603,7 +582,6 @@ public class TestProjectionSet extends BaseTest {
   /**
    * Wildcard projection, no schema, custom column transform.
    */
-
   @Test
   public void testTransformConversion() {
     ColumnConversionFactory conv = StandardConversions.factory(ConvertStringToInt.class);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/RowSetTestUtils.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/RowSetTestUtils.java
index 5c9f76d..9f4266f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/RowSetTestUtils.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/impl/RowSetTestUtils.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.resultSet.impl;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.drill.common.expression.SchemaPath;
@@ -30,7 +31,7 @@ public class RowSetTestUtils {
   public static List<SchemaPath> projectList(String... names) {
     List<SchemaPath> selected = new ArrayList<>();
     for (String name : names) {
-      if (name.equals(SchemaPath.DYNAMIC_STAR)) {
+      if (name.equals(SchemaPath.DYNAMIC_STAR) || name.equals("*")) {
         selected.add(SchemaPath.STAR_COLUMN);
       } else {
         selected.add(SchemaPath.parseFromString(name));
@@ -58,6 +59,10 @@ public class RowSetTestUtils {
         new SchemaPath[] {SchemaPath.STAR_COLUMN});
   }
 
+  public static List<SchemaPath> projectNone() {
+    return Collections.emptyList();
+  }
+
   @SafeVarargs
   public static List<SchemaPath> concat(List<SchemaPath>... parts) {
     List<SchemaPath> selected = new ArrayList<>();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedPath.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedPath.java
new file mode 100644
index 0000000..4f7104d
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedPath.java
@@ -0,0 +1,325 @@
+/*
+ * 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.physical.resultSet.project;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.physical.impl.scan.project.projSet.ProjectionChecker;
+import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
+import org.apache.drill.exec.record.metadata.ColumnMetadata;
+import org.apache.drill.exec.record.metadata.MetadataUtils;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.junit.Test;
+
+/**
+ * Projection creates a pattern which we match against a particular type
+ * to see if the projection path is consistent with the type. Tests here
+ * verify the consistency checks.
+ */
+public class TestProjectedPath {
+
+  // INT is a proxy for all scalar columns.
+  private static final ColumnMetadata INT_COLUMN = intSchema().metadata("a");
+  private static final ColumnMetadata INT_ARRAY_COLUMN = intArraySchema().metadata("a");
+  private static final ColumnMetadata MAP_COLUMN = mapSchema().metadata("a");
+  private static final ColumnMetadata MAP_ARRAY_COLUMN = mapArraySchema().metadata("a");
+  private static final ColumnMetadata UNION_COLUMN = unionSchema().metadata("a");
+  private static final ColumnMetadata LIST_COLUMN = listSchema().metadata("a");
+  private static final ColumnMetadata DICT_INT_INT_COLUMN = dictSchema(MinorType.INT).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_INT_INT_COLUMN = dictArraySchema(MinorType.INT).metadata("a");
+  private static final ColumnMetadata DICT_BIGINT_INT_COLUMN = dictSchema(MinorType.BIGINT).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_BIGINT_INT_COLUMN = dictArraySchema(MinorType.BIGINT).metadata("a");
+  private static final ColumnMetadata DICT_VARCHAR_INT_COLUMN = dictSchema(MinorType.VARCHAR).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_VARCHAR_INT_COLUMN = dictArraySchema(MinorType.VARCHAR).metadata("a");
+  private static final ColumnMetadata DICT_DOUBLE_INT_COLUMN = dictSchema(MinorType.FLOAT8).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_DOUBLE_INT_COLUMN = dictArraySchema(MinorType.FLOAT8).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_INT_INT_ARRAY_COLUMN = dictArrayArraySchema(MinorType.INT).metadata("a");
+  private static final ColumnMetadata DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN = dictArrayArraySchema(MinorType.VARCHAR).metadata("a");
+
+  private static TupleMetadata intSchema() {
+    return new SchemaBuilder()
+        .add("a", MinorType.INT)
+        .build();
+  }
+
+  private static TupleMetadata intArraySchema() {
+    return new SchemaBuilder()
+        .addArray("a", MinorType.INT)
+        .build();
+  }
+
+  private static TupleMetadata mapSchema() {
+    return new SchemaBuilder()
+        .addMap("a")
+          .add("i", MinorType.INT)
+          .addMap("m")
+            .add("mi", MinorType.INT)
+            .resumeMap()
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata mapArraySchema() {
+    return new SchemaBuilder()
+        .addMapArray("a")
+          .add("i", MinorType.INT)
+          .addMap("m")
+            .add("mi", MinorType.INT)
+            .resumeMap()
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata dictSchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDict("a", keyType)
+          .value(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata dictArraySchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDictArray("a", keyType)
+          .value(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata dictArrayArraySchema(MinorType keyType) {
+    return new SchemaBuilder()
+        .addDictArray("a", keyType)
+          .value(Types.repeated(MinorType.INT))
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata unionSchema() {
+    return new SchemaBuilder()
+        .addUnion("a")
+          .addType(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private static TupleMetadata listSchema() {
+    return new SchemaBuilder()
+        .addList("a")
+          .addType(MinorType.INT)
+          .resumeSchema()
+        .build();
+  }
+
+  private void assertConsistent(RequestedTuple projSet, ColumnMetadata col) {
+    assertTrue(ProjectionChecker.isConsistent(projSet, col));
+  }
+
+  private void assertNotConsistent(RequestedTuple projSet, ColumnMetadata col) {
+    assertFalse(ProjectionChecker.isConsistent(projSet, col));
+  }
+
+  private void assertAllConsistent(RequestedTuple projSet) {
+    assertConsistent(projSet, INT_COLUMN);
+    assertConsistent(projSet, INT_ARRAY_COLUMN);
+    assertConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+    assertConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+  }
+
+  @Test
+  public void testSimplePath() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a"));
+
+    assertAllConsistent(projSet);
+
+    // No constraints on an unprojected column.
+
+    assertTrue(ProjectionChecker.isConsistent(projSet,
+        MetadataUtils.newScalar("b", Types.required(MinorType.INT))));
+  }
+
+  @Test
+  public void testProjectAll() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectAll());
+
+    // No constraints on wildcard projection
+    assertAllConsistent(projSet);
+  }
+
+  @Test
+  public void testProjectNone() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectNone());
+
+    // No constraints on empty projection
+    assertAllConsistent(projSet);
+  }
+
+  @Test
+  public void test1DArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[0]"));
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void test2DArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[0][1]"));
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertNotConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void test3DArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[0][1][2]"));
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertNotConsistent(projSet, MAP_ARRAY_COLUMN);
+    assertConsistent(projSet, UNION_COLUMN);
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void testMap() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a.b"));
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+
+    // A UNION could contain a map, which would allow the
+    // a.b path to be valid.
+    assertConsistent(projSet, UNION_COLUMN);
+    // A LIST could be a list of MAPs, so a.b could mean
+    // to pick out the b column in all array entries.
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+
+  @Test
+  public void testMapArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[0].b"));
+
+    assertNotConsistent(projSet, INT_COLUMN);
+    assertNotConsistent(projSet, INT_ARRAY_COLUMN);
+    assertNotConsistent(projSet, MAP_COLUMN);
+    assertConsistent(projSet, MAP_ARRAY_COLUMN);
+
+    // A UNION could contain a repeated map, which would allow the
+    // a.b path to be valid.
+    assertConsistent(projSet, UNION_COLUMN);
+    // A LIST could contain MAPs.
+    assertConsistent(projSet, LIST_COLUMN);
+
+    assertConsistent(projSet, DICT_INT_INT_COLUMN);
+
+    // TODO: Enforce specific DICT keys, if needed.
+//    assertDictNotConsistent(projSet, DICT_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_INT_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_BIGINT_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_COLUMN);
+//    assertDictConsistent(projSet, DICT_ARRAY_VARCHAR_INT_ARRAY_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_DOUBLE_INT_COLUMN);
+//    assertDictNotConsistent(projSet, DICT_ARRAY_DOUBLE_INT_COLUMN);
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedTuple.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedTuple.java
index b4700a6..a9017a4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedTuple.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectedTuple.java
@@ -21,18 +21,15 @@ 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.assertSame;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.resultSet.impl.RowSetTestUtils;
-import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.RequestedColumn;
 import org.apache.drill.exec.physical.resultSet.project.RequestedTuple.TupleProjectionType;
 import org.apache.drill.test.BaseTest;
 import org.junit.Test;
@@ -48,186 +45,211 @@ import org.junit.experimental.categories.Category;
  * parsing; the only bits not tested here is that which is
  * inherently specific to some use case.
  */
-
 @Category(RowSetTests.class)
 public class TestProjectedTuple extends BaseTest {
 
+  /**
+   * Null map means everything is projected
+   */
   @Test
   public void testProjectionAll() {
+    RequestedTuple projSet = Projections.parse(null);
+    assertSame(TupleProjectionType.ALL, projSet.type());
+    assertTrue(projSet.isProjected("foo"));
+    assertTrue(projSet.projections().isEmpty());
+  }
 
-    // Null map means everything is projected
-
-    RequestedTuple projSet = RequestedTupleImpl.parse(null);
-    assertEquals(TupleProjectionType.ALL, projSet.type());
-    // Not defined well; the tuple contains a wildcard
-    // assertEquals(ProjectionType.GENERAL, projSet.projectionType("foo"));
-
-    projSet = ImpliedTupleRequest.ALL_MEMBERS;
-    assertTrue(projSet instanceof ImpliedTupleRequest);
-    assertEquals(ProjectionType.GENERAL, projSet.projectionType("foo"));
+  /**
+   * SELECT * means everything is projected
+   */
+  @Test
+  public void testWildcard() {
+    RequestedTuple projSet = Projections.parse(RowSetTestUtils.projectAll());
+    assertSame(TupleProjectionType.ALL, projSet.type());
+    assertTrue(projSet.isProjected("foo"));
+    assertNull(projSet.get("foo"));
+    assertEquals(1, projSet.projections().size());
   }
 
   /**
    * Test an empty projection which occurs in a
    * SELECT COUNT(*) query.
+   * Empty list means nothing is projected.
    */
-
   @Test
   public void testProjectionNone() {
-
-    // Empty list means nothing is projected
-
-    RequestedTuple projSet = RequestedTupleImpl.parse(Collections.emptyList());
-    assertEquals(TupleProjectionType.NONE, projSet.type());
-    assertTrue(projSet instanceof ImpliedTupleRequest);
-    List<RequestedColumn> cols = projSet.projections();
-    assertEquals(0, cols.size());
-    assertEquals(ProjectionType.UNPROJECTED, projSet.projectionType("foo"));
+    RequestedTuple projSet = Projections.parse(new ArrayList<SchemaPath>());
+    assertSame(TupleProjectionType.NONE, projSet.type());
+    assertFalse(projSet.isProjected("foo"));
+    assertTrue(projSet.projections().isEmpty());
   }
 
+  /**
+   * Simple non-map columns
+   */
   @Test
   public void testProjectionSimple() {
-
-    // Simple non-map columns
-
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a", "b", "c"));
-    assertTrue(projSet instanceof RequestedTupleImpl);
-    assertEquals(ProjectionType.GENERAL, projSet.projectionType("a"));
-    assertEquals(ProjectionType.GENERAL, projSet.projectionType("b"));
-    assertEquals(ProjectionType.UNPROJECTED, projSet.projectionType("d"));
+    assertSame(TupleProjectionType.SOME, projSet.type());
+    assertTrue(projSet.isProjected("a"));
+    assertTrue(projSet.isProjected("b"));
+    assertTrue(projSet.isProjected("c"));
+    assertFalse(projSet.isProjected("d"));
 
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(3, cols.size());
 
     RequestedColumn a = cols.get(0);
     assertEquals("a", a.name());
-    assertEquals(ProjectionType.GENERAL, a.type());
     assertTrue(a.isSimple());
-    assertFalse(a.isWildcard());
-
-    // We don't know if a is a map or not (the simple term "a" under-determines
-    // the column type.) In case it is a map, we assume all of the map is
-    // projected.
-
-    assertNotNull(a.mapProjection());
-    assertEquals(TupleProjectionType.ALL, a.mapProjection().type());
-    assertNull(a.indexes());
-
-    assertEquals("b", cols.get(1).name());
-    assertEquals(ProjectionType.GENERAL, cols.get(1).type());
-    assertTrue(cols.get(1).isSimple());
+    assertFalse(a.isArray());
+    assertFalse(a.isTuple());
+  }
 
-    assertEquals("c", cols.get(2).name());
-    assertEquals(ProjectionType.GENERAL, cols.get(2).type());
-    assertTrue(cols.get(2).isSimple());
+  /**
+   * The projection set does not enforce uniqueness.
+   */
+  @Test
+  public void testSimpleDups() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a", "b", "a"));
+    assertSame(TupleProjectionType.SOME, projSet.type());
+    assertEquals(2, projSet.projections().size());
+    assertEquals(2, ((RequestedColumnImpl) projSet.get("a")).refCount());
   }
 
+  /**
+   * Whole-map projection (note, fully projected maps are
+   * identical to projected simple columns at this level of
+   * abstraction.)
+   */
   @Test
   public void testProjectionWholeMap() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("map"));
 
-    // Whole-map projection (note, fully projected maps are
-    // identical to projected simple columns at this level of
-    // abstraction.)
-
-    List<SchemaPath> projCols = new ArrayList<>();
-    projCols.add(SchemaPath.getSimplePath("map"));
-    RequestedTuple projSet = RequestedTupleImpl.parse(projCols);
+    assertSame(TupleProjectionType.SOME, projSet.type());
+    assertTrue(projSet.isProjected("map"));
+    assertFalse(projSet.isProjected("another"));
 
-    assertTrue(projSet instanceof RequestedTupleImpl);
-    assertEquals(ProjectionType.GENERAL, projSet.projectionType("map"));
-    assertEquals(ProjectionType.UNPROJECTED, projSet.projectionType("another"));
     RequestedTuple mapProj = projSet.mapProjection("map");
     assertNotNull(mapProj);
-    assertTrue(mapProj instanceof ImpliedTupleRequest);
-    assertEquals(ProjectionType.GENERAL, mapProj.projectionType("foo"));
-    assertNotNull(projSet.mapProjection("another"));
-    assertEquals(ProjectionType.UNPROJECTED, projSet.mapProjection("another").projectionType("anyCol"));
+    assertSame(TupleProjectionType.ALL, mapProj.type());
+    assertTrue(mapProj.isProjected("foo"));
+
+    RequestedTuple anotherProj = projSet.mapProjection("another");
+    assertNotNull(anotherProj);
+    assertSame(TupleProjectionType.NONE, anotherProj.type());
+    assertFalse(anotherProj.isProjected("anyCol"));
   }
 
+  /**
+   * Selected map projection, multiple levels, full projection
+   * at leaf level.
+   */
   @Test
   public void testProjectionMapSubset() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("map.a", "map.b", "map.map2.x"));
+    assertSame(TupleProjectionType.SOME, projSet.type());
 
-    // Selected map projection, multiple levels, full projection
-    // at leaf level.
-
-    List<SchemaPath> projCols = new ArrayList<>();
-    projCols.add(SchemaPath.getCompoundPath("map", "a"));
-    projCols.add(SchemaPath.getCompoundPath("map", "b"));
-    projCols.add(SchemaPath.getCompoundPath("map", "map2", "x"));
-    RequestedTuple projSet = RequestedTupleImpl.parse(projCols);
-    assertTrue(projSet instanceof RequestedTupleImpl);
-    assertEquals(ProjectionType.TUPLE, projSet.projectionType("map"));
+    // Map itself is projected and has a map qualifier
+    assertTrue(projSet.isProjected("map"));
 
     // Map: an explicit map at top level
 
     RequestedTuple mapProj = projSet.mapProjection("map");
-    assertTrue(mapProj instanceof RequestedTupleImpl);
-    assertEquals(ProjectionType.GENERAL, mapProj.projectionType("a"));
-    assertEquals(ProjectionType.GENERAL, mapProj.projectionType("b"));
-    assertEquals(ProjectionType.TUPLE, mapProj.projectionType("map2"));
-    assertEquals(ProjectionType.UNPROJECTED, mapProj.projectionType("bogus"));
+    assertSame(TupleProjectionType.SOME, projSet.type());
+    assertTrue(mapProj.isProjected("a"));
+    assertTrue(mapProj.isProjected("b"));
+    assertTrue(mapProj.isProjected("map2"));
+    assertFalse(mapProj.isProjected("bogus"));
 
     // Map b: an implied nested map
 
+    assertTrue(mapProj.get("b").isSimple());
     RequestedTuple bMapProj = mapProj.mapProjection("b");
     assertNotNull(bMapProj);
-    assertTrue(bMapProj instanceof ImpliedTupleRequest);
-    assertEquals(ProjectionType.GENERAL, bMapProj.projectionType("foo"));
+    assertSame(TupleProjectionType.ALL, bMapProj.type());
+    assertTrue(bMapProj.isProjected("foo"));
 
     // Map2, an nested map, has an explicit projection
 
     RequestedTuple map2Proj = mapProj.mapProjection("map2");
     assertNotNull(map2Proj);
-    assertTrue(map2Proj instanceof RequestedTupleImpl);
-    assertEquals(ProjectionType.GENERAL, map2Proj.projectionType("x"));
-    assertEquals(ProjectionType.UNPROJECTED, map2Proj.projectionType("bogus"));
+    assertSame(TupleProjectionType.SOME, map2Proj.type());
+    assertTrue(map2Proj.isProjected("x"));
+    assertFalse(map2Proj.isProjected("bogus"));
   }
 
+  /**
+   * Project both a map member and the entire map.
+   */
   @Test
-  public void testProjectionMapFieldAndMap() {
-
-    // Project both a map member and the entire map.
+  public void testProjectionMapAndSimple() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("map.a", "map"));
 
-    {
-      List<SchemaPath> projCols = new ArrayList<>();
-      projCols.add(SchemaPath.getCompoundPath("map", "a"));
-      projCols.add(SchemaPath.getCompoundPath("map"));
+    RequestedTuple mapProj = projSet.mapProjection("map");
+    assertSame(TupleProjectionType.ALL, mapProj.type());
+    assertTrue(mapProj.isProjected("a"));
+    assertTrue(mapProj.isProjected("b"));
+  }
 
-      RequestedTuple projSet = RequestedTupleImpl.parse(projCols);
-      assertTrue(projSet instanceof RequestedTupleImpl);
-      assertEquals(ProjectionType.TUPLE, projSet.projectionType("map"));
+  /**
+   * Project both an entire map and a map member.
+   */
+  @Test
+  public void testProjectionSimpleAndMap() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("map", "map.a"));
 
-      RequestedTuple mapProj = projSet.mapProjection("map");
-      assertTrue(mapProj instanceof ImpliedTupleRequest);
-      assertEquals(ProjectionType.GENERAL, mapProj.projectionType("a"));
+    RequestedTuple mapProj = projSet.mapProjection("map");
+    assertSame(TupleProjectionType.ALL, mapProj.type());
+    assertTrue(mapProj.isProjected("a"));
+    assertTrue(mapProj.isProjected("b"));
+  }
 
-      // Didn't ask for b, but did ask for whole map.
+  /**
+   * Project both a map member and the entire map.
+   */
+  @Test
+  public void testProjectionMapAndWildcard() {
 
-      assertEquals(ProjectionType.GENERAL, mapProj.projectionType("b"));
-    }
+    // Built up by hand because "map.*" is not valid Drill
+    // expression syntax.
+    List<SchemaPath> projCols = new ArrayList<>();
+    projCols.add(SchemaPath.getCompoundPath("map", "a"));
+    projCols.add(SchemaPath.getCompoundPath("map", SchemaPath.DYNAMIC_STAR));
 
-    // Now the other way around.
+    RequestedTuple projSet = Projections.parse(projCols);
+    RequestedTuple mapProj = projSet.mapProjection("map");
+    assertSame(TupleProjectionType.ALL, mapProj.type());
+    assertTrue(mapProj.isProjected("a"));
+    assertTrue(mapProj.isProjected("b"));
+  }
 
-    {
-      List<SchemaPath> projCols = new ArrayList<>();
-      projCols.add(SchemaPath.getCompoundPath("map"));
-      projCols.add(SchemaPath.getCompoundPath("map", "a"));
+  /**
+   * Project both an entire map and a map member.
+   */
+    @Test
+    public void testProjectionWildcardAndMap() {
 
-      RequestedTuple projSet = RequestedTupleImpl.parse(projCols);
-      assertTrue(projSet instanceof RequestedTupleImpl);
-      assertEquals(ProjectionType.TUPLE, projSet.projectionType("map"));
+    List<SchemaPath> projCols = new ArrayList<>();
+    projCols.add(SchemaPath.getCompoundPath("map", SchemaPath.DYNAMIC_STAR));
+    projCols.add(SchemaPath.getCompoundPath("map", "a"));
 
-      RequestedTuple mapProj = projSet.mapProjection("map");
-      assertTrue(mapProj instanceof ImpliedTupleRequest);
-      assertEquals(ProjectionType.GENERAL, mapProj.projectionType("a"));
-      assertEquals(ProjectionType.GENERAL, mapProj.projectionType("b"));
-    }
+    RequestedTuple projSet = Projections.parse(projCols);
+    RequestedTuple mapProj = projSet.mapProjection("map");
+    assertSame(TupleProjectionType.ALL, mapProj.type());
+    assertTrue(mapProj.isProjected("a"));
+    assertTrue(mapProj.isProjected("b"));
   }
 
   @Test
   public void testMapDetails() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a.b.c", "a.c", "d"));
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(2, cols.size());
@@ -238,96 +260,62 @@ public class TestProjectedTuple extends BaseTest {
     assertFalse(a.isArray());
     assertTrue(a.isTuple());
 
-    {
-      assertNotNull(a.mapProjection());
-      List<RequestedColumn> aMembers = a.mapProjection().projections();
-      assertEquals(2, aMembers.size());
+    // a{}
+    assertNotNull(a.tuple());
+    List<RequestedColumn> aMembers = a.tuple().projections();
+    assertEquals(2, aMembers.size());
+
+    // a.b
+    RequestedColumn a_b = aMembers.get(0);
+    assertEquals("b", a_b.name());
+    assertTrue(a_b.isTuple());
 
-      RequestedColumn a_b = aMembers.get(0);
-      assertEquals("b", a_b.name());
-      assertTrue(a_b.isTuple());
+    // a.b{}
+    assertNotNull(a_b.tuple());
+    List<RequestedColumn> a_bMembers = a_b.tuple().projections();
+    assertEquals(1, a_bMembers.size());
 
-      {
-        assertNotNull(a_b.mapProjection());
-        List<RequestedColumn> a_bMembers = a_b.mapProjection().projections();
-        assertEquals(1, a_bMembers.size());
-        assertEquals("c", a_bMembers.get(0).name());
-        assertTrue(a_bMembers.get(0).isSimple());
-      }
+    // a.b.c
+    assertEquals("c", a_bMembers.get(0).name());
+    assertTrue(a_bMembers.get(0).isSimple());
 
-      assertEquals("c", aMembers.get(1).name());
-      assertTrue(aMembers.get(1).isSimple());
-    }
+    // a.c
+    assertEquals("c", aMembers.get(1).name());
+    assertTrue(aMembers.get(1).isSimple());
 
+    // d
     assertEquals("d", cols.get(1).name());
     assertTrue(cols.get(1).isSimple());
   }
 
-  @Test
-  public void testMapDups() {
-    try {
-      RequestedTupleImpl.parse(
-          RowSetTestUtils.projectList("a.b", "a.c", "a.b"));
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
-  }
-
   /**
-   * When the project list includes references to both the
-   * map as a whole, and members, then the parser is forgiving
-   * of duplicate map members since all members are projected.
+   * Duplicate column names are merged for projection.
    */
-
-  @Test
-  public void testMapDupsIgnored() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
-          RowSetTestUtils.projectList("a", "a.b", "a.c", "a.b"));
-    List<RequestedColumn> cols = projSet.projections();
-    assertEquals(1, cols.size());
-  }
-
   @Test
-  public void testWildcard() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
-        RowSetTestUtils.projectList(SchemaPath.DYNAMIC_STAR));
-    List<RequestedColumn> cols = projSet.projections();
-    assertEquals(1, cols.size());
-
-    RequestedColumn wildcard = cols.get(0);
-    assertEquals(ProjectionType.WILDCARD, wildcard.type());
-    assertEquals(SchemaPath.DYNAMIC_STAR, wildcard.name());
-    assertFalse(wildcard.isSimple());
-    assertTrue(wildcard.isWildcard());
-    assertNull(wildcard.mapProjection());
-    assertNull(wildcard.indexes());
-  }
+  public void testMapDups() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a.b", "a.c", "a.b"));
 
-  @Test
-  public void testSimpleDups() {
-    try {
-      RequestedTupleImpl.parse(RowSetTestUtils.projectList("a", "b", "a"));
-      fail();
-    } catch (UserException e) {
-      // Expected
-    }
+    RequestedTuple aMap = projSet.mapProjection("a");
+    assertEquals(2, aMap.projections().size());
+    assertEquals(2, ((RequestedColumnImpl) aMap.get("b")).refCount());
   }
 
   @Test
   public void testArray() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a[1]", "a[3]"));
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(1, cols.size());
 
-    assertEquals(ProjectionType.ARRAY, projSet.projectionType("a"));
     RequestedColumn a = cols.get(0);
     assertEquals("a", a.name());
     assertTrue(a.isArray());
+    assertEquals(1, a.arrayDims());
     assertFalse(a.isSimple());
     assertFalse(a.isTuple());
-    boolean[] indexes = a.indexes();
+    assertTrue(a.hasIndexes());
+    boolean indexes[] = a.indexes();
     assertNotNull(indexes);
     assertEquals(4, indexes.length);
     assertFalse(indexes[0]);
@@ -336,15 +324,37 @@ public class TestProjectedTuple extends BaseTest {
     assertTrue(indexes[3]);
   }
 
+  @Test
+  public void testMultiDimArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[0][1][2]", "a[2][3]"));
+    List<RequestedColumn> cols = projSet.projections();
+    assertEquals(1, cols.size());
+
+    RequestedColumn a = cols.get(0);
+    assertEquals("a", a.name());
+    assertTrue(a.isArray());
+    // Dimension count is the maximum seen.
+    assertEquals(3, a.arrayDims());
+    assertFalse(a.isSimple());
+    assertFalse(a.isTuple());
+    boolean[] indexes = a.indexes();
+    assertNotNull(indexes);
+    assertEquals(3, indexes.length);
+    assertTrue(indexes[0]);
+    assertFalse(indexes[1]);
+    assertTrue(indexes[2]);
+  }
+
   /**
    * Duplicate array entries are allowed to handle the
    * use case of a[1], a[1].z. Each element is reported once;
    * the project operator will create copies as needed.
    */
   @Test
-  public void testArrayDups() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
-      RowSetTestUtils.projectList("a[1]", "a[3]", "a[1]", "a[3].z"));
+  public void testArrayDupsIgnored() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[1]", "a[3]", "a[1]", "a[3].z"));
 
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(1, cols.size());
@@ -352,7 +362,7 @@ public class TestProjectedTuple extends BaseTest {
     RequestedColumn a = cols.get(0);
     assertEquals("a", a.name());
     assertTrue(a.isArray());
-    boolean[] indexes = a.indexes();
+    boolean indexes[] = a.indexes();
     assertNotNull(indexes);
     assertEquals(4, indexes.length);
     assertFalse(indexes[0]);
@@ -363,7 +373,7 @@ public class TestProjectedTuple extends BaseTest {
 
   @Test
   public void testArrayAndSimple() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a[1]", "a"));
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(1, cols.size());
@@ -376,7 +386,7 @@ public class TestProjectedTuple extends BaseTest {
 
   @Test
   public void testSimpleAndArray() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a", "a[1]"));
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(1, cols.size());
@@ -384,20 +394,87 @@ public class TestProjectedTuple extends BaseTest {
     RequestedColumn a = cols.get(0);
     assertEquals("a", a.name());
     assertTrue(a.isArray());
+    assertFalse(a.hasIndexes());
     assertNull(a.indexes());
-    assertEquals(ProjectionType.ARRAY, projSet.projectionType("a"));
-    assertEquals(ProjectionType.UNPROJECTED, projSet.projectionType("foo"));
   }
 
   @Test
   // Drill syntax does not support map arrays
   public void testMapArray() {
-    RequestedTuple projSet = RequestedTupleImpl.parse(
+    RequestedTuple projSet = Projections.parse(
         RowSetTestUtils.projectList("a[1].x"));
     List<RequestedColumn> cols = projSet.projections();
     assertEquals(1, cols.size());
 
-    assertEquals(ProjectionType.TUPLE_ARRAY, cols.get(0).type());
-    assertEquals(ProjectionType.TUPLE_ARRAY, projSet.projectionType("a"));
+    RequestedColumn a = cols.get(0);
+
+    // Column acts like an array
+    assertTrue(a.isArray());
+    assertTrue(a.hasIndexes());
+    assertEquals(1, a.arrayDims());
+
+    // And the column acts like a map
+    assertTrue(a.isTuple());
+    RequestedTuple aProj = a.tuple();
+    assertSame(TupleProjectionType.SOME, aProj.type());
+    assertTrue(aProj.isProjected("x"));
+    assertFalse(aProj.isProjected("y"));
+  }
+
+  @Test
+  // Drill syntax does not support map arrays
+  public void testMap2DArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("a[1][2].x"));
+    List<RequestedColumn> cols = projSet.projections();
+    assertEquals(1, cols.size());
+
+    RequestedColumn a = cols.get(0);
+
+    // Column acts like an array
+    assertTrue(a.isArray());
+    assertTrue(a.hasIndexes());
+
+    // Note that the multiple dimensions are inferred only through
+    // the multiple levels of qualifiers.
+
+    // And the column acts like a map
+    assertTrue(a.isTuple());
+    RequestedTuple aProj = a.tuple();
+    assertSame(TupleProjectionType.SOME, aProj.type());
+    assertTrue(aProj.isProjected("x"));
+    assertFalse(aProj.isProjected("y"));
+  }
+
+  /**
+   * Projection does not enforce semantics; it just report what it
+   * sees. This allows cases such as m.a and m[0], which might mean
+   * that m is a map array, m.a wants an array of a-member values, and m[0]
+   * wants the first map in the array. Not clear Drill actually supports
+   * these cases, however.
+   */
+  @Test
+  public void testArrayAndMap() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("m.a", "m[0]"));
+    RequestedColumn m = projSet.get("m");
+    assertTrue(m.isArray());
+    assertEquals(1, m.arrayDims());
+    assertTrue(m.isTuple());
+    assertTrue(m.tuple().isProjected("a"));
+    assertFalse(m.tuple().isProjected("b"));
+  }
+
+  @Test
+  public void testMapAndArray() {
+    RequestedTuple projSet = Projections.parse(
+        RowSetTestUtils.projectList("m[0]", "m.a"));
+    RequestedColumn m = projSet.get("m");
+    assertTrue(m.isArray());
+    assertEquals(1, m.arrayDims());
+    assertTrue(m.isTuple());
+    assertTrue(m.tuple().isProjected("a"));
+    // m[0] requests the entire tuple
+    assertTrue(m.tuple().isProjected("b"));
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectionType.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectionType.java
deleted file mode 100644
index 341e10d..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/resultSet/project/TestProjectionType.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.resultSet.project;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.drill.categories.RowSetTests;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.test.BaseTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(RowSetTests.class)
-public class TestProjectionType extends BaseTest {
-
-  @Test
-  public void testQueries() {
-    assertFalse(ProjectionType.UNPROJECTED.isTuple());
-    assertFalse(ProjectionType.WILDCARD.isTuple());
-    assertFalse(ProjectionType.GENERAL.isTuple());
-    assertFalse(ProjectionType.SCALAR.isTuple());
-    assertTrue(ProjectionType.TUPLE.isTuple());
-    assertFalse(ProjectionType.ARRAY.isTuple());
-    assertTrue(ProjectionType.TUPLE_ARRAY.isTuple());
-
-    assertFalse(ProjectionType.UNPROJECTED.isArray());
-    assertFalse(ProjectionType.WILDCARD.isArray());
-    assertFalse(ProjectionType.GENERAL.isArray());
-    assertFalse(ProjectionType.SCALAR.isArray());
-    assertFalse(ProjectionType.TUPLE.isArray());
-    assertTrue(ProjectionType.ARRAY.isArray());
-    assertTrue(ProjectionType.TUPLE_ARRAY.isArray());
-
-    assertFalse(ProjectionType.UNPROJECTED.isMaybeScalar());
-    assertFalse(ProjectionType.WILDCARD.isMaybeScalar());
-    assertTrue(ProjectionType.GENERAL.isMaybeScalar());
-    assertTrue(ProjectionType.SCALAR.isMaybeScalar());
-    assertFalse(ProjectionType.TUPLE.isMaybeScalar());
-    assertFalse(ProjectionType.ARRAY.isMaybeScalar());
-    assertFalse(ProjectionType.TUPLE_ARRAY.isMaybeScalar());
-  }
-
-  @Test
-  public void testLabel() {
-
-    // Only worry about the types that could conflict and thus
-    // would show up in error messages.
-
-    assertEquals(ProjectionType.UNPROJECTED.name(), ProjectionType.UNPROJECTED.label());
-    assertEquals("wildcard (*)", ProjectionType.WILDCARD.label());
-    assertEquals(ProjectionType.GENERAL.name(), ProjectionType.GENERAL.label());
-    assertEquals("scalar (a)", ProjectionType.SCALAR.label());
-    assertEquals("tuple (a.x)", ProjectionType.TUPLE.label());
-    assertEquals("array (a[n])", ProjectionType.ARRAY.label());
-    assertEquals("tuple array (a[n].x)", ProjectionType.TUPLE_ARRAY.label());
-  }
-
-  @Test
-  public void testTypeFor() {
-
-    // Test the return of the projection type most specific
-    // for a data type. The projection type under-specifies
-    // the data type, but is a hint.
-
-    assertEquals(ProjectionType.TUPLE, ProjectionType.typeFor(Types.required(MinorType.MAP)));
-    assertEquals(ProjectionType.TUPLE_ARRAY, ProjectionType.typeFor(Types.repeated(MinorType.MAP)));
-    assertEquals(ProjectionType.ARRAY, ProjectionType.typeFor(Types.repeated(MinorType.INT)));
-    assertEquals(ProjectionType.ARRAY, ProjectionType.typeFor(Types.required(MinorType.LIST)));
-    assertEquals(ProjectionType.SCALAR, ProjectionType.typeFor(Types.required(MinorType.INT)));
-  }
-
-  @Test
-  public void testCompatibility() {
-
-    // Only SCALAR, TUPLE, ARRAY and TUPLE_ARRAY are expected for the
-    // argument, but we check all cases for completeness.
-    // Note that the cases are not always symmetrical:
-    // a map array column is compatible with a map projection,
-    // but a map column is not compatible with a map array projection.
-
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.GENERAL));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.SCALAR));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.TUPLE));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.UNPROJECTED.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.GENERAL));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.SCALAR));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.TUPLE));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.WILDCARD.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.GENERAL));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.SCALAR));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.TUPLE));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.GENERAL.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.SCALAR.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.SCALAR.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.SCALAR.isCompatible(ProjectionType.GENERAL));
-    assertTrue(ProjectionType.SCALAR.isCompatible(ProjectionType.SCALAR));
-    assertFalse(ProjectionType.SCALAR.isCompatible(ProjectionType.TUPLE));
-    assertFalse(ProjectionType.SCALAR.isCompatible(ProjectionType.ARRAY));
-    assertFalse(ProjectionType.SCALAR.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.TUPLE.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.TUPLE.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.TUPLE.isCompatible(ProjectionType.GENERAL));
-    assertFalse(ProjectionType.TUPLE.isCompatible(ProjectionType.SCALAR));
-    assertTrue(ProjectionType.TUPLE.isCompatible(ProjectionType.TUPLE));
-    assertFalse(ProjectionType.TUPLE.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.TUPLE.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.ARRAY.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.ARRAY.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.ARRAY.isCompatible(ProjectionType.GENERAL));
-    assertFalse(ProjectionType.ARRAY.isCompatible(ProjectionType.SCALAR));
-    assertFalse(ProjectionType.ARRAY.isCompatible(ProjectionType.TUPLE));
-    assertTrue(ProjectionType.ARRAY.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.ARRAY.isCompatible(ProjectionType.TUPLE_ARRAY));
-
-    assertTrue(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.UNPROJECTED));
-    assertTrue(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.WILDCARD));
-    assertTrue(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.GENERAL));
-    assertFalse(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.SCALAR));
-    assertFalse(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.TUPLE));
-    assertFalse(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.ARRAY));
-    assertTrue(ProjectionType.TUPLE_ARRAY.isCompatible(ProjectionType.TUPLE_ARRAY));
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithHeaders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithHeaders.java
index 171669a..0bbdf78 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithHeaders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithHeaders.java
@@ -60,7 +60,6 @@ import static org.junit.Assert.fail;
  *
  * @see TestHeaderBuilder
  */
-
 @Category(RowSetTests.class)
 public class TestCsvWithHeaders extends BaseCsvTest {
 
@@ -136,7 +135,6 @@ public class TestCsvWithHeaders extends BaseCsvTest {
   /**
    * Trivial case: empty header. This case should fail.
    */
-
   @Test
   public void testEmptyCsvHeaders() throws IOException {
     buildFile(EMPTY_HEADERS_FILE, emptyHeaders);
@@ -174,7 +172,6 @@ public class TestCsvWithHeaders extends BaseCsvTest {
     buildFile(EMPTY_BODY_FILE, emptyBody);
 
     // SELECT * query: expect schema-only result.
-
     RowSet rowSet = client.queryBuilder().sql(makeStatement(EMPTY_BODY_FILE)).rowSet();
     TupleMetadata expectedSchema = new SchemaBuilder()
         .add("a", MinorType.VARCHAR)
@@ -186,7 +183,6 @@ public class TestCsvWithHeaders extends BaseCsvTest {
     RowSetUtilities.verify(expected, rowSet);
 
     // Try again with COUNT(*)
-
     long count = client.queryBuilder().sql(COUNT_STAR, EMPTY_BODY_FILE).singletonLong();
     assertEquals(0, count);
   }
@@ -289,7 +285,6 @@ public class TestCsvWithHeaders extends BaseCsvTest {
    * of just one implicit column. V3 uses non-nullable VARCHAR for file
    * metadata columns.
    */
-
   @Test
   public void testImplicitColsExplicitSelect() throws IOException {
     String sql = "SELECT A, filename FROM `dfs.data`.`%s`";
@@ -311,7 +306,6 @@ public class TestCsvWithHeaders extends BaseCsvTest {
    * of just one implicit column. V3 uses non-nullable VARCHAR for file
    * metadata columns.
    */
-
   @Test
   public void testImplicitColWildcard() throws IOException {
     String sql = "SELECT *, filename FROM `dfs.data`.`%s`";
@@ -435,8 +429,8 @@ public class TestCsvWithHeaders extends BaseCsvTest {
 
     RowSet rowSet;
     if (SCHEMA_BATCH_ENABLED) {
-      // First batch is empty; just carries the schema.
 
+      // First batch is empty; just carries the schema.
       assertTrue(iter.hasNext());
       rowSet = iter.next();
       assertEquals(0, rowSet.rowCount());
@@ -444,13 +438,11 @@ public class TestCsvWithHeaders extends BaseCsvTest {
     }
 
     // Read the other two batches.
-
     for (int i = 0; i < 2; i++) {
       assertTrue(iter.hasNext());
       rowSet = iter.next();
 
       // Figure out which record this is and test accordingly.
-
       RowSetReader reader = rowSet.reader();
       assertTrue(reader.next());
       String col1 = reader.scalar(0).getString();
@@ -493,8 +485,8 @@ public class TestCsvWithHeaders extends BaseCsvTest {
 
     RowSet rowSet;
     if (SCHEMA_BATCH_ENABLED) {
-      // First batch is empty; just carries the schema.
 
+      // First batch is empty; just carries the schema.
       assertTrue(iter.hasNext());
       rowSet = iter.next();
       RowSetUtilities.verify(new RowSetBuilder(client.allocator(), expectedSchema).build(),
@@ -502,13 +494,11 @@ public class TestCsvWithHeaders extends BaseCsvTest {
     }
 
     // Read the two batches.
-
     for (int i = 0; i < 2; i++) {
       assertTrue(iter.hasNext());
       rowSet = iter.next();
 
       // Figure out which record this is and test accordingly.
-
       RowSetReader reader = rowSet.reader();
       assertTrue(reader.next());
       String aCol = reader.scalar("a").getString();
@@ -548,8 +538,8 @@ public class TestCsvWithHeaders extends BaseCsvTest {
 
     RowSet rowSet;
     if (SCHEMA_BATCH_ENABLED) {
-      // First batch is empty; just carries the schema.
 
+      // First batch is empty; just carries the schema.
       assertTrue(iter.hasNext());
       rowSet = iter.next();
       RowSetUtilities.verify(new RowSetBuilder(client.allocator(), expectedSchema).build(),
@@ -557,13 +547,11 @@ public class TestCsvWithHeaders extends BaseCsvTest {
     }
 
     // Read the two batches.
-
     for (int i = 0; i < 2; i++) {
       assertTrue(iter.hasNext());
       rowSet = iter.next();
 
       // Figure out which record this is and test accordingly.
-
       RowSetReader reader = rowSet.reader();
       assertTrue(reader.next());
       String aCol = reader.scalar("a").getString();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithoutHeaders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithoutHeaders.java
index 9416748..363bb70 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithoutHeaders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsvWithoutHeaders.java
@@ -47,7 +47,6 @@ import org.junit.experimental.categories.Category;
  * and without an external schema. Data is represented with the
  * `columns` array column.
  */
-
 @Category(RowSetTests.class)
 public class TestCsvWithoutHeaders extends BaseCsvTest {
 
@@ -79,7 +78,6 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
   protected static void buildNestedTableWithoutHeaders() throws IOException {
 
     // Two-level partitioned table
-
     File rootDir = new File(testDir, PART_DIR);
     rootDir.mkdir();
     buildFile(new File(rootDir, ROOT_FILE), sampleData);
@@ -91,7 +89,6 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
   /**
    * Verify that the wildcard expands to the `columns` array
    */
-
   @Test
   public void testWildcard() throws IOException {
     String sql = "SELECT * FROM `dfs.data`.`%s`";
@@ -222,7 +219,6 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
    * data columns (so that data columns don't shift positions if
    * files are nested to another level.)
    */
-
   @Test
   public void testPartitionExpansion() throws IOException {
     String sql = "SELECT * FROM `dfs.data`.`%s`";
@@ -235,8 +231,8 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
 
     RowSet rowSet;
     if (SCHEMA_BATCH_ENABLED) {
-      // First batch is empty; just carries the schema.
 
+      // First batch is empty; just carries the schema.
       assertTrue(iter.hasNext());
       rowSet = iter.next();
       assertEquals(0, rowSet.rowCount());
@@ -244,13 +240,11 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
     }
 
     // Read the two data batches.
-
     for (int i = 0; i < 2; i++) {
       assertTrue(iter.hasNext());
       rowSet = iter.next();
 
       // Figure out which record this is and test accordingly.
-
       RowSetReader reader = rowSet.reader();
       assertTrue(reader.next());
       ArrayReader ar = reader.array(0);
@@ -279,7 +273,6 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
    * V2 message: DATA_READ ERROR: Selected column 'columns' must be an array index
    * @throws Exception
    */
-
   @Test
   public void testColumnsAsMap() throws Exception {
     String sql = "SELECT `%s`.columns.foo FROM `dfs.data`.`%s`";
@@ -298,7 +291,6 @@ public class TestCsvWithoutHeaders extends BaseCsvTest {
    * V2 message: INTERNAL_ERROR ERROR: 70000
    * @throws Exception
    */
-
   @Test
   public void testColumnsIndexOverflow() throws Exception {
     String sql = "SELECT columns[70000] FROM `dfs.data`.`%s`";


[drill] 03/10: DRILL-7586: Fix loading incorrect version of commons-lang3

Posted by ag...@apache.org.
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 e8f9b7ee2940984c59783e36e4fbca3fa25d6ad7
Author: ozinoviev <oz...@solit-clouds.ru>
AuthorDate: Fri Feb 14 13:48:50 2020 +0300

    DRILL-7586: Fix loading incorrect version of commons-lang3
    
    closes #1984
---
 contrib/storage-hive/hive-exec-shade/pom.xml | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 82ebfbc..f6ba72e 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -148,6 +148,14 @@
               <pattern>com.google.protobuf.</pattern>
               <shadedPattern>hive.com.google.protobuf.</shadedPattern>
             </relocation>
+            <relocation>
+              <pattern>org.apache.commons.lang.</pattern>
+              <shadedPattern>hive.org.apache.commons.lang.</shadedPattern>
+            </relocation>
+            <relocation>
+              <pattern>org.apache.commons.lang3.</pattern>
+              <shadedPattern>hive.org.apache.commons.lang3.</shadedPattern>
+            </relocation>
           </relocations>
           <filters>
             <filter>


[drill] 01/10: DRILL-7594: Remove unused DrillStoreRel

Posted by ag...@apache.org.
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 75171761f4d5620375ef0328e7c070a3199dd262
Author: Igor Guzenko <ih...@gmail.com>
AuthorDate: Thu Feb 20 11:20:42 2020 +0200

    DRILL-7594: Remove unused DrillStoreRel
    
    closes #1990
---
 .../exec/planner/common/DrillStoreRelBase.java     | 43 ----------------
 .../drill/exec/planner/logical/DrillStoreRel.java  | 46 -----------------
 .../planner/sql/handlers/AnalyzeTableHandler.java  | 57 +++++++++-------------
 .../planner/sql/handlers/DefaultSqlHandler.java    | 25 ++++------
 4 files changed, 33 insertions(+), 138 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
deleted file mode 100644
index 2781564..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.common;
-
-import java.util.List;
-
-import org.apache.calcite.prepare.Prepare.CatalogReader;
-
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rex.RexNode;
-
-/**
- * Base class for logical and physical Store implemented in Drill
- */
-public abstract class DrillStoreRelBase extends TableModify implements DrillRelNode {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRelBase.class);
-
-  protected DrillStoreRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader, RelNode child,
-                              Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
-
-  }
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
deleted file mode 100644
index 9872f95..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.logical;
-
-import java.util.List;
-
-import org.apache.calcite.prepare.Prepare.CatalogReader;
-
-import org.apache.calcite.rex.RexNode;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.common.DrillStoreRelBase;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-
-public class DrillStoreRel extends DrillStoreRelBase implements DrillRel{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRel.class);
-
-  protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader, RelNode child,
-                          Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
-
-  }
-
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    return null;
-  }
-
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
index 232e81a..b7a5add 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
@@ -39,7 +39,6 @@ import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
-import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DrillWriterRel;
 import org.apache.drill.exec.planner.physical.Prel;
@@ -100,40 +99,36 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
           .message("No table with given name [%s] exists in schema [%s]", tableName,
               drillSchema.getFullSchemaName())
           .build(logger);
+    } else if (!(table instanceof DrillTable)) {
+      return DrillStatsTable.notSupported(context, tableName);
     }
 
-    if(! (table instanceof DrillTable)) {
+    DrillTable drillTable = (DrillTable) table;
+    final Object selection = drillTable.getSelection();
+    if (!(selection instanceof FormatSelection)) {
+      return DrillStatsTable.notSupported(context, tableName);
+    }
+    // Do not support non-parquet tables
+    FormatSelection formatSelection = (FormatSelection) selection;
+    FormatPluginConfig formatConfig = formatSelection.getFormat();
+    if (!((formatConfig instanceof ParquetFormatConfig)
+          || ((formatConfig instanceof NamedFormatPluginConfig)
+               && ((NamedFormatPluginConfig) formatConfig).name.equals("parquet")))) {
       return DrillStatsTable.notSupported(context, tableName);
     }
 
-    if (table instanceof DrillTable) {
-      DrillTable drillTable = (DrillTable) table;
-      final Object selection = drillTable.getSelection();
-      if (!(selection instanceof FormatSelection)) {
-        return DrillStatsTable.notSupported(context, tableName);
-      }
-      // Do not support non-parquet tables
-      FormatSelection formatSelection = (FormatSelection) selection;
-      FormatPluginConfig formatConfig = formatSelection.getFormat();
-      if (!((formatConfig instanceof ParquetFormatConfig)
-            || ((formatConfig instanceof NamedFormatPluginConfig)
-                 && ((NamedFormatPluginConfig) formatConfig).name.equals("parquet")))) {
-        return DrillStatsTable.notSupported(context, tableName);
-      }
-
-      FileSystemPlugin plugin = (FileSystemPlugin) drillTable.getPlugin();
-      DrillFileSystem fs = new DrillFileSystem(plugin.getFormatPlugin(
-          formatSelection.getFormat()).getFsConf());
+    FileSystemPlugin plugin = (FileSystemPlugin) drillTable.getPlugin();
+    DrillFileSystem fs = new DrillFileSystem(plugin.getFormatPlugin(
+        formatSelection.getFormat()).getFsConf());
 
-      Path selectionRoot = formatSelection.getSelection().getSelectionRoot();
-      if (!selectionRoot.toUri().getPath().endsWith(tableName) || !fs.getFileStatus(selectionRoot).isDirectory()) {
-        return DrillStatsTable.notSupported(context, tableName);
-      }
-      // Do not recompute statistics, if stale
-      Path statsFilePath = new Path(selectionRoot, DotDrillType.STATS.getEnding());
-      if (fs.exists(statsFilePath) && !isStatsStale(fs, statsFilePath)) {
-       return DrillStatsTable.notRequired(context, tableName);
-      }
+    Path selectionRoot = formatSelection.getSelection().getSelectionRoot();
+    if (!selectionRoot.toUri().getPath().endsWith(tableName) || !fs.getFileStatus(selectionRoot).isDirectory()) {
+      return DrillStatsTable.notSupported(context, tableName);
+    }
+    // Do not recompute statistics, if stale
+    Path statsFilePath = new Path(selectionRoot, DotDrillType.STATS.getEnding());
+    if (fs.exists(statsFilePath) && !isStatsStale(fs, statsFilePath)) {
+     return DrillStatsTable.notRequired(context, tableName);
     }
     // Convert the query to Drill Logical plan and insert a writer operator on top.
     DrillRel drel = convertToDrel(relScan, drillSchema, tableName, sqlAnalyzeTable.getSamplePercent());
@@ -201,10 +196,6 @@ public class AnalyzeTableHandler extends DefaultSqlHandler {
       double samplePercent) throws SqlUnsupportedException {
     DrillRel convertedRelNode = convertToRawDrel(relNode);
 
-    if (convertedRelNode instanceof DrillStoreRel) {
-      throw new UnsupportedOperationException();
-    }
-
     final RelNode analyzeRel = new DrillAnalyzeRel(
         convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode, samplePercent);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 4cc7a4f..a59d8a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -77,7 +77,6 @@ import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
-import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.PreProcessLogicalRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
@@ -275,23 +274,17 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       // Convert SUM to $SUM0
       final RelNode convertedRelNodeWithSum0 = transform(PlannerType.HEP_BOTTOM_UP, PlannerPhase.SUM_CONVERSION, convertedRelNode);
 
-      final DrillRel drillRel = (DrillRel) convertedRelNodeWithSum0;
-
-      if (drillRel instanceof DrillStoreRel) {
-        throw new UnsupportedOperationException();
-      } else {
-
-        // If the query contains a limit 0 clause, disable distributed mode since it is overkill for determining schema.
-        if (FindLimit0Visitor.containsLimit0(convertedRelNodeWithSum0) &&
-            FindHardDistributionScans.canForceSingleMode(convertedRelNodeWithSum0)) {
-          context.getPlannerSettings().forceSingleMode();
-          if (context.getOptions().getOption(ExecConstants.LATE_LIMIT0_OPT)) {
-            return FindLimit0Visitor.addLimitOnTopOfLeafNodes(drillRel);
-          }
+      DrillRel drillRel = (DrillRel) convertedRelNodeWithSum0;
+      // If the query contains a limit 0 clause, disable distributed mode since it is overkill for determining schema.
+      if (FindLimit0Visitor.containsLimit0(convertedRelNodeWithSum0) &&
+          FindHardDistributionScans.canForceSingleMode(convertedRelNodeWithSum0)) {
+        context.getPlannerSettings().forceSingleMode();
+        if (context.getOptions().getOption(ExecConstants.LATE_LIMIT0_OPT)) {
+          drillRel = FindLimit0Visitor.addLimitOnTopOfLeafNodes(drillRel);
         }
-
-        return drillRel;
       }
+
+      return drillRel;
     } catch (RelOptPlanner.CannotPlanException ex) {
       logger.error(ex.getMessage());
 


[drill] 04/10: DRILL-7583: Remove STOP status from operator outcome

Posted by ag...@apache.org.
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 5015d0e9b70092af0b9cb475ec3e748583c4c897
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Thu Feb 13 10:17:00 2020 -0800

    DRILL-7583: Remove STOP status from operator outcome
    
    Now that all operators have been converted to throw
    exceptions on error condistions, the STOP status is
    unused. This patch removes the STOP status and the
    related kill() and killIncoming() methods. The
    "kill" methods are replaced by "cancel" methods which
    handle "normal" case cancellation, such as for
    LIMIT.
    
    closes #1981
---
 .../physical/PhysicalOperatorSetupException.java   |    8 +-
 .../apache/drill/exec/physical/impl/ScanBatch.java |   17 +-
 .../drill/exec/physical/impl/ScreenCreator.java    |    2 -
 .../exec/physical/impl/SingleSenderCreator.java    |    9 +-
 .../physical/impl/StatisticsWriterRecordBatch.java |    7 +-
 .../drill/exec/physical/impl/TopN/TopNBatch.java   |    9 +-
 .../exec/physical/impl/WriterRecordBatch.java      |    7 +-
 .../exec/physical/impl/aggregate/HashAggBatch.java |  131 ++-
 .../physical/impl/aggregate/HashAggTemplate.java   |  232 +++--
 .../impl/aggregate/SpilledRecordBatch.java         |   10 +-
 .../physical/impl/aggregate/StreamingAggBatch.java |    9 +-
 .../impl/aggregate/StreamingAggTemplate.java       |    2 -
 .../broadcastsender/BroadcastSenderRootExec.java   |    1 -
 .../physical/impl/filter/EvalSetupException.java   |    2 +-
 .../physical/impl/filter/EvaluationPredicate.java  |   10 +-
 .../physical/impl/flatten/FlattenRecordBatch.java  |    4 +-
 .../physical/impl/flatten/FlattenTemplate.java     |    8 +-
 .../exec/physical/impl/join/HashJoinBatch.java     | 1014 ++++++++++++--------
 .../impl/join/HashJoinMemoryCalculatorImpl.java    |   20 +-
 .../physical/impl/join/HashJoinProbeTemplate.java  |    3 +-
 .../impl/join/HashJoinStateCalculator.java         |    2 +-
 .../drill/exec/physical/impl/join/JoinStatus.java  |    5 -
 .../exec/physical/impl/join/LateralJoinBatch.java  |   20 +-
 .../exec/physical/impl/join/MergeJoinBatch.java    |   12 +-
 .../physical/impl/join/NestedLoopJoinBatch.java    |   10 +-
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |   13 +-
 .../exec/physical/impl/limit/LimitRecordBatch.java |   42 +-
 .../impl/mergereceiver/MergingRecordBatch.java     |   12 +-
 .../impl/metadata/MetadataControllerBatch.java     |    8 -
 .../impl/metadata/MetadataHandlerBatch.java        |    4 +-
 .../OrderedPartitionProjectorTemplate.java         |   10 +-
 .../OrderedPartitionRecordBatch.java               |    7 +-
 .../orderedpartitioner/SampleCopierTemplate.java   |   11 +-
 .../partitionsender/PartitionSenderRootExec.java   |   30 +-
 .../impl/producer/ProducerConsumerBatch.java       |   30 +-
 .../physical/impl/project/ProjectRecordBatch.java  |    4 +-
 .../impl/protocol/OperatorRecordBatch.java         |    7 +-
 .../RangePartitionRecordBatch.java                 |    5 -
 .../exec/physical/impl/sort/RecordBatchData.java   |    4 +-
 .../drill/exec/physical/impl/sort/SortBatch.java   |    6 +-
 .../impl/statistics/StatisticsMergeBatch.java      |    1 -
 .../physical/impl/union/UnionAllRecordBatch.java   |    9 -
 .../physical/impl/unnest/UnnestRecordBatch.java    |    8 +-
 .../unorderedreceiver/UnorderedReceiverBatch.java  |   11 +-
 .../impl/unpivot/UnpivotMapsRecordBatch.java       |    1 -
 .../validate/IteratorValidatorBatchIterator.java   |   20 +-
 .../impl/window/WindowFrameRecordBatch.java        |   27 +-
 .../physical/impl/xsort/ExternalSortBatch.java     |   13 +-
 .../exec/physical/impl/xsort/MergeSortWrapper.java |    2 +-
 .../exec/physical/resultSet/impl/TupleState.java   |    2 +
 .../exec/planner/physical/PlannerSettings.java     |   24 +-
 .../exec/record/AbstractBinaryRecordBatch.java     |   11 +-
 .../drill/exec/record/AbstractRecordBatch.java     |   17 +-
 .../exec/record/AbstractUnaryRecordBatch.java      |    6 +-
 .../apache/drill/exec/record/MajorTypeSerDe.java   |    2 +
 .../org/apache/drill/exec/record/RecordBatch.java  |   52 +-
 .../exec/record/RecordBatchMemoryManager.java      |   10 +-
 .../apache/drill/exec/record/RecordBatchSizer.java |   13 +-
 .../apache/drill/exec/record/RecordIterator.java   |    1 -
 .../apache/drill/exec/record/SchemalessBatch.java  |   17 +-
 .../drill/exec/record/SimpleRecordBatch.java       |    8 +-
 .../exec/record/VectorAccessibleComplexWriter.java |    1 -
 .../java/org/apache/drill/TestCorrelation.java     |    1 -
 .../java/org/apache/drill/TestExampleQueries.java  |  128 ++-
 .../test/java/org/apache/drill/TestTpchLimit0.java |    1 -
 .../drill/exec/DrillSeparatePlanningTest.java      |    2 +-
 .../drill/exec/physical/impl/MockRecordBatch.java  |   15 +-
 .../drill/exec/physical/impl/SimpleRootExec.java   |    1 -
 .../exec/physical/impl/TestBroadcastExchange.java  |    1 +
 .../physical/impl/join/TestHashJoinOutcome.java    |   14 +-
 .../impl/join/TestLateralJoinCorrectness.java      |    2 +-
 .../impl/limit/TestLimitBatchEmitOutcome.java      |   54 +-
 .../impl/protocol/TestOperatorRecordBatch.java     |    9 +-
 .../physical/impl/unnest/MockLateralJoinBatch.java |   30 +-
 .../impl/unnest/TestUnnestCorrectness.java         |    5 +-
 .../unnest/TestUnnestWithLateralCorrectness.java   |    5 +-
 .../drill/exec/work/filter/BloomFilterTest.java    |   10 +-
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |    3 +-
 78 files changed, 1109 insertions(+), 1175 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalOperatorSetupException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalOperatorSetupException.java
index 66acec0..0c2e06c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalOperatorSetupException.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalOperatorSetupException.java
@@ -20,12 +20,10 @@ package org.apache.drill.exec.physical;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 
 
+@SuppressWarnings("serial")
 public class PhysicalOperatorSetupException extends ExecutionSetupException {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorSetupException.class);
 
-  public PhysicalOperatorSetupException() {
-    super();
-  }
+  public PhysicalOperatorSetupException() { }
 
   public PhysicalOperatorSetupException(String message, Throwable cause, boolean enableSuppression,
       boolean writableStackTrace) {
@@ -43,6 +41,4 @@ public class PhysicalOperatorSetupException extends ExecutionSetupException {
   public PhysicalOperatorSetupException(Throwable cause) {
     super(cause);
   }
-
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index d314794..e159ec6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -174,12 +174,9 @@ public class ScanBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    if (sendUpstream) {
-      done = true;
-    } else {
-      releaseAssets();
-    }
+  public void cancel() {
+    done = true;
+    releaseAssets();
   }
 
   /**
@@ -285,13 +282,10 @@ public class ScanBatch implements CloseableRecordBatch {
       return internalNext();
     } catch (OutOfMemoryException ex) {
       clearFieldVectorMap();
-      lastOutcome = IterOutcome.STOP;
       throw UserException.memoryError(ex).build(logger);
     } catch (UserException ex) {
-      lastOutcome = IterOutcome.STOP;
       throw ex;
     } catch (Exception ex) {
-      lastOutcome = IterOutcome.STOP;
       throw UserException.internalError(ex).build(logger);
     } finally {
       oContext.getStats().stopProcessing();
@@ -630,11 +624,6 @@ public class ScanBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public boolean hasFailed() {
-    return lastOutcome == IterOutcome.STOP;
-  }
-
-  @Override
   public void dump() {
     logger.error("ScanBatch[container={}, currentReader={}, schema={}]", container, currentReader, schema);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 765e1de..0f7ca13 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -83,8 +83,6 @@ public class ScreenCreator implements RootCreator<Screen> {
       IterOutcome outcome = next(incoming);
       logger.trace("Screen Outcome {}", outcome);
       switch (outcome) {
-        case STOP:
-          return false;
         case NONE:
           if (firstBatch) {
             // this is the only data message sent to the client and may contain the schema
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index d14d09a..c23d968 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -53,7 +53,6 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
     private AccountingDataTunnel tunnel;
     private final FragmentHandle handle;
     private final int recMajor;
-    private volatile boolean ok = true;
     private volatile boolean done = false;
 
     public enum Metric implements MetricDef {
@@ -82,21 +81,15 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
 
     @Override
     public boolean innerNext() {
-      if (!ok) {
-        incoming.kill(false);
-
-        return false;
-      }
 
       IterOutcome out;
       if (!done) {
         out = next(incoming);
       } else {
-        incoming.kill(true);
+        incoming.cancel();
         out = IterOutcome.NONE;
       }
       switch (out) {
-      case STOP:
       case NONE:
         // if we didn't do anything yet, send an empty schema.
         final BatchSchema sendSchema = incoming.getSchema() == null ?
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
index e54f76d..daa5e33 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
@@ -69,8 +69,8 @@ public class StatisticsWriterRecordBatch extends AbstractRecordBatch<Writer> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
@@ -97,9 +97,6 @@ public class StatisticsWriterRecordBatch extends AbstractRecordBatch<Writer> {
       upstream = next(incoming);
 
       switch(upstream) {
-        case STOP:
-          return upstream;
-
         case NOT_YET:
         case NONE:
           break;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index d600f12..e094d56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -157,9 +157,6 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
         container.setRecordCount(0);
 
         return;
-      case STOP:
-        state = BatchState.STOP;
-        return;
       case NONE:
         state = BatchState.DONE;
         return;
@@ -223,8 +220,6 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
         break outer;
       case NOT_YET:
         throw new UnsupportedOperationException();
-      case STOP:
-        return lastKnownOutcome;
       case OK_NEW_SCHEMA:
         // only change in the case that the schema truly changes.  Artificial schema changes are ignored.
         // schema change handling in case when EMIT is also seen is same as without EMIT. i.e. only if union type
@@ -486,8 +481,8 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index 1b368d2..3c2ba08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -70,8 +70,8 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
@@ -93,9 +93,6 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
       upstream = next(incoming);
 
       switch(upstream) {
-        case STOP:
-          return upstream;
-
         case NOT_YET:
           break;
         case NONE:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
index 506b594..7171e24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
@@ -212,10 +212,10 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
     boolean fallbackEnabled = context.getOptions().getOption(ExecConstants.HASHAGG_FALLBACK_ENABLED_KEY).bool_val;
     final AggPrelBase.OperatorPhase phase = popConfig.getAggPhase();
 
-    if ( phase.is2nd() && !fallbackEnabled ) {
+    if (phase.is2nd() && !fallbackEnabled) {
       minBatchesNeeded *= 2;  // 2nd phase (w/o fallback) needs at least 2 partitions
     }
-    if ( configuredBatchSize > memAvail / minBatchesNeeded ) { // no cast - memAvail may be bigger than max-int
+    if (configuredBatchSize > memAvail / minBatchesNeeded) { // no cast - memAvail may be bigger than max-int
       int reducedBatchSize = (int)(memAvail / minBatchesNeeded);
       logger.trace("Reducing configured batch size from: {} to: {}, due to Mem limit: {}",
         configuredBatchSize, reducedBatchSize, memAvail);
@@ -250,9 +250,6 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
         state = BatchState.DONE;
         container.buildSchema(SelectionVectorMode.NONE);
         return;
-      case STOP:
-        state = BatchState.STOP;
-        return;
       default:
         break;
     }
@@ -280,11 +277,11 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
         // or: 1st phase need to return (not fully grouped) partial output due to memory pressure
         aggregator.earlyOutput() ||
         // or: while handling an EMIT - returning output for that section
-        aggregator.handlingEmit() ) {
+        aggregator.handlingEmit()) {
       // then output the next batch downstream
       HashAggregator.AggIterOutcome aggOut = aggregator.outputCurrentBatch();
       // if Batch returned, or end of data, or Emit - then return the appropriate iter outcome
-      switch ( aggOut ) {
+      switch (aggOut) {
         case AGG_NONE:
           return IterOutcome.NONE;
         case AGG_OK:
@@ -297,62 +294,64 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       }
     }
 
-    if (wasKilled) { // if kill() was called before, then finish up
+    if (wasKilled) { // if cancel() was called before, then finish up
       aggregator.cleanup();
-      incoming.kill(false);
       return IterOutcome.NONE;
     }
 
     // Read and aggregate records
-    // ( may need to run again if the spilled partition that was read
-    //   generated new partitions that were all spilled )
+    // (may need to run again if the spilled partition that was read
+    //  generated new partitions that were all spilled)
     AggOutcome out;
     do {
-      //
       //  Read incoming batches and process their records
-      //
       out = aggregator.doWork();
     } while (out == AggOutcome.CALL_WORK_AGAIN);
 
     switch (out) {
-    case CLEANUP_AND_RETURN:
-      container.zeroVectors();
-      aggregator.cleanup();
-      state = BatchState.DONE;
-      // fall through
-    case RETURN_OUTCOME:
-      // rebuilds the schema in the case of complex writer expressions,
-      // since vectors would be added to batch run-time
-      IterOutcome outcome = aggregator.getOutcome();
-      switch (outcome) {
-        case OK:
-        case OK_NEW_SCHEMA:
-          if (firstBatch) {
-            if (CollectionUtils.isNotEmpty(complexWriters)) {
-              container.buildSchema(SelectionVectorMode.NONE);
-              outcome = IterOutcome.OK_NEW_SCHEMA;
+      case CLEANUP_AND_RETURN:
+        container.zeroVectors();
+        aggregator.cleanup();
+        state = BatchState.DONE;
+        // fall through
+      case RETURN_OUTCOME:
+        // rebuilds the schema in the case of complex writer expressions,
+        // since vectors would be added to batch run-time
+        IterOutcome outcome = aggregator.getOutcome();
+        switch (outcome) {
+          case OK:
+          case OK_NEW_SCHEMA:
+            if (firstBatch) {
+              if (CollectionUtils.isNotEmpty(complexWriters)) {
+                container.buildSchema(SelectionVectorMode.NONE);
+                // You'd be forgiven for thinking we should always return
+                // OK_NEW_SCHEMA for the first batch. It turns out, when
+                // two hash aggs are stacked, we get an error if the
+                // upstream one returns OK_NEW_SCHEMA first. Not sure the
+                // details, only know several tests fail.
+                outcome = IterOutcome.OK_NEW_SCHEMA;
+              }
+              firstBatch = false;
             }
-            firstBatch = false;
-          }
-          // fall thru
-        default:
-          return outcome;
-      }
-
-    case UPDATE_AGGREGATOR:
-      throw UserException.unsupportedError()
-          .message(SchemaChangeException.schemaChanged(
-              "Hash aggregate does not support schema change",
-              incomingSchema,
-              incoming.getSchema()).getMessage())
-          .build(logger);
-    default:
-      throw new IllegalStateException(String.format("Unknown state %s.", out));
+            break;
+          default:
+        }
+        return outcome;
+
+      case UPDATE_AGGREGATOR:
+        throw UserException.unsupportedError()
+            .message(SchemaChangeException.schemaChanged(
+                "Hash aggregate does not support schema change",
+                incomingSchema,
+                incoming.getSchema()).getMessage())
+            .build(logger);
+      default:
+        throw new IllegalStateException(String.format("Unknown state %s.", out));
     }
   }
 
   /**
-   * Creates a new Aggregator based on the current schema. If setup fails, this
+   * Creates a new aggregator based on the current schema. If setup fails, this
    * method is responsible for cleaning up and informing the context of the
    * failure state, as well is informing the upstream operators.
    *
@@ -503,22 +502,22 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
 
   private void setupGetIndex(ClassGenerator<HashAggregator> cg) {
     switch (incoming.getSchema().getSelectionVectorMode()) {
-    case FOUR_BYTE: {
-      JVar var = cg.declareClassField("sv4_", cg.getModel()._ref(SelectionVector4.class));
-      cg.getBlock("doSetup").assign(var, JExpr.direct("incoming").invoke("getSelectionVector4"));
-      cg.getBlock("getVectorIndex")._return(var.invoke("get").arg(JExpr.direct("recordIndex")));
-      return;
-    }
-    case NONE: {
-      cg.getBlock("getVectorIndex")._return(JExpr.direct("recordIndex"));
-      return;
-    }
-    case TWO_BYTE: {
-      JVar var = cg.declareClassField("sv2_", cg.getModel()._ref(SelectionVector2.class));
-      cg.getBlock("doSetup").assign(var, JExpr.direct("incoming").invoke("getSelectionVector2"));
-      cg.getBlock("getVectorIndex")._return(var.invoke("getIndex").arg(JExpr.direct("recordIndex")));
-      return;
-    }
+      case FOUR_BYTE: {
+        JVar var = cg.declareClassField("sv4_", cg.getModel()._ref(SelectionVector4.class));
+        cg.getBlock("doSetup").assign(var, JExpr.direct("incoming").invoke("getSelectionVector4"));
+        cg.getBlock("getVectorIndex")._return(var.invoke("get").arg(JExpr.direct("recordIndex")));
+        return;
+      }
+      case NONE: {
+        cg.getBlock("getVectorIndex")._return(JExpr.direct("recordIndex"));
+        return;
+      }
+      case TWO_BYTE: {
+        JVar var = cg.declareClassField("sv2_", cg.getModel()._ref(SelectionVector2.class));
+        cg.getBlock("doSetup").assign(var, JExpr.direct("incoming").invoke("getSelectionVector2"));
+        cg.getBlock("getVectorIndex")._return(var.invoke("getIndex").arg(JExpr.direct("recordIndex")));
+        return;
+      }
     }
   }
 
@@ -552,16 +551,16 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
+  protected void cancelIncoming() {
     wasKilled = true;
-    incoming.kill(sendUpstream);
+    incoming.cancel();
   }
 
   @Override
   public void dump() {
     logger.error("HashAggBatch[container={}, aggregator={}, groupByOutFieldIds={}, aggrOutFieldIds={}, " +
-            "incomingSchema={}, wasKilled={}, numGroupByExprs={}, numAggrExprs={}, popConfig={}]",
+            "incomingSchema={}, numGroupByExprs={}, numAggrExprs={}, popConfig={}]",
         container, aggregator, Arrays.toString(groupByOutFieldIds), Arrays.toString(aggrOutFieldIds), incomingSchema,
-        wasKilled, numGroupByExprs, numAggrExprs, popConfig);
+        numGroupByExprs, numAggrExprs, popConfig);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 3f13a5a..ba94faa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -67,6 +67,7 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
@@ -109,7 +110,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   private boolean useMemoryPrediction; // whether to use memory prediction to decide when to spill
   private long estMaxBatchSize; // used for adjusting #partitions and deciding when to spill
   private long estRowWidth; // the size of the internal "row" (keys + values + extra columns)
-  private long estValuesRowWidth; // the size of the internal values ( values + extra )
+  private long estValuesRowWidth; // the size of the internal values (values + extra)
   private long estOutputRowWidth; // the size of the output "row" (no extra columns)
   private long estValuesBatchSize; // used for "reserving" memory for the Values batch to overcome an OOM
   private long estOutgoingAllocSize; // used for "reserving" memory for the Outgoing Output Values to overcome an OOM
@@ -201,7 +202,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     }
 
     public void setTargetBatchRowCount(int batchRowCount) {
-      this.targetBatchRowCount = batchRowCount;
+      targetBatchRowCount = batchRowCount;
     }
 
     public int getCurrentRowCount() {
@@ -212,7 +213,7 @@ public abstract class HashAggTemplate implements HashAggregator {
 
       aggrValuesContainer = new VectorContainer();
       boolean success = false;
-      this.targetBatchRowCount = batchRowCount;
+      targetBatchRowCount = batchRowCount;
 
       try {
         ValueVector vector;
@@ -350,7 +351,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           "expressions.");
     }
 
-    this.htIdxHolder = new IndexPointer();
+    htIdxHolder = new IndexPointer();
     materializedValueFields = new MaterializedField[valueFieldIds.size()];
 
     if (valueFieldIds.size() > 0) {
@@ -368,7 +369,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     this.groupByOutFieldIds = groupByOutFieldIds; // retain these for delayedSetup, and to allow recreating hash tables (after a spill)
     numGroupByOutFields = groupByOutFieldIds.length;
 
-    // Start calculating the row widths (with the extra columns; the rest would be done in updateEstMaxBatchSize() )
+    // Start calculating the row widths (with the extra columns; the rest would be done in updateEstMaxBatchSize())
     estRowWidth = extraRowBytes;
     estValuesRowWidth = extraRowBytes;
 
@@ -390,13 +391,13 @@ public abstract class HashAggTemplate implements HashAggregator {
 
     // Set the number of partitions from the configuration (raise to a power of two, if needed)
     int numPartitions = (int)context.getOptions().getOption(ExecConstants.HASHAGG_NUM_PARTITIONS_VALIDATOR);
-    if ( numPartitions == 1 && phase.is2nd() ) { // 1st phase can still do early return with 1 partition
+    if (numPartitions == 1 && phase.is2nd()) { // 1st phase can still do early return with 1 partition
       canSpill = false;
       logger.warn("Spilling is disabled due to configuration setting of num_partitions to 1");
     }
     numPartitions = BaseAllocator.nextPowerOfTwo(numPartitions); // in case not a power of 2
 
-    if ( schema == null ) { estValuesBatchSize = estOutgoingAllocSize = estMaxBatchSize = 0; } // incoming was an empty batch
+    if (schema == null) { estValuesBatchSize = estOutgoingAllocSize = estMaxBatchSize = 0; } // incoming was an empty batch
     else {
       // Estimate the max batch size; should use actual data (e.g. lengths of varchars)
       updateEstMaxBatchSize(incoming);
@@ -405,17 +406,17 @@ public abstract class HashAggTemplate implements HashAggregator {
     reserveValueBatchMemory = reserveOutgoingMemory = estValuesBatchSize;
     long newMemoryLimit = allocator.getLimit() - reserveValueBatchMemory - reserveOutgoingMemory;
     long memAvail = newMemoryLimit - allocator.getAllocatedMemory();
-    if ( memAvail <= 0 ) { throw new OutOfMemoryException("Too little memory available"); }
+    if (memAvail <= 0) { throw new OutOfMemoryException("Too little memory available"); }
     allocator.setLimit(newMemoryLimit);
 
-    if ( !canSpill ) { // single phase, or spill disabled by configuation
+    if (!canSpill) { // single phase, or spill disabled by configuation
       numPartitions = 1; // single phase should use only a single partition (to save memory)
     } else { // two phase
       // Adjust down the number of partitions if needed - when the memory available can not hold as
       // many batches (configurable option), plus overhead (e.g. hash table, links, hash values))
-      while ( numPartitions * ( estMaxBatchSize * minBatchesPerPartition + 2 * 1024 * 1024) > memAvail ) {
+      while (numPartitions * (estMaxBatchSize * minBatchesPerPartition + 2 * 1024 * 1024) > memAvail) {
         numPartitions /= 2;
-        if ( numPartitions < 2) {
+        if (numPartitions < 2) {
           if (phase.is2nd()) {
             canSpill = false;  // 2nd phase needs at least 2 to make progress
 
@@ -440,7 +441,7 @@ public abstract class HashAggTemplate implements HashAggregator {
 
     // The following initial safety check should be revisited once we can lower the number of rows in a batch
     // In cases of very tight memory -- need at least memory to process one batch, plus overhead (e.g. hash table)
-    if ( numPartitions == 1 && ! canSpill ) {
+    if (numPartitions == 1 && !canSpill) {
       // if too little memory - behave like the old code -- practically no memory limit for hash aggregate
       // (but 1st phase can still spill, so it will maintain the original memory limit)
       allocator.setLimit(AbstractBase.MAX_ALLOCATION);  // 10_000_000_000L
@@ -459,9 +460,9 @@ public abstract class HashAggTemplate implements HashAggregator {
     plannedBatches = numPartitions; // each partition should allocate its first batch
 
     // initialize every (per partition) entry in the arrays
-    for (int i = 0; i < numPartitions; i++ ) {
+    for (int i = 0; i < numPartitions; i++) {
       try {
-        this.htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds);
+        htables[i] = baseHashTable.createAndSetupHashTable(groupByOutFieldIds);
       } catch (ClassTransformationException e) {
         throw UserException.unsupportedError(e)
             .message("Code generation error - likely an error in the code.")
@@ -473,7 +474,7 @@ public abstract class HashAggTemplate implements HashAggregator {
       } catch (SchemaChangeException sce) {
         throw new IllegalStateException("Unexpected Schema Change while creating a hash table",sce);
       }
-      this.batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
+      batchHolders[i] = new ArrayList<BatchHolder>(); // First BatchHolder is created when the first put request is received.
     }
     // Initialize the value vectors in the generated code (which point to the incoming or outgoing fields)
     try {
@@ -491,13 +492,13 @@ public abstract class HashAggTemplate implements HashAggregator {
 
   private void initializeSetup(RecordBatch newIncoming) throws SchemaChangeException, IOException {
     baseHashTable.updateIncoming(newIncoming, null); // after a spill - a new incoming
-    this.incoming = newIncoming;
+    incoming = newIncoming;
     currentBatchRecordCount = newIncoming.getRecordCount(); // first batch in this spill file
     nextPartitionToReturn = 0;
-    for (int i = 0; i < spilledState.getNumPartitions(); i++ ) {
+    for (int i = 0; i < spilledState.getNumPartitions(); i++) {
       htables[i].updateIncoming(newIncoming.getContainer(), null);
       htables[i].reset();
-      if ( batchHolders[i] != null) {
+      if (batchHolders[i] != null) {
         for (BatchHolder bh : batchHolders[i]) {
           bh.clear();
         }
@@ -517,7 +518,7 @@ public abstract class HashAggTemplate implements HashAggregator {
    * @param incoming
    */
   private void updateEstMaxBatchSize(RecordBatch incoming) {
-    if ( estMaxBatchSize > 0 ) { return; }  // no handling of a schema (or varchar) change
+    if (estMaxBatchSize > 0) { return; }  // no handling of a schema (or varchar) change
     // Use the sizer to get the input row width and the length of the longest varchar column
     RecordBatchSizer sizer = outgoing.getRecordBatchMemoryManager().getRecordBatchSizer();
     logger.trace("Incoming sizer: {}",sizer);
@@ -543,7 +544,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           TypeHelper.getSize(mr.getType());
       estRowWidth += fieldSize;
       estOutputRowWidth += fieldSize;
-      if ( fieldId < numGroupByOutFields ) { fieldId++; }
+      if (fieldId < numGroupByOutFields) { fieldId++; }
       else { estValuesRowWidth += fieldSize; }
     }
     // multiply by the max number of rows in a batch to get the final estimated max size
@@ -561,15 +562,16 @@ public abstract class HashAggTemplate implements HashAggregator {
     logger.trace("{} phase. Estimated internal row width: {} Values row width: {} batch size: {}  memory limit: {}  max column width: {}",
       phase.getName(),estRowWidth,estValuesRowWidth,estMaxBatchSize,allocator.getLimit(),maxColumnWidth);
 
-    if ( estMaxBatchSize > allocator.getLimit() ) {
+    if (estMaxBatchSize > allocator.getLimit()) {
       logger.warn("HashAggregate: Estimated max batch size {} is larger than the memory limit {}",estMaxBatchSize,allocator.getLimit());
     }
   }
 
   /**
-   *  Read and process (i.e., insert into the hash table and aggregate) records from the current batch.
-   *  Once complete, get the incoming NEXT batch and process it as well, etc.
-   *  For 1st phase, may return when an early output needs to be performed.
+   * Read and process (i.e., insert into the hash table and aggregate) records
+   * from the current batch. Once complete, get the incoming NEXT batch and
+   * process it as well, etc. For 1st phase, may return when an early output
+   * needs to be performed.
    *
    * @return Agg outcome status
    */
@@ -578,9 +580,9 @@ public abstract class HashAggTemplate implements HashAggregator {
 
     while (true) {
 
-      // This would be called only once - first time actual data arrives on incoming
-      if ( schema == null && incoming.getRecordCount() > 0 ) {
-        this.schema = incoming.getSchema();
+      // This is called only once - first time actual data arrives on incoming
+      if (schema == null && incoming.getRecordCount() > 0) {
+        schema = incoming.getSchema();
         currentBatchRecordCount = incoming.getRecordCount(); // initialize for first non empty batch
         // Calculate the number of partitions based on actual incoming data
         delayedSetup();
@@ -591,9 +593,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         outgoing.getRecordBatchMemoryManager().update(incoming);
       }
 
-      //
       //  loop through existing records in this batch, aggregating the values as necessary.
-      //
       if (EXTRA_DEBUG_1) {
         logger.debug("Starting outer loop of doWork()...");
       }
@@ -603,12 +603,12 @@ public abstract class HashAggTemplate implements HashAggregator {
         }
         checkGroupAndAggrValues(currentIndex);
 
-        if ( retrySameIndex ) { retrySameIndex = false; }  // need to retry this row (e.g. we had an OOM)
+        if (retrySameIndex) {retrySameIndex = false; }  // need to retry this row (e.g. we had an OOM)
         else { incIndex(); } // next time continue with the next incoming row
 
         // If adding a group discovered a memory pressure during 1st phase, then start
         // outputing some partition downstream in order to free memory.
-        if ( earlyOutput ) {
+        if (earlyOutput) {
           outputCurrentBatch();
           return AggOutcome.RETURN_OUTCOME;
         }
@@ -619,14 +619,11 @@ public abstract class HashAggTemplate implements HashAggregator {
       }
 
       // Cleanup the previous batch since we are done processing it.
-      for (VectorWrapper<?> v : incoming) {
-        v.getValueVector().clear();
-      }
+      VectorAccessibleUtilities.clear(incoming);
 
-      if ( handleEmit ) {
+      if (handleEmit) {
         outcome = IterOutcome.NONE; // finished behaving like OK, now behave like NONE
-      }
-      else {
+      } else {
         //
         // Get the NEXT input batch, initially from the upstream, later (if there was a spill)
         // from one of the spill files (The spill case is handled differently here to avoid
@@ -661,7 +658,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           if (EXTRA_DEBUG_1) {
             logger.debug("Received new schema.  Batch has {} records.", incoming.getRecordCount());
           }
-          this.cleanup();
+          cleanup();
           // TODO: new schema case needs to be handled appropriately
           return AggOutcome.UPDATE_AGGREGATOR;
 
@@ -690,7 +687,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           // the outgoing batch downstream (see innerNext() in HashAggBatch).
           buildComplete = true; // now should go and return outgoing
 
-          if ( handleEmit ) {
+          if (handleEmit) {
             buildComplete = false; // This was not a real NONE - more incoming is expected
             // don't aggregate this incoming batch again (in the loop above; when doWork() is called again)
             currentBatchRecordCount = 0;
@@ -701,7 +698,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           // in response to each next() call by a downstream operator
           AggIterOutcome aggOutcome = outputCurrentBatch();
 
-          switch ( aggOutcome ) {
+          switch (aggOutcome) {
             case AGG_RESTART:
               // Output of first batch returned a RESTART (all new partitions were spilled)
               return AggOutcome.CALL_WORK_AGAIN; // need to read/process the next partition
@@ -730,7 +727,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   private void useReservedValuesMemory() {
     // try to preempt an OOM by using the reserved memory
     long reservedMemory = reserveValueBatchMemory;
-    if ( reservedMemory > 0 ) { allocator.setLimit(allocator.getLimit() + reservedMemory); }
+    if (reservedMemory > 0) { allocator.setLimit(allocator.getLimit() + reservedMemory); }
 
     reserveValueBatchMemory = 0;
   }
@@ -740,7 +737,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   private void useReservedOutgoingMemory() {
     // try to preempt an OOM by using the reserved memory
     long reservedMemory = reserveOutgoingMemory;
-    if ( reservedMemory > 0 ) { allocator.setLimit(allocator.getLimit() + reservedMemory); }
+    if (reservedMemory > 0) { allocator.setLimit(allocator.getLimit() + reservedMemory); }
 
     reserveOutgoingMemory = 0;
   }
@@ -749,16 +746,16 @@ public abstract class HashAggTemplate implements HashAggregator {
    *
    */
   private void restoreReservedMemory() {
-    if ( 0 == reserveOutgoingMemory ) { // always restore OutputValues first (needed for spilling)
+    if (0 == reserveOutgoingMemory) { // always restore OutputValues first (needed for spilling)
       long memAvail = allocator.getLimit() - allocator.getAllocatedMemory();
-      if ( memAvail > estOutgoingAllocSize) {
+      if (memAvail > estOutgoingAllocSize) {
         allocator.setLimit(allocator.getLimit() - estOutgoingAllocSize);
         reserveOutgoingMemory = estOutgoingAllocSize;
       }
     }
-    if ( 0 == reserveValueBatchMemory ) {
+    if (0 == reserveValueBatchMemory) {
       long memAvail = allocator.getLimit() - allocator.getAllocatedMemory();
-      if ( memAvail > estValuesBatchSize) {
+      if (memAvail > estValuesBatchSize) {
         allocator.setLimit(allocator.getLimit() - estValuesBatchSize);
         reserveValueBatchMemory = estValuesBatchSize;
       }
@@ -816,7 +813,7 @@ public abstract class HashAggTemplate implements HashAggregator {
 
   @Override
   public void adjustOutputCount(int outputBatchSize, int oldRowWidth, int newRowWidth) {
-    for (int i = 0; i < spilledState.getNumPartitions(); i++ ) {
+    for (int i = 0; i < spilledState.getNumPartitions(); i++) {
       if (batchHolders[i] == null || batchHolders[i].size() == 0) {
         continue;
       }
@@ -832,18 +829,18 @@ public abstract class HashAggTemplate implements HashAggregator {
 
   @Override
   public void cleanup() {
-    if ( schema == null ) { return; } // not set up; nothing to clean
-    if ( phase.is2nd() && spillSet.getWriteBytes() > 0 ) {
+    if (schema == null) { return; } // not set up; nothing to clean
+    if (phase.is2nd() && spillSet.getWriteBytes() > 0) {
       stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
           (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
     }
     // clean (and deallocate) each partition
-    for ( int i = 0; i < spilledState.getNumPartitions(); i++) {
+    for (int i = 0; i < spilledState.getNumPartitions(); i++) {
           if (htables[i] != null) {
               htables[i].clear();
               htables[i] = null;
           }
-          if ( batchHolders[i] != null) {
+          if (batchHolders[i] != null) {
               for (BatchHolder bh : batchHolders[i]) {
                     bh.clear();
               }
@@ -852,7 +849,7 @@ public abstract class HashAggTemplate implements HashAggregator {
           }
 
           // delete any (still active) output spill file
-          if ( writers[i] != null && spillFiles[i] != null) {
+          if (writers[i] != null && spillFiles[i] != null) {
             try {
               spillSet.close(writers[i]);
               writers[i] = null;
@@ -873,7 +870,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         }
     }
     // Delete the currently handled (if any) spilled file
-    if ( newIncoming != null ) { newIncoming.close();  }
+    if (newIncoming != null) { newIncoming.close();  }
     spillSet.close(); // delete the spill directory(ies)
     htIdxHolder = null;
     materializedValueFields = null;
@@ -884,7 +881,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   private void reinitPartition(int part) /* throws SchemaChangeException /*, IOException */ {
     assert htables[part] != null;
     htables[part].reset();
-    if ( batchHolders[part] != null) {
+    if (batchHolders[part] != null) {
       for (BatchHolder bh : batchHolders[part]) {
         bh.clear();
       }
@@ -929,20 +926,20 @@ public abstract class HashAggTemplate implements HashAggregator {
    * @return The partition (number) chosen to be spilled
    */
   private int chooseAPartitionToFlush(int currPart, boolean tryAvoidCurr) {
-    if ( phase.is1st() && ! tryAvoidCurr) { return currPart; } // 1st phase: just use the current partition
+    if (phase.is1st() && !tryAvoidCurr) { return currPart; } // 1st phase: just use the current partition
     int currPartSize = batchHolders[currPart].size();
-    if ( currPartSize == 1 ) { currPartSize = -1; } // don't pick current if size is 1
+    if (currPartSize == 1) { currPartSize = -1; } // don't pick current if size is 1
     // first find the largest spilled partition
     int maxSizeSpilled = -1;
     int indexMaxSpilled = -1;
-    for (int isp = 0; isp < spilledState.getNumPartitions(); isp++ ) {
-      if ( isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size() ) {
+    for (int isp = 0; isp < spilledState.getNumPartitions(); isp++) {
+      if (isSpilled(isp) && maxSizeSpilled < batchHolders[isp].size()) {
         maxSizeSpilled = batchHolders[isp].size();
         indexMaxSpilled = isp;
       }
     }
     // Give the current (if already spilled) some priority
-    if ( ! tryAvoidCurr && isSpilled(currPart) && ( currPartSize + 1 >= maxSizeSpilled )) {
+    if (!tryAvoidCurr && isSpilled(currPart) && (currPartSize + 1 >= maxSizeSpilled)) {
       maxSizeSpilled = currPartSize;
       indexMaxSpilled = currPart;
     }
@@ -950,21 +947,21 @@ public abstract class HashAggTemplate implements HashAggregator {
     int maxSize = -1;
     int indexMax = -1;
     // Use the largest spilled (if found) as a base line, with a factor of 4
-    if ( indexMaxSpilled > -1 && maxSizeSpilled > 1 ) {
+    if (indexMaxSpilled > -1 && maxSizeSpilled > 1) {
       indexMax = indexMaxSpilled;
       maxSize = 4 * maxSizeSpilled;
     }
-    for ( int insp = 0; insp < spilledState.getNumPartitions(); insp++) {
-      if ( ! isSpilled(insp) && maxSize < batchHolders[insp].size() ) {
+    for (int insp = 0; insp < spilledState.getNumPartitions(); insp++) {
+      if (!isSpilled(insp) && maxSize < batchHolders[insp].size()) {
         indexMax = insp;
         maxSize = batchHolders[insp].size();
       }
     }
     // again - priority to the current partition
-    if ( ! tryAvoidCurr && ! isSpilled(currPart) && (currPartSize + 1 >= maxSize) ) {
+    if (!tryAvoidCurr && !isSpilled(currPart) && (currPartSize + 1 >= maxSize)) {
       return currPart;
     }
-    if ( maxSize <= 1 ) { // Can not make progress by spilling a single batch!
+    if (maxSize <= 1) { // Can not make progress by spilling a single batch!
       return -1; // try skipping this spill
     }
     return indexMax;
@@ -979,14 +976,14 @@ public abstract class HashAggTemplate implements HashAggregator {
 
     ArrayList<BatchHolder> currPartition = batchHolders[part];
     rowsInPartition = 0;
-    if ( EXTRA_DEBUG_SPILL ) {
+    if (EXTRA_DEBUG_SPILL) {
       logger.debug("HashAggregate: Spilling partition {} current cycle {} part size {}", part, spilledState.getCycle(), currPartition.size());
     }
 
-    if ( currPartition.size() == 0 ) { return; } // in case empty - nothing to spill
+    if (currPartition.size() == 0) { return; } // in case empty - nothing to spill
 
     // If this is the first spill for this partition, create an output stream
-    if ( ! isSpilled(part) ) {
+    if (!isSpilled(part)) {
 
       spillFiles[part] = spillSet.getNextSpillFile(spilledState.getCycle() > 0 ? Integer.toString(spilledState.getCycle()) : null);
 
@@ -999,7 +996,7 @@ public abstract class HashAggTemplate implements HashAggregator {
       }
     }
 
-    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++ ) {
+    for (int currOutBatchIndex = 0; currOutBatchIndex < currPartition.size(); currOutBatchIndex++) {
 
       // get the number of records in the batch holder that are pending output
       int numOutputRecords = currPartition.get(currOutBatchIndex).getNumPendingOutput();
@@ -1009,7 +1006,7 @@ public abstract class HashAggTemplate implements HashAggregator {
 
       allocateOutgoing(numOutputRecords);
       currPartition.get(currOutBatchIndex).outputValues();
-      this.htables[part].outputKeys(currOutBatchIndex, this.outContainer, numOutputRecords);
+      htables[part].outputKeys(currOutBatchIndex, outContainer, numOutputRecords);
 
       // set the value count for outgoing batch value vectors
       outContainer.setValueCount(numOutputRecords);
@@ -1045,7 +1042,7 @@ public abstract class HashAggTemplate implements HashAggregator {
 
   // These methods are overridden in the generated class when created as plain Java code.
   protected BatchHolder newBatchHolder(int batchRowCount) {
-    return this.injectMembers(new BatchHolder(batchRowCount));
+    return injectMembers(new BatchHolder(batchRowCount));
   }
 
   protected BatchHolder injectMembers(BatchHolder batchHolder) {
@@ -1063,24 +1060,24 @@ public abstract class HashAggTemplate implements HashAggregator {
   public AggIterOutcome outputCurrentBatch() {
 
     // Handle the case of an EMIT with an empty batch
-    if ( handleEmit && ( batchHolders == null || batchHolders[0].size() == 0 ) ) {
+    if (handleEmit && (batchHolders == null || batchHolders[0].size() == 0)) {
       lastBatchOutputCount = 0; // empty
       allocateOutgoing(0);
       outgoing.getContainer().setValueCount(0);
 
       // When returning the last outgoing batch (following an incoming EMIT), then replace OK with EMIT
-      this.outcome = IterOutcome.EMIT;
+      outcome = IterOutcome.EMIT;
       handleEmit = false; // finish handling EMIT
-      if ( outBatchIndex != null ) {
+      if (outBatchIndex != null) {
         outBatchIndex[0] = 0; // reset, for the next EMIT
       }
       return AggIterOutcome.AGG_EMIT;
     }
 
     // when incoming was an empty batch, just finish up
-    if ( schema == null ) {
+    if (schema == null) {
       logger.trace("Incoming was empty; output is an empty batch.");
-      this.outcome = IterOutcome.NONE; // no records were read
+      outcome = IterOutcome.NONE; // no records were read
       allFlushed = true;
       return AggIterOutcome.AGG_NONE;
     }
@@ -1090,14 +1087,14 @@ public abstract class HashAggTemplate implements HashAggregator {
     int currOutBatchIndex = outBatchIndex[earlyPartition];
     int partitionToReturn = earlyPartition;
 
-    if ( ! earlyOutput ) {
+    if (!earlyOutput) {
       // Update the next partition to return (if needed)
       // skip fully returned (or spilled) partitions
       while (nextPartitionToReturn < spilledState.getNumPartitions()) {
         //
         // If this partition was spilled - spill the rest of it and skip it
         //
-        if ( isSpilled(nextPartitionToReturn) ) {
+        if (isSpilled(nextPartitionToReturn)) {
           spillAPartition(nextPartitionToReturn); // spill the rest
           HashAggSpilledPartition sp = new HashAggSpilledPartition(
             spilledState.getCycle(),
@@ -1134,8 +1131,8 @@ public abstract class HashAggTemplate implements HashAggregator {
         // The following "if" is probably never used; due to a similar check at the end of this method
         if (spilledState.isEmpty()) { // and no spilled partitions
           allFlushed = true;
-          this.outcome = IterOutcome.NONE;
-          if ( phase.is2nd() && spillSet.getWriteBytes() > 0 ) {
+          outcome = IterOutcome.NONE;
+          if (phase.is2nd() && spillSet.getWriteBytes() > 0) {
             stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
                 (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
           }
@@ -1162,7 +1159,6 @@ public abstract class HashAggTemplate implements HashAggregator {
       }
 
       partitionToReturn = nextPartitionToReturn;
-
     }
 
     // get the number of records in the batch holder that are pending output
@@ -1170,15 +1166,15 @@ public abstract class HashAggTemplate implements HashAggregator {
 
     // The following accounting is for logging, metrics, etc.
     rowsInPartition += numPendingOutput;
-    if ( ! handlingSpills ) { rowsNotSpilled += numPendingOutput; }
+    if (!handlingSpills) { rowsNotSpilled += numPendingOutput; }
     else { rowsSpilledReturned += numPendingOutput; }
-    if ( earlyOutput ) { rowsReturnedEarly += numPendingOutput; }
+    if (earlyOutput) { rowsReturnedEarly += numPendingOutput; }
 
     allocateOutgoing(numPendingOutput);
 
     currPartition.get(currOutBatchIndex).outputValues();
     int numOutputRecords = numPendingOutput;
-    this.htables[partitionToReturn].outputKeys(currOutBatchIndex, this.outContainer, numPendingOutput);
+    htables[partitionToReturn].outputKeys(currOutBatchIndex, outContainer, numPendingOutput);
 
     // set the value count for outgoing batch value vectors
     outgoing.getContainer().setValueCount(numOutputRecords);
@@ -1186,9 +1182,9 @@ public abstract class HashAggTemplate implements HashAggregator {
     outgoing.getRecordBatchMemoryManager().updateOutgoingStats(numOutputRecords);
     RecordBatchStats.logRecordBatchStats(RecordBatchIOType.OUTPUT, outgoing, outgoing.getRecordBatchStatsContext());
 
-    this.outcome = IterOutcome.OK;
+    outcome = IterOutcome.OK;
 
-    if ( EXTRA_DEBUG_SPILL && phase.is2nd() ) {
+    if (EXTRA_DEBUG_SPILL && phase.is2nd()) {
       logger.debug("So far returned {} + SpilledReturned {}  total {} (spilled {})",rowsNotSpilled,rowsSpilledReturned,
         rowsNotSpilled+rowsSpilledReturned,
         rowsSpilled);
@@ -1199,7 +1195,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     // if just flushed the last batch in the partition
     if (outBatchIndex[partitionToReturn] == currPartition.size()) {
 
-      if ( EXTRA_DEBUG_SPILL ) {
+      if (EXTRA_DEBUG_SPILL) {
         logger.debug("HashAggregate: {} Flushed partition {} with {} batches total {} rows",
             earlyOutput ? "(Early)" : "",
             partitionToReturn, outBatchIndex[partitionToReturn], rowsInPartition);
@@ -1209,17 +1205,17 @@ public abstract class HashAggTemplate implements HashAggregator {
       // deallocate memory used by this partition, and re-initialize
       reinitPartition(partitionToReturn);
 
-      if ( earlyOutput ) {
+      if (earlyOutput) {
 
-        if ( EXTRA_DEBUG_SPILL ) {
+        if (EXTRA_DEBUG_SPILL) {
           logger.debug("HASH AGG: Finished (early) re-init partition {}, mem allocated: {}", earlyPartition, allocator.getAllocatedMemory());
         }
         outBatchIndex[earlyPartition] = 0; // reset, for next time
         earlyOutput = false; // done with early output
       }
-      else if ( handleEmit ) {
+      else if (handleEmit) {
         // When returning the last outgoing batch (following an incoming EMIT), then replace OK with EMIT
-        this.outcome = IterOutcome.EMIT;
+        outcome = IterOutcome.EMIT;
         handleEmit = false; // finished handling EMIT
         outBatchIndex[partitionToReturn] = 0; // reset, for the next EMIT
         return AggIterOutcome.AGG_EMIT;
@@ -1231,7 +1227,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         logger.trace("HashAggregate: All batches flushed.");
 
         // cleanup my internal state since there is nothing more to return
-        this.cleanup();
+        cleanup();
       }
     }
 
@@ -1274,8 +1270,8 @@ public abstract class HashAggTemplate implements HashAggregator {
     } else { // a bug ?
       errmsg = prefix + " OOM at " + phase.getName() + " Phase. Partitions: " + spilledState.getNumPartitions() +
       ". Estimated batch size: " + estMaxBatchSize + ". values size: " + estValuesBatchSize + ". Output alloc size: " + estOutgoingAllocSize;
-      if ( plannedBatches > 0 ) { errmsg += ". Planned batches: " + plannedBatches; }
-      if ( rowsSpilled > 0 ) { errmsg += ". Rows spilled so far: " + rowsSpilled; }
+      if (plannedBatches > 0) { errmsg += ". Planned batches: " + plannedBatches; }
+      if (rowsSpilled > 0) { errmsg += ". Rows spilled so far: " + rowsSpilled; }
     }
     errmsg += " Memory limit: " + allocator.getLimit() + " so far allocated: " + allocator.getAllocatedMemory() + ". ";
 
@@ -1291,7 +1287,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   // index is also used for the aggregation values maintained by the hash aggregate.
   private void checkGroupAndAggrValues(int incomingRowIdx) {
     assert incomingRowIdx >= 0;
-    assert ! earlyOutput;
+    assert !earlyOutput;
 
     // The hash code is computed once, then its lower bits are used to determine the
     // partition to use, and the higher bits determine the location in the hash table.
@@ -1315,7 +1311,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     String tryingTo = phase.is1st() ? "early return" : "spill";
 
     // Proactive spill - in case there is no reserve memory - spill and retry putting later
-    if ( reserveValueBatchMemory == 0 && canSpill ) {
+    if (reserveValueBatchMemory == 0 && canSpill) {
       logger.trace("Reserved memory runs short, trying to {} a partition and retry Hash Table put() again.", tryingTo);
 
       doSpill(currentPartition); // spill to free some memory
@@ -1332,13 +1328,13 @@ public abstract class HashAggTemplate implements HashAggregator {
       putStatus = htables[currentPartition].put(incomingRowIdx, htIdxHolder, hashCode, getTargetBatchCount());
 
     } catch (RetryAfterSpillException re) {
-      if ( ! canSpill ) { throw new OutOfMemoryException(getOOMErrorMsg("Can not spill")); }
+      if (!canSpill) { throw new OutOfMemoryException(getOOMErrorMsg("Can not spill")); }
 
       logger.trace("HT put failed with an OOM, trying to {} a partition and retry Hash Table put() again.", tryingTo);
 
       // for debugging - in case there's a leak
       long memDiff = allocator.getAllocatedMemory() - allocatedBeforeHTput;
-      if ( memDiff > 0 ) { logger.warn("Leak: HashTable put() OOM left behind {} bytes allocated",memDiff); }
+      if (memDiff > 0) { logger.warn("Leak: HashTable put() OOM left behind {} bytes allocated",memDiff); }
 
       doSpill(currentPartition); // spill to free some memory
 
@@ -1357,7 +1353,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     //       In case put() added a new batch (for the keys) inside the hash table,
     //       then a matching batch (for the aggregate columns) needs to be created
     //
-    if ( putStatus == HashTable.PutStatus.NEW_BATCH_ADDED ) {
+    if (putStatus == HashTable.PutStatus.NEW_BATCH_ADDED) {
       try {
 
         useReservedValuesMemory(); // try to preempt an OOM by using the reserve
@@ -1366,9 +1362,9 @@ public abstract class HashAggTemplate implements HashAggregator {
 
         restoreReservedMemory(); // restore the reserve, if possible
         // A reason to check for a spill - In case restore-reserve failed
-        needToCheckIfSpillIsNeeded = ( 0 == reserveValueBatchMemory );
+        needToCheckIfSpillIsNeeded = (0 == reserveValueBatchMemory);
 
-        if ( plannedBatches > 0 ) { plannedBatches--; } // just allocated a planned batch
+        if (plannedBatches > 0) { plannedBatches--; } // just allocated a planned batch
         long totalAddedMem = allocator.getAllocatedMemory() - allocatedBeforeHTput;
         long aggValuesAddedMem = allocator.getAllocatedMemory() - allocatedBeforeAggCol;
         logger.trace("MEMORY CHECK AGG: allocated now {}, added {}, total (with HT) added {}", allocator.getAllocatedMemory(),
@@ -1387,7 +1383,7 @@ public abstract class HashAggTemplate implements HashAggregator {
       } catch (OutOfMemoryException exc) {
           throw new OutOfMemoryException(getOOMErrorMsg("AGGR"), exc);
       }
-    } else if ( putStatus == HashTable.PutStatus.KEY_ADDED_LAST ) {
+    } else if (putStatus == HashTable.PutStatus.KEY_ADDED_LAST) {
         // If a batch just became full (i.e. another batch would be allocated soon) -- then need to
         // check (later, see below) if the memory limits are too close, and if so -- then spill !
         plannedBatches++; // planning to allocate one more batch
@@ -1411,7 +1407,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     // (Skip this if cannot spill, or not checking memory limits; in such case an OOM may
     // be encountered later - and some OOM cases are recoverable by spilling and retrying)
     // ===================================================================================
-    if ( needToCheckIfSpillIsNeeded && canSpill && useMemoryPrediction ) {
+    if (needToCheckIfSpillIsNeeded && canSpill && useMemoryPrediction) {
       spillIfNeeded(currentPartition);
     }
   }
@@ -1425,7 +1421,7 @@ public abstract class HashAggTemplate implements HashAggregator {
    */
   private void spillIfNeeded(int currentPartition, boolean forceSpill) {
     long maxMemoryNeeded = 0;
-    if ( !forceSpill ) { // need to check the memory in order to decide
+    if (!forceSpill) { // need to check the memory in order to decide
       // calculate the (max) new memory needed now; plan ahead for at least MIN batches
       maxMemoryNeeded = minBatchesPerPartition * Math.max(1, plannedBatches) * (estMaxBatchSize + MAX_BATCH_ROW_COUNT * (4 + 4 /* links + hash-values */));
       // Add the (max) size of the current hash table, in case it will double
@@ -1443,7 +1439,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     //
     //   Spill if (forced, or) the allocated memory plus the memory needed exceed the memory limit.
     //
-    if ( forceSpill || allocator.getAllocatedMemory() + maxMemoryNeeded > allocator.getLimit() ) {
+    if (forceSpill || allocator.getAllocatedMemory() + maxMemoryNeeded > allocator.getLimit()) {
 
       // Pick a "victim" partition to spill or return
       int victimPartition = chooseAPartitionToFlush(currentPartition, forceSpill);
@@ -1460,7 +1456,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         return;
       }
 
-      if ( phase.is2nd() ) {
+      if (phase.is2nd()) {
         long before = allocator.getAllocatedMemory();
 
         spillAPartition(victimPartition);
@@ -1472,7 +1468,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         // In case spilling did not free enough memory to recover the reserves
         boolean spillAgain = reserveOutgoingMemory == 0 || reserveValueBatchMemory == 0;
         // in some "edge" cases (e.g. testing), spilling one partition may not be enough
-        if ( spillAgain || allocator.getAllocatedMemory() + maxMemoryNeeded > allocator.getLimit() ) {
+        if (spillAgain || allocator.getAllocatedMemory() + maxMemoryNeeded > allocator.getLimit()) {
           int victimPartition2 = chooseAPartitionToFlush(victimPartition, true);
           if (victimPartition2 < 0) {
             // In the case of the forced spill, there is not enough memory to continue.
@@ -1495,7 +1491,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         earlyOutput = true;
         earlyPartition = victimPartition;
 
-        if ( EXTRA_DEBUG_SPILL ) {
+        if (EXTRA_DEBUG_SPILL) {
           logger.debug("picked partition {} for early output", victimPartition);
         }
       }
@@ -1510,7 +1506,7 @@ public abstract class HashAggTemplate implements HashAggregator {
    */
   private void updateStats(HashTable[] htables) {
     if (!spilledState.isFirstCycle() ||  // These stats are only for before processing spilled files
-      handleEmit ) { return; } // and no stats collecting when handling an EMIT
+      handleEmit) { return; } // and no stats collecting when handling an EMIT
     long numSpilled = 0;
     HashTableStats newStats = new HashTableStats();
     // sum the stats from all the partitions
@@ -1521,18 +1517,18 @@ public abstract class HashAggTemplate implements HashAggregator {
         numSpilled++;
       }
     }
-    this.stats.setLongStat(Metric.NUM_BUCKETS, htStats.numBuckets);
-    this.stats.setLongStat(Metric.NUM_ENTRIES, htStats.numEntries);
-    this.stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
-    this.stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
-    this.stats.setLongStat(Metric.NUM_PARTITIONS, spilledState.getNumPartitions());
-    this.stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in case no spill
-    if ( phase.is2nd() ) {
-      this.stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
+    stats.setLongStat(Metric.NUM_BUCKETS, htStats.numBuckets);
+    stats.setLongStat(Metric.NUM_ENTRIES, htStats.numEntries);
+    stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
+    stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
+    stats.setLongStat(Metric.NUM_PARTITIONS, spilledState.getNumPartitions());
+    stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in case no spill
+    if (phase.is2nd()) {
+      stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
     }
-    if ( rowsReturnedEarly > 0 ) {
+    if (rowsReturnedEarly > 0) {
       stats.setLongStat(Metric.SPILL_MB, // update stats - est. total MB returned early
-          (int) Math.round( rowsReturnedEarly * estOutputRowWidth / 1024.0D / 1024.0));
+          (int) Math.round(rowsReturnedEarly * estOutputRowWidth / 1024.0D / 1024.0));
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/SpilledRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/SpilledRecordBatch.java
index 586d34d..72e127e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/SpilledRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/SpilledRecordBatch.java
@@ -122,8 +122,8 @@ public class SpilledRecordBatch implements CloseableRecordBatch {
   public int getRecordCount() { return container.getRecordCount(); }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    this.close(); // delete the current spill file
+  public void cancel() {
+    close(); // delete the current spill file
   }
 
   /**
@@ -143,7 +143,6 @@ public class SpilledRecordBatch implements CloseableRecordBatch {
     }
 
     if (spillStream == null) {
-      lastOutcome = IterOutcome.STOP;
       throw new IllegalStateException("Spill stream was null");
     }
 
@@ -186,11 +185,6 @@ public class SpilledRecordBatch implements CloseableRecordBatch {
         container, spilledBatches, schema, spillFile, spillSet);
   }
 
-  @Override
-  public boolean hasFailed() {
-    return lastOutcome == IterOutcome.STOP;
-  }
-
   /**
    * Note: ignoring any IO errors (e.g. file not found)
    */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 476b316..7886a5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -169,9 +169,6 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
         state = BatchState.DONE;
         container.buildSchema(SelectionVectorMode.NONE);
         return;
-      case STOP:
-        state = BatchState.STOP;
-        return;
       default:
         break;
     }
@@ -235,8 +232,6 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
           }
           // else fall thru
         case NOT_YET:
-        case STOP:
-          return lastKnownOutcome;
         case OK_NEW_SCHEMA:
           createAggregator();
           firstBatchForSchema = true;
@@ -650,8 +645,8 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
index e2f431b..4a6822d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggTemplate.java
@@ -150,7 +150,6 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
 
                 case NONE:
                   out = IterOutcome.OK_NEW_SCHEMA;
-                case STOP:
                 default:
                   lastOutcome = out;
                   outcome = out;
@@ -328,7 +327,6 @@ public abstract class StreamingAggTemplate implements StreamingAggregator {
                     continue outside;
                   }
                 }
-              case STOP:
               default:
                 lastOutcome = out;
                 outcome = out;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
index be60982..73e464e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
@@ -95,7 +95,6 @@ public class BroadcastSenderRootExec extends BaseRootExec {
     RecordBatch.IterOutcome out = next(incoming);
     logger.debug("Outcome of sender next {}", out);
     switch(out){
-      case STOP:
       case NONE:
         for (int i = 0; i < tunnels.length; ++i) {
           FragmentWritableBatch b2 = FragmentWritableBatch.getEmptyLast(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
index a39d8a1..6fc7154 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvalSetupException.java
@@ -17,6 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl.filter;
 
+@SuppressWarnings("serial")
 public class EvalSetupException extends Exception{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EvalSetupException.class);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
index 6f48efb..d80f818 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/EvaluationPredicate.java
@@ -17,14 +17,6 @@
  */
 package org.apache.drill.exec.physical.impl.filter;
 
-import org.apache.drill.exec.record.selection.SelectionVector2;
-
 public class EvaluationPredicate {
-  private SelectionVector2 vector;
-
-  EvaluationPredicate(String pred){
-
-  }
-
-
+  EvaluationPredicate(String pred) { }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
index a90c9aa..4179077 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenRecordBatch.java
@@ -184,8 +184,8 @@ public class FlattenRecordBatch extends AbstractSingleRecordBatch<FlattenPOP> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    super.killIncoming(sendUpstream);
+  protected void cancelIncoming() {
+    super.cancelIncoming();
     hasRemainder = false;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
index fe38244..f2eb611 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
@@ -23,7 +23,6 @@ import javax.inject.Named;
 
 import org.apache.drill.exec.exception.OversizedAllocationException;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.RecordBatch;
@@ -42,7 +41,6 @@ public abstract class FlattenTemplate implements Flattener {
   private static final int OUTPUT_ROW_COUNT = ValueVector.MAX_ROW_COUNT;
 
   private ImmutableList<TransferPair> transfers;
-  private BufferAllocator outputAllocator;
   private SelectionVectorMode svMode;
   private RepeatedValueVector fieldToFlatten;
   private RepeatedValueVector.RepeatedAccessor accessor;
@@ -158,16 +156,16 @@ public abstract class FlattenTemplate implements Flattener {
         throw new UnsupportedOperationException("Flatten does not support selection vector inputs.");
       case TWO_BYTE:
         throw new UnsupportedOperationException("Flatten does not support selection vector inputs.");
+      default:
     }
     this.transfers = ImmutableList.copyOf(transfers);
-    outputAllocator = outgoing.getOutgoingContainer().getAllocator();
     doSetup(context, incoming, outgoing);
   }
 
   @Override
   public void resetGroupIndex() {
-    this.valueIndex = 0;
-    this.currentInnerValueIndex = 0;
+    valueIndex = 0;
+    currentInnerValueIndex = 0;
   }
 
   public abstract void doSetup(@Named("context") FragmentContext context,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index eeedc72..843d20b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -96,8 +96,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Implements the runtime execution for the Hash-Join operator
- * supporting INNER, LEFT OUTER, RIGHT OUTER, and FULL OUTER joins
+ * Implements the runtime execution for the Hash-Join operator supporting INNER,
+ * LEFT OUTER, RIGHT OUTER, and FULL OUTER joins
  * <p>
  * This implementation splits the incoming Build side rows into multiple
  * Partitions, thus allowing spilling of some of these partitions to disk if
@@ -125,8 +125,10 @@ import org.slf4j.LoggerFactory;
  * greater) is a waste, indicating that the number of partitions chosen was too
  * small.
  */
-public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implements RowKeyJoin {
-  private static final Logger logger = LoggerFactory.getLogger(HashJoinBatch.class);
+public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP>
+    implements RowKeyJoin {
+  private static final Logger logger = LoggerFactory
+      .getLogger(HashJoinBatch.class);
 
   /**
    * The maximum number of records within each internal batch.
@@ -138,7 +140,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   private final boolean semiJoin;
   private final boolean joinIsLeftOrFull;
   private final boolean joinIsRightOrFull;
-  private boolean skipHashTableBuild; // when outer side is empty, and the join is inner or left (see DRILL-6755)
+  private boolean skipHashTableBuild; // when outer side is empty, and the join
+                                      // is inner or left (see DRILL-6755)
 
   // Join conditions
   private final List<JoinCondition> conditions;
@@ -159,7 +162,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   // Fields used for partitioning
   /**
    * The number of {@link HashPartition}s. This is configured via a system
-   * option and set in {@link #partitionNumTuning(int, HashJoinMemoryCalculator.BuildSidePartitioning)}.
+   * option and set in
+   * {@link #partitionNumTuning(int, HashJoinMemoryCalculator.BuildSidePartitioning)}.
    */
   private int numPartitions = 1; // must be 2 to the power of bitsInMask
 
@@ -170,7 +174,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   private final MutableBoolean buildSideIsEmpty = new MutableBoolean(false);
   private final MutableBoolean probeSideIsEmpty = new MutableBoolean(false);
   private boolean canSpill = true;
-  private boolean wasKilled; // a kill was received, may need to clean spilled partns
+  private boolean wasKilled; // a kill was received, may need to clean spilled
+                             // partns
 
   /**
    * This array holds the currently active {@link HashPartition}s.
@@ -190,7 +195,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
   private final JoinControl joinControl;
 
-  // An iterator over the build side hash table (only applicable for row-key joins)
+  // An iterator over the build side hash table (only applicable for row-key
+  // joins)
   private boolean buildComplete;
 
   // indicates if we have previously returned an output batch
@@ -198,16 +204,19 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
   private int rightHVColPosition;
   private final BufferAllocator allocator;
-  // Local fields for left/right incoming - may be replaced when reading from spilled
+  // Local fields for left/right incoming - may be replaced when reading from
+  // spilled
   private RecordBatch buildBatch;
   private RecordBatch probeBatch;
 
   /**
-   * Flag indicating whether or not the first data holding build batch needs to be fetched.
+   * Flag indicating whether or not the first data holding build batch needs to
+   * be fetched.
    */
   private final MutableBoolean prefetchedBuild = new MutableBoolean(false);
   /**
-   * Flag indicating whether or not the first data holding probe batch needs to be fetched.
+   * Flag indicating whether or not the first data holding probe batch needs to
+   * be fetched.
    */
   private final MutableBoolean prefetchedProbe = new MutableBoolean(false);
 
@@ -215,10 +224,15 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   private final SpillSet spillSet;
   HashJoinPOP popConfig;
 
-  private final int originalPartition = -1; // the partition a secondary reads from
-  IntVector read_right_HV_vector; // HV vector that was read from the spilled batch
+  private final int originalPartition = -1; // the partition a secondary reads
+                                            // from
+  IntVector read_right_HV_vector; // HV vector that was read from the spilled
+                                  // batch
   private final int maxBatchesInMemory;
-  private final List<String> probeFields = new ArrayList<>(); // keep the same sequence with the bloomFilters
+  private final List<String> probeFields = new ArrayList<>(); // keep the same
+                                                              // sequence with
+                                                              // the
+                                                              // bloomFilters
   private boolean enableRuntimeFilter;
   private RuntimeFilterReporter runtimeFilterReporter;
   private ValueVectorHashHelper.Hash64 hash64;
@@ -228,20 +242,20 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   private boolean bloomFiltersGenerated;
 
   /**
-   * This holds information about the spilled partitions for the build and probe side.
+   * This holds information about the spilled partitions for the build and probe
+   * side.
    */
-  public static class HashJoinSpilledPartition extends AbstractSpilledPartitionMetadata {
+  public static class HashJoinSpilledPartition
+      extends AbstractSpilledPartitionMetadata {
     private final int innerSpilledBatches;
     private final String innerSpillFile;
     private int outerSpilledBatches;
     private String outerSpillFile;
     private boolean updatedOuter;
 
-    public HashJoinSpilledPartition(int cycle,
-                                    int originPartition,
-                                    int prevOriginPartition,
-                                    int innerSpilledBatches,
-                                    String innerSpillFile) {
+    public HashJoinSpilledPartition(int cycle, int originPartition,
+        int prevOriginPartition, int innerSpilledBatches,
+        String innerSpillFile) {
       super(cycle, originPartition, prevOriginPartition);
 
       this.innerSpilledBatches = innerSpilledBatches;
@@ -276,8 +290,10 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
     @Override
     public String makeDebugString() {
-      return String.format("Start reading spilled partition %d (prev %d) from cycle %d (with %d-%d batches).",
-        this.getOriginPartition(), this.getPrevOriginPartition(), this.getCycle(), outerSpilledBatches, innerSpilledBatches);
+      return String.format(
+          "Start reading spilled partition %d (prev %d) from cycle %d (with %d-%d batches).",
+          this.getOriginPartition(), this.getPrevOriginPartition(),
+          this.getCycle(), outerSpilledBatches, innerSpilledBatches);
     }
   }
 
@@ -308,35 +324,28 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
    */
   private final SpilledState<HashJoinSpilledPartition> spilledState = new SpilledState<>();
   private final HashJoinUpdater spilledStateUpdater = new HashJoinUpdater();
-  private HashJoinSpilledPartition spilledInners[]; // for the outer to find the partition
+  private HashJoinSpilledPartition spilledInners[]; // for the outer to find the
+                                                    // partition
 
   public enum Metric implements MetricDef {
-    NUM_BUCKETS,
-    NUM_ENTRIES,
-    NUM_RESIZING,
-    RESIZING_TIME_MS,
-    NUM_PARTITIONS,
-    SPILLED_PARTITIONS, // number of original partitions spilled to disk
-    SPILL_MB,         // Number of MB of data spilled to disk. This amount is first written,
-                      // then later re-read. So, disk I/O is twice this amount.
-    SPILL_CYCLE,       // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
-    LEFT_INPUT_BATCH_COUNT,
-    LEFT_AVG_INPUT_BATCH_BYTES,
-    LEFT_AVG_INPUT_ROW_BYTES,
-    LEFT_INPUT_RECORD_COUNT,
-    RIGHT_INPUT_BATCH_COUNT,
-    RIGHT_AVG_INPUT_BATCH_BYTES,
-    RIGHT_AVG_INPUT_ROW_BYTES,
-    RIGHT_INPUT_RECORD_COUNT,
-    OUTPUT_BATCH_COUNT,
-    AVG_OUTPUT_BATCH_BYTES,
-    AVG_OUTPUT_ROW_BYTES,
-    OUTPUT_RECORD_COUNT;
+    NUM_BUCKETS, NUM_ENTRIES, NUM_RESIZING, RESIZING_TIME_MS, NUM_PARTITIONS,
+    // number of original partitions spilled to disk
+    SPILLED_PARTITIONS,
+    SPILL_MB, // Number of MB of data spilled to disk. This amount is first
+              // written,
+              // then later re-read. So, disk I/O is twice this amount.
+    SPILL_CYCLE, // 0 - no spill, 1 - spill, 2 - SECONDARY, 3 - TERTIARY
+    LEFT_INPUT_BATCH_COUNT, LEFT_AVG_INPUT_BATCH_BYTES, LEFT_AVG_INPUT_ROW_BYTES,
+    LEFT_INPUT_RECORD_COUNT, RIGHT_INPUT_BATCH_COUNT, RIGHT_AVG_INPUT_BATCH_BYTES,
+    RIGHT_AVG_INPUT_ROW_BYTES, RIGHT_INPUT_RECORD_COUNT, OUTPUT_BATCH_COUNT,
+    AVG_OUTPUT_BATCH_BYTES, AVG_OUTPUT_ROW_BYTES, OUTPUT_RECORD_COUNT;
 
     // duplicate for hash ag
 
     @Override
-    public int metricId() { return ordinal(); }
+    public int metricId() {
+      return ordinal();
+    }
   }
 
   @Override
@@ -352,7 +361,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
     if (validSchema) {
       // We are able to construct a valid schema from the upstream data.
-      // Setting the state here makes sure AbstractRecordBatch returns OK_NEW_SCHEMA
+      // Setting the state here makes sure AbstractRecordBatch returns
+      // OK_NEW_SCHEMA
       state = BatchState.BUILD_SCHEMA;
 
       if (leftUpstream == OK_NEW_SCHEMA) {
@@ -366,7 +376,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
         rightHVColPosition = right.getContainer().getNumberOfColumns();
         // In special cases, when the probe side is empty, and
         // inner/left join - no need for Hash Table
-        skipHashTableBuild = leftUpstream == IterOutcome.NONE && ! joinIsRightOrFull;
+        skipHashTableBuild = leftUpstream == IterOutcome.NONE
+            && !joinIsRightOrFull;
         // We only need the hash tables if we have data on the build side.
         setupHashTable();
       }
@@ -376,7 +387,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
     // If we have a valid schema, this will build a valid container.
     // If we were unable to obtain a valid schema,
-    // we still need to build a dummy schema. This code handles both cases for us.
+    // we still need to build a dummy schema. This code handles both cases for
+    // us.
     setupOutputContainerSchema();
     container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
     container.setEmpty();
@@ -386,52 +398,52 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
    * Prefetches the first build side data holding batch.
    */
   private void prefetchFirstBuildBatch() {
-    rightUpstream = prefetchFirstBatch(rightUpstream,
-      prefetchedBuild,
-      buildSideIsEmpty,
-      RIGHT_INDEX,
-      buildBatch,
-      () -> {
-        batchMemoryManager.update(RIGHT_INDEX, 0, true);
-        RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
-            batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
-            getRecordBatchStatsContext());
-      });
+    rightUpstream = prefetchFirstBatch(rightUpstream, prefetchedBuild,
+        buildSideIsEmpty, RIGHT_INDEX, buildBatch, () -> {
+          batchMemoryManager.update(RIGHT_INDEX, 0, true);
+          RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_RIGHT,
+              batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX),
+              getRecordBatchStatsContext());
+        });
   }
 
   /**
    * Prefetches the first build side data holding batch.
    */
   private void prefetchFirstProbeBatch() {
-    leftUpstream =  prefetchFirstBatch(leftUpstream,
-      prefetchedProbe,
-      probeSideIsEmpty,
-      LEFT_INDEX,
-      probeBatch,
-      () -> {
-        batchMemoryManager.update(LEFT_INDEX, 0);
-        RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
-            batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
-            getRecordBatchStatsContext());
-      });
+    leftUpstream = prefetchFirstBatch(leftUpstream, prefetchedProbe,
+        probeSideIsEmpty, LEFT_INDEX, probeBatch, () -> {
+          batchMemoryManager.update(LEFT_INDEX, 0);
+          RecordBatchStats.logRecordBatchStats(RecordBatchIOType.INPUT_LEFT,
+              batchMemoryManager.getRecordBatchSizer(LEFT_INDEX),
+              getRecordBatchStatsContext());
+        });
   }
 
   /**
-   * Used to fetch the first data holding batch from either the build or probe side.
-   * @param outcome The current upstream outcome for either the build or probe side.
-   * @param prefetched A flag indicating if we have already done a prefetch of the first data holding batch for the probe or build side.
-   * @param isEmpty A flag indicating if the probe or build side is empty.
-   * @param index The upstream index of the probe or build batch.
-   * @param batch The probe or build batch itself.
-   * @param memoryManagerUpdate A lambda function to execute the memory manager update for the probe or build batch.
-   * @return The current {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}.
+   * Used to fetch the first data holding batch from either the build or probe
+   * side.
+   *
+   * @param outcome
+   *          The current upstream outcome for either the build or probe side.
+   * @param prefetched
+   *          A flag indicating if we have already done a prefetch of the first
+   *          data holding batch for the probe or build side.
+   * @param isEmpty
+   *          A flag indicating if the probe or build side is empty.
+   * @param index
+   *          The upstream index of the probe or build batch.
+   * @param batch
+   *          The probe or build batch itself.
+   * @param memoryManagerUpdate
+   *          A lambda function to execute the memory manager update for the
+   *          probe or build batch.
+   * @return The current
+   *         {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}.
    */
   private IterOutcome prefetchFirstBatch(IterOutcome outcome,
-                                         MutableBoolean prefetched,
-                                         MutableBoolean isEmpty,
-                                         int index,
-                                         RecordBatch batch,
-                                         Runnable memoryManagerUpdate) {
+      MutableBoolean prefetched, MutableBoolean isEmpty, int index,
+      RecordBatch batch, Runnable memoryManagerUpdate) {
     if (prefetched.booleanValue()) {
       // We have already prefetch the first data holding batch
       return outcome;
@@ -445,32 +457,34 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       outcome = sniffNonEmptyBatch(outcome, index, batch);
     }
 
-    isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE there is no data.
+    isEmpty.setValue(outcome == IterOutcome.NONE); // If we received NONE there
+                                                   // is no data.
 
-    if (outcome == IterOutcome.STOP) {
-      // We reached a termination state
-      state = BatchState.STOP;
-    } else {
-      // Got our first batch(es)
-      if (spilledState.isFirstCycle()) {
-        // Only collect stats for the first cycle
-        memoryManagerUpdate.run();
-      }
-      state = BatchState.FIRST;
+    // Got our first batch(es)
+    if (spilledState.isFirstCycle()) {
+      // Only collect stats for the first cycle
+      memoryManagerUpdate.run();
     }
-
+    state = BatchState.FIRST;
     return outcome;
   }
 
   /**
-   * Currently in order to accurately predict memory usage for spilling, the first non-empty build or probe side batch is needed. This method
-   * fetches the first non-empty batch from the probe or build side.
-   * @param curr The current outcome.
-   * @param inputIndex Index specifying whether to work with the prorbe or build input.
-   * @param recordBatch The probe or build record batch.
-   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} for the left or right record batch.
+   * Currently in order to accurately predict memory usage for spilling, the
+   * first non-empty build or probe side batch is needed. This method fetches
+   * the first non-empty batch from the probe or build side.
+   *
+   * @param curr
+   *          The current outcome.
+   * @param inputIndex
+   *          Index specifying whether to work with the prorbe or build input.
+   * @param recordBatch
+   *          The probe or build record batch.
+   * @return The {@link org.apache.drill.exec.record.RecordBatch.IterOutcome}
+   *         for the left or right record batch.
    */
-  private IterOutcome sniffNonEmptyBatch(IterOutcome curr, int inputIndex, RecordBatch recordBatch) {
+  private IterOutcome sniffNonEmptyBatch(IterOutcome curr, int inputIndex,
+      RecordBatch recordBatch) {
     while (true) {
       if (recordBatch.getRecordCount() != 0) {
         return curr;
@@ -479,34 +493,41 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       curr = next(inputIndex, recordBatch);
 
       switch (curr) {
-        case OK:
-          // We got a data batch
-          break;
-        case NOT_YET:
-          // We need to try again
-          break;
-        case EMIT:
-          throw new UnsupportedOperationException("We do not support " + EMIT);
-        default:
-          // Other cases are termination conditions
-          return curr;
+      case OK:
+        // We got a data batch
+        break;
+      case NOT_YET:
+        // We need to try again
+        break;
+      case EMIT:
+        throw new UnsupportedOperationException("We do not support " + EMIT);
+      default:
+        // Other cases are termination conditions
+        return curr;
       }
     }
   }
 
   /**
-   * Determines the memory calculator to use. If maxNumBatches is configured simple batch counting is used to spill. Otherwise
-   * memory calculations are used to determine when to spill.
+   * Determines the memory calculator to use. If maxNumBatches is configured
+   * simple batch counting is used to spill. Otherwise memory calculations are
+   * used to determine when to spill.
+   *
    * @return The memory calculator to use.
    */
   public HashJoinMemoryCalculator getCalculatorImpl() {
     if (maxBatchesInMemory == 0) {
-      double safetyFactor = context.getOptions().getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
-      double fragmentationFactor = context.getOptions().getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
-      double hashTableDoublingFactor = context.getOptions().getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
-      String hashTableCalculatorType = context.getOptions().getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
-
-      return new HashJoinMemoryCalculatorImpl(safetyFactor, fragmentationFactor, hashTableDoublingFactor, hashTableCalculatorType, semiJoin);
+      double safetyFactor = context.getOptions()
+          .getDouble(ExecConstants.HASHJOIN_SAFETY_FACTOR_KEY);
+      double fragmentationFactor = context.getOptions()
+          .getDouble(ExecConstants.HASHJOIN_FRAGMENTATION_FACTOR_KEY);
+      double hashTableDoublingFactor = context.getOptions()
+          .getDouble(ExecConstants.HASHJOIN_HASH_DOUBLE_FACTOR_KEY);
+      String hashTableCalculatorType = context.getOptions()
+          .getString(ExecConstants.HASHJOIN_HASHTABLE_CALC_TYPE_KEY);
+
+      return new HashJoinMemoryCalculatorImpl(safetyFactor, fragmentationFactor,
+          hashTableDoublingFactor, hashTableCalculatorType, semiJoin);
     } else {
       return new HashJoinMechanicalMemoryCalculator(maxBatchesInMemory);
     }
@@ -515,30 +536,32 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   @Override
   public IterOutcome innerNext() {
     if (wasKilled) {
-      // We have received a kill signal. We need to stop processing.
-      this.cleanup();
-      super.close();
+      // We have received a cancel signal. We need to stop processing.
+      cleanup();
       return IterOutcome.NONE;
     }
 
     prefetchFirstBuildBatch();
 
     if (rightUpstream.isError()) {
-      // A termination condition was reached while prefetching the first build side data holding batch.
+      // A termination condition was reached while prefetching the first build
+      // side data holding batch.
       // We need to terminate.
       return rightUpstream;
     }
 
     try {
-      /* If we are here for the first time, execute the build phase of the
-       * hash join and setup the run time generated class for the probe side
+      /*
+       * If we are here for the first time, execute the build phase of the hash
+       * join and setup the run time generated class for the probe side
        */
       if (state == BatchState.FIRST) {
         // Build the hash table, using the build side record batches.
         IterOutcome buildExecuteTermination = executeBuildPhase();
 
         if (buildExecuteTermination != null) {
-          // A termination condition was reached while executing the build phase.
+          // A termination condition was reached while executing the build
+          // phase.
           // We need to terminate.
           return buildExecuteTermination;
         }
@@ -546,14 +569,10 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
         buildComplete = true;
 
         if (isRowKeyJoin) {
-          // discard the first left batch which was fetched by buildSchema, and get the new
+          // discard the first left batch which was fetched by buildSchema, and
+          // get the new
           // one based on rowkey join
           leftUpstream = next(left);
-
-          if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) {
-            state = BatchState.STOP;
-            return leftUpstream;
-          }
         }
 
         // Update the hash table related stats for the operator
@@ -561,53 +580,51 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       }
 
       // Try to probe and project, or recursively handle a spilled partition
-      if (!buildSideIsEmpty.booleanValue() ||  // If there are build-side rows
-          joinIsLeftOrFull) {  // or if this is a left/full outer join
+      if (!buildSideIsEmpty.booleanValue() || // If there are build-side rows
+          joinIsLeftOrFull) { // or if this is a left/full outer join
 
         prefetchFirstProbeBatch();
 
-        if (leftUpstream.isError() ||
-            ( leftUpstream == NONE && ! joinIsRightOrFull )) {
-          // A termination condition was reached while prefetching the first probe side data holding batch.
+        if (leftUpstream.isError()
+            || (leftUpstream == NONE && !joinIsRightOrFull)) {
+          // A termination condition was reached while prefetching the first
+          // probe side data holding batch.
           // We need to terminate.
           return leftUpstream;
         }
 
-        if (!buildSideIsEmpty.booleanValue() || !probeSideIsEmpty.booleanValue()) {
-          // Only allocate outgoing vectors and execute probing logic if there is data
+        if (!buildSideIsEmpty.booleanValue()
+            || !probeSideIsEmpty.booleanValue()) {
+          // Only allocate outgoing vectors and execute probing logic if there
+          // is data
 
           if (state == BatchState.FIRST) {
             // Initialize various settings for the probe side
-            hashJoinProbe.setupHashJoinProbe(probeBatch,
-              this,
-              joinType,
-              semiJoin,
-              leftUpstream,
-              partitions,
-              spilledState.getCycle(),
-              container,
-              spilledInners,
-              buildSideIsEmpty.booleanValue(),
-              numPartitions,
-              rightHVColPosition);
+            hashJoinProbe.setupHashJoinProbe(probeBatch, this, joinType,
+                semiJoin, leftUpstream, partitions, spilledState.getCycle(),
+                container, spilledInners, buildSideIsEmpty.booleanValue(),
+                numPartitions, rightHVColPosition);
           }
 
           // Allocate the memory for the vectors in the output container
           batchMemoryManager.allocateVectors(container);
 
-          hashJoinProbe.setTargetOutputCount(batchMemoryManager.getOutputRowCount());
+          hashJoinProbe
+              .setTargetOutputCount(batchMemoryManager.getOutputRowCount());
 
           outputRecords = hashJoinProbe.probeAndProject();
 
           container.setValueCount(outputRecords);
 
           batchMemoryManager.updateOutgoingStats(outputRecords);
-          RecordBatchStats.logRecordBatchStats(RecordBatchIOType.OUTPUT, this, getRecordBatchStatsContext());
-
-          /* We are here because of one the following
-           * 1. Completed processing of all the records and we are done
-           * 2. We've filled up the outgoing batch to the maximum and we need to return upstream
-           * Either case build the output container's schema and return
+          RecordBatchStats.logRecordBatchStats(RecordBatchIOType.OUTPUT, this,
+              getRecordBatchStatsContext());
+
+          /*
+           * We are here because of one the following 1. Completed processing of
+           * all the records and we are done 2. We've filled up the outgoing
+           * batch to the maximum and we need to return upstream Either case
+           * build the output container's schema and return
            */
           if (outputRecords > 0 || state == BatchState.FIRST) {
             state = BatchState.NOT_FIRST;
@@ -623,47 +640,66 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
         }
 
         //
-        //  (recursively) Handle the spilled partitions, if any
+        // (recursively) Handle the spilled partitions, if any
         //
         if (!buildSideIsEmpty.booleanValue()) {
-          while (!spilledState.isEmpty()) {  // "while" is only used for skipping; see "continue" below
+          while (!spilledState.isEmpty()) { // "while" is only used for
+                                            // skipping; see "continue" below
 
             // Get the next (previously) spilled partition to handle as incoming
-            HashJoinSpilledPartition currSp = spilledState.getNextSpilledPartition();
+            HashJoinSpilledPartition currSp = spilledState
+                .getNextSpilledPartition();
 
-            // If the outer is empty (and it's not a right/full join) - try the next spilled partition
+            // If the outer is empty (and it's not a right/full join) - try the
+            // next spilled partition
             if (currSp.outerSpilledBatches == 0 && !joinIsRightOrFull) {
               continue;
             }
 
-            // Create a BUILD-side "incoming" out of the inner spill file of that partition
-            buildBatch = new SpilledRecordBatch(currSp.innerSpillFile, currSp.innerSpilledBatches, context, buildSchema, oContext, spillSet);
-            // The above ctor call also got the first batch; need to update the outcome
-            rightUpstream = ((SpilledRecordBatch) buildBatch).getInitialOutcome();
+            // Create a BUILD-side "incoming" out of the inner spill file of
+            // that partition
+            buildBatch = new SpilledRecordBatch(currSp.innerSpillFile,
+                currSp.innerSpilledBatches, context, buildSchema, oContext,
+                spillSet);
+            // The above ctor call also got the first batch; need to update the
+            // outcome
+            rightUpstream = ((SpilledRecordBatch) buildBatch)
+                .getInitialOutcome();
 
             if (currSp.outerSpilledBatches > 0) {
-              // Create a PROBE-side "incoming" out of the outer spill file of that partition
-              probeBatch = new SpilledRecordBatch(currSp.outerSpillFile, currSp.outerSpilledBatches, context, probeSchema, oContext, spillSet);
-              // The above ctor call also got the first batch; need to update the outcome
-              leftUpstream = ((SpilledRecordBatch) probeBatch).getInitialOutcome();
+              // Create a PROBE-side "incoming" out of the outer spill file of
+              // that partition
+              probeBatch = new SpilledRecordBatch(currSp.outerSpillFile,
+                  currSp.outerSpilledBatches, context, probeSchema, oContext,
+                  spillSet);
+              // The above ctor call also got the first batch; need to update
+              // the outcome
+              leftUpstream = ((SpilledRecordBatch) probeBatch)
+                  .getInitialOutcome();
             } else {
-              probeBatch = left; // if no outer batch then reuse left - needed for updateIncoming()
+              probeBatch = left; // if no outer batch then reuse left - needed
+                                 // for updateIncoming()
               leftUpstream = IterOutcome.NONE;
               hashJoinProbe.changeToFinalProbeState();
             }
 
             spilledState.updateCycle(stats, currSp, spilledStateUpdater);
-            state = BatchState.FIRST;  // TODO need to determine if this is still necessary since prefetchFirstBatchFromBothSides sets this
+            state = BatchState.FIRST; // TODO need to determine if this is still
+                                      // necessary since
+                                      // prefetchFirstBatchFromBothSides sets
+                                      // this
 
             prefetchedBuild.setValue(false);
             prefetchedProbe.setValue(false);
 
-            return innerNext(); // start processing the next spilled partition "recursively"
+            return innerNext(); // start processing the next spilled partition
+                                // "recursively"
           }
         }
 
       } else {
-        // Our build side is empty, we won't have any matches, clear the probe side
+        // Our build side is empty, we won't have any matches, clear the probe
+        // side
         killAndDrainLeftUpstream();
       }
 
@@ -679,67 +715,91 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   }
 
   /**
-   * In case an upstream data is no longer needed, send a kill and flush any remaining batch
+   * In case an upstream data is no longer needed, send a kill and flush any
+   * remaining batch
    *
-   * @param batch probe or build batch
-   * @param upstream which upstream
-   * @param isLeft is it the left or right
+   * @param batch
+   *          probe or build batch
+   * @param upstream
+   *          which upstream
+   * @param isLeft
+   *          is it the left or right
    */
-  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream, boolean isLeft) {
-    batch.kill(true);
-    while (upstream == IterOutcome.OK_NEW_SCHEMA || upstream == IterOutcome.OK) {
+  private void killAndDrainUpstream(RecordBatch batch, IterOutcome upstream,
+      boolean isLeft) {
+    batch.cancel();
+    while (upstream == IterOutcome.OK_NEW_SCHEMA
+        || upstream == IterOutcome.OK) {
       VectorAccessibleUtilities.clear(batch);
-      upstream = next( isLeft ? HashJoinHelper.LEFT_INPUT : HashJoinHelper.RIGHT_INPUT, batch);
+      upstream = next(
+          isLeft ? HashJoinHelper.LEFT_INPUT : HashJoinHelper.RIGHT_INPUT,
+          batch);
     }
   }
 
-  private void killAndDrainLeftUpstream() { killAndDrainUpstream(probeBatch, leftUpstream, true); }
-  private void killAndDrainRightUpstream() { killAndDrainUpstream(buildBatch, rightUpstream, false); }
+  private void killAndDrainLeftUpstream() {
+    killAndDrainUpstream(probeBatch, leftUpstream, true);
+  }
+
+  private void killAndDrainRightUpstream() {
+    killAndDrainUpstream(buildBatch, rightUpstream, false);
+  }
 
   private void setupHashTable() {
-    List<Comparator> comparators = Lists.newArrayListWithExpectedSize(conditions.size());
-    conditions.forEach(cond->comparators.add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
+    List<Comparator> comparators = Lists
+        .newArrayListWithExpectedSize(conditions.size());
+    conditions.forEach(cond -> comparators
+        .add(JoinUtils.checkAndReturnSupportedJoinComparator(cond)));
 
-    if ( skipHashTableBuild ) { return; }
+    if (skipHashTableBuild) {
+      return;
+    }
 
     // Setup the hash table configuration object
     List<NamedExpression> leftExpr = new ArrayList<>(conditions.size());
 
     // Create named expressions from the conditions
     for (int i = 0; i < conditions.size(); i++) {
-      leftExpr.add(new NamedExpression(conditions.get(i).getLeft(), new FieldReference("probe_side_" + i)));
+      leftExpr.add(new NamedExpression(conditions.get(i).getLeft(),
+          new FieldReference("probe_side_" + i)));
     }
 
     // Set the left named expression to be null if the probe batch is empty.
-    if (leftUpstream != IterOutcome.OK_NEW_SCHEMA && leftUpstream != IterOutcome.OK) {
+    if (leftUpstream != IterOutcome.OK_NEW_SCHEMA
+        && leftUpstream != IterOutcome.OK) {
       leftExpr = null;
     } else {
-      if (probeBatch.getSchema().getSelectionVectorMode() != BatchSchema.SelectionVectorMode.NONE) {
-        throw UserException.internalError(null)
-          .message("Hash join does not support probe batch with selection vectors.")
-          .addContext("Probe batch has selection mode",
-              (probeBatch.getSchema().getSelectionVectorMode()).toString())
-          .build(logger);
+      if (probeBatch.getSchema()
+          .getSelectionVectorMode() != BatchSchema.SelectionVectorMode.NONE) {
+        throw UserException.internalError(null).message(
+            "Hash join does not support probe batch with selection vectors.")
+            .addContext("Probe batch has selection mode",
+                (probeBatch.getSchema().getSelectionVectorMode()).toString())
+            .build(logger);
       }
     }
 
-    HashTableConfig htConfig = new HashTableConfig((int) context.getOptions().getOption(ExecConstants.MIN_HASH_TABLE_SIZE),
-      true, HashTable.DEFAULT_LOAD_FACTOR, rightExpr, leftExpr, comparators, joinControl.asInt());
+    HashTableConfig htConfig = new HashTableConfig(
+        (int) context.getOptions().getOption(ExecConstants.MIN_HASH_TABLE_SIZE),
+        true, HashTable.DEFAULT_LOAD_FACTOR, rightExpr, leftExpr, comparators,
+        joinControl.asInt());
 
     // Create the chained hash table
-    baseHashTable =
-      new ChainedHashTable(htConfig, context, allocator, buildBatch, probeBatch, null);
+    baseHashTable = new ChainedHashTable(htConfig, context, allocator,
+        buildBatch, probeBatch, null);
     if (enableRuntimeFilter) {
       setupHash64(htConfig);
     }
   }
 
   private void setupHash64(HashTableConfig htConfig) {
-    LogicalExpression[] keyExprsBuild = new LogicalExpression[htConfig.getKeyExprsBuild().size()];
+    LogicalExpression[] keyExprsBuild = new LogicalExpression[htConfig
+        .getKeyExprsBuild().size()];
     ErrorCollector collector = new ErrorCollectorImpl();
     int i = 0;
     for (NamedExpression ne : htConfig.getKeyExprsBuild()) {
-      LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), buildBatch, collector, context.getFunctionRegistry());
+      LogicalExpression expr = ExpressionTreeMaterializer.materialize(
+          ne.getExpr(), buildBatch, collector, context.getFunctionRegistry());
       collector.reportErrors(logger);
       if (expr == null) {
         continue;
@@ -761,15 +821,18 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       i++;
     }
     if (missingField) {
-      logger.info("As some build side key fields not found, runtime filter was disabled");
+      logger.info(
+          "As some build side key fields not found, runtime filter was disabled");
       enableRuntimeFilter = false;
       return;
     }
     RuntimeFilterDef runtimeFilterDef = popConfig.getRuntimeFilterDef();
-    List<BloomFilterDef> bloomFilterDefs = runtimeFilterDef.getBloomFilterDefs();
+    List<BloomFilterDef> bloomFilterDefs = runtimeFilterDef
+        .getBloomFilterDefs();
     for (BloomFilterDef bloomFilterDef : bloomFilterDefs) {
       String buildField = bloomFilterDef.getBuildField();
-      SchemaPath schemaPath = new SchemaPath(new PathSegment.NameSegment(buildField), ExpressionPosition.UNKNOWN);
+      SchemaPath schemaPath = new SchemaPath(
+          new PathSegment.NameSegment(buildField), ExpressionPosition.UNKNOWN);
       TypedFieldId typedFieldId = buildBatch.getValueVectorId(schemaPath);
       if (typedFieldId == null) {
         missingField = true;
@@ -779,28 +842,30 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       bloomFilterDef2buildId.put(bloomFilterDef, fieldId);
     }
     if (missingField) {
-      logger.info("As some build side join key fields not found, runtime filter was disabled");
+      logger.info(
+          "As some build side join key fields not found, runtime filter was disabled");
       enableRuntimeFilter = false;
       return;
     }
-    ValueVectorHashHelper hashHelper = new ValueVectorHashHelper(buildBatch, context);
+    ValueVectorHashHelper hashHelper = new ValueVectorHashHelper(buildBatch,
+        context);
     try {
       hash64 = hashHelper.getHash64(keyExprsBuild, buildSideTypeFieldIds);
     } catch (Exception e) {
       throw UserException.internalError(e)
-            .message("Failed to construct a field's hash64 dynamic codes")
-            .build(logger);
+          .message("Failed to construct a field's hash64 dynamic codes")
+          .build(logger);
     }
   }
 
   /**
-   *  Call only after num partitions is known
+   * Call only after num partitions is known
    */
   private void delayedSetup() {
     //
-    //  Find out the estimated max batch size, etc
-    //  and compute the max numPartitions possible
-    //  See partitionNumTuning()
+    // Find out the estimated max batch size, etc
+    // and compute the max numPartitions possible
+    // See partitionNumTuning()
     //
 
     spilledState.initialize(numPartitions);
@@ -812,11 +877,13 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
    * Initialize fields (that may be reused when reading spilled partitions)
    */
   private void initializeBuild() {
-    baseHashTable.updateIncoming(buildBatch, probeBatch); // in case we process the spilled files
+    baseHashTable.updateIncoming(buildBatch, probeBatch); // in case we process
+                                                          // the spilled files
     // Recreate the partitions every time build is initialized
-    for (int part = 0; part < numPartitions; part++ ) {
-      partitions[part] = new HashPartition(context, allocator, baseHashTable, buildBatch, probeBatch, semiJoin,
-        RECORDS_PER_BATCH, spillSet, part, spilledState.getCycle(), numPartitions);
+    for (int part = 0; part < numPartitions; part++) {
+      partitions[part] = new HashPartition(context, allocator, baseHashTable,
+          buildBatch, probeBatch, semiJoin, RECORDS_PER_BATCH, spillSet, part,
+          spilledState.getCycle(), numPartitions);
     }
 
     spilledInners = new HashJoinSpilledPartition[numPartitions];
@@ -824,25 +891,29 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   }
 
   /**
-   * Note:
-   * This method can not be called again as part of recursive call of executeBuildPhase() to handle spilled build partitions.
+   * Note: This method can not be called again as part of recursive call of
+   * executeBuildPhase() to handle spilled build partitions.
    */
   private void initializeRuntimeFilter() {
     if (!enableRuntimeFilter || bloomFiltersGenerated) {
       return;
     }
-    runtimeFilterReporter = new RuntimeFilterReporter((ExecutorFragmentContext) context);
+    runtimeFilterReporter = new RuntimeFilterReporter(
+        (ExecutorFragmentContext) context);
     RuntimeFilterDef runtimeFilterDef = popConfig.getRuntimeFilterDef();
-    //RuntimeFilter is not a necessary part of a HashJoin operator, only the query which satisfy the
-    //RuntimeFilterRouter's judgement will have the RuntimeFilterDef.
+    // RuntimeFilter is not a necessary part of a HashJoin operator, only the
+    // query which satisfy the
+    // RuntimeFilterRouter's judgement will have the RuntimeFilterDef.
     if (runtimeFilterDef != null) {
-      List<BloomFilterDef> bloomFilterDefs = runtimeFilterDef.getBloomFilterDefs();
+      List<BloomFilterDef> bloomFilterDefs = runtimeFilterDef
+          .getBloomFilterDefs();
       for (BloomFilterDef bloomFilterDef : bloomFilterDefs) {
         int buildFieldId = bloomFilterDef2buildId.get(bloomFilterDef);
         int numBytes = bloomFilterDef.getNumBytes();
-        String probeField =  bloomFilterDef.getProbeField();
+        String probeField = bloomFilterDef.getProbeField();
         probeFields.add(probeField);
-        BloomFilter bloomFilter = new BloomFilter(numBytes, context.getAllocator());
+        BloomFilter bloomFilter = new BloomFilter(numBytes,
+            context.getAllocator());
         bloomFilters.add(bloomFilter);
         bloomFilter2buildId.put(bloomFilter, buildFieldId);
       }
@@ -851,15 +922,12 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   }
 
   /**
-   * Tunes the number of partitions used by {@link HashJoinBatch}. If it is not possible to spill it gives up and reverts
-   * to unbounded in memory operation.
-   * @param maxBatchSize
-   * @param buildCalc
-   * @return
+   * Tunes the number of partitions used by {@link HashJoinBatch}. If it is not
+   * possible to spill it gives up and reverts to unbounded in memory operation.
    */
   private HashJoinMemoryCalculator.BuildSidePartitioning partitionNumTuning(
-    int maxBatchSize,
-    HashJoinMemoryCalculator.BuildSidePartitioning buildCalc) {
+      int maxBatchSize,
+      HashJoinMemoryCalculator.BuildSidePartitioning buildCalc) {
     // Get auto tuning result
     numPartitions = buildCalc.getNumPartitions();
 
@@ -868,15 +936,19 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     }
 
     if (buildCalc.getMaxReservedMemory() > allocator.getLimit()) {
-      // We don't have enough memory to do any spilling. Give up and do no spilling and have no limits
-
-      // TODO dirty hack to prevent regressions. Remove this once batch sizing is implemented.
-      // We don't have enough memory to do partitioning, we have to do everything in memory
-      String message = String.format("When using the minimum number of partitions %d we require %s memory but only have %s available. " +
-          "Forcing legacy behavoir of using unbounded memory in order to prevent regressions.",
-        numPartitions,
-        FileUtils.byteCountToDisplaySize(buildCalc.getMaxReservedMemory()),
-        FileUtils.byteCountToDisplaySize(allocator.getLimit()));
+      // We don't have enough memory to do any spilling. Give up and do no
+      // spilling and have no limits
+
+      // TODO dirty hack to prevent regressions. Remove this once batch sizing
+      // is implemented.
+      // We don't have enough memory to do partitioning, we have to do
+      // everything in memory
+      String message = String.format(
+          "When using the minimum number of partitions %d we require %s memory but only have %s available. "
+              + "Forcing legacy behavoir of using unbounded memory in order to prevent regressions.",
+          numPartitions,
+          FileUtils.byteCountToDisplaySize(buildCalc.getMaxReservedMemory()),
+          FileUtils.byteCountToDisplaySize(allocator.getLimit()));
       logger.warn(message);
 
       // create a Noop memory calculator
@@ -884,20 +956,14 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       calc.initialize(false);
       buildCalc = calc.next();
 
-      buildCalc.initialize(true,
-        true, // TODO Fix after growing hash values bug fixed
-        buildBatch,
-        probeBatch,
-        buildJoinColumns,
-        leftUpstream == IterOutcome.NONE, // probeEmpty
-        allocator.getLimit(),
-        numPartitions,
-        RECORDS_PER_BATCH,
-        RECORDS_PER_BATCH,
-        maxBatchSize,
-        maxBatchSize,
-        batchMemoryManager.getOutputBatchSize(),
-        HashTable.DEFAULT_LOAD_FACTOR);
+      buildCalc.initialize(true, true, // TODO Fix after growing hash values bug
+                                       // fixed
+          buildBatch, probeBatch, buildJoinColumns,
+          leftUpstream == IterOutcome.NONE, // probeEmpty
+          allocator.getLimit(), numPartitions, RECORDS_PER_BATCH,
+          RECORDS_PER_BATCH, maxBatchSize, maxBatchSize,
+          batchMemoryManager.getOutputBatchSize(),
+          HashTable.DEFAULT_LOAD_FACTOR);
 
       disableSpilling(null);
     }
@@ -906,20 +972,29 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   }
 
   /**
-   *  Disable spilling - use only a single partition and set the memory limit to the max ( 10GB )
-   *  @param reason If not null - log this as warning, else check fallback setting to either warn or fail.
+   * Disable spilling - use only a single partition and set the memory limit to
+   * the max ( 10GB )
+   *
+   * @param reason
+   *          If not null - log this as warning, else check fallback setting to
+   *          either warn or fail.
    */
   private void disableSpilling(String reason) {
-    // Fail, or just issue a warning if a reason was given, or a fallback option is enabled
-    if ( reason == null ) {
-      boolean fallbackEnabled = context.getOptions().getOption(ExecConstants.HASHJOIN_FALLBACK_ENABLED_KEY).bool_val;
+    // Fail, or just issue a warning if a reason was given, or a fallback option
+    // is enabled
+    if (reason == null) {
+      boolean fallbackEnabled = context.getOptions()
+          .getOption(ExecConstants.HASHJOIN_FALLBACK_ENABLED_KEY).bool_val;
       if (fallbackEnabled) {
-        logger.warn("Spilling is disabled - not enough memory available for internal partitioning. Falling back" +
-          " to use unbounded memory");
+        logger.warn(
+            "Spilling is disabled - not enough memory available for internal partitioning. Falling back"
+                + " to use unbounded memory");
       } else {
-        throw UserException.resourceError().message(String.format("Not enough memory for internal partitioning and fallback mechanism for " +
-          "HashJoin to use unbounded memory is disabled. Either enable fallback config %s using Alter " +
-          "session/system command or increase memory limit for Drillbit", ExecConstants.HASHJOIN_FALLBACK_ENABLED_KEY)).build(logger);
+        throw UserException.resourceError().message(String.format(
+            "Not enough memory for internal partitioning and fallback mechanism for "
+                + "HashJoin to use unbounded memory is disabled. Either enable fallback config %s using Alter "
+                + "session/system command or increase memory limit for Drillbit",
+            ExecConstants.HASHJOIN_FALLBACK_ENABLED_KEY)).build(logger);
       }
     } else {
       logger.warn(reason);
@@ -927,14 +1002,17 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
     numPartitions = 1; // We are only using one partition
     canSpill = false; // We cannot spill
-    allocator.setLimit(AbstractBase.MAX_ALLOCATION); // Violate framework and force unbounded memory
+    allocator.setLimit(AbstractBase.MAX_ALLOCATION); // Violate framework and
+                                                     // force unbounded memory
   }
 
   /**
-   *  Execute the BUILD phase; first read incoming and split rows into partitions;
-   *  may decide to spill some of the partitions
+   * Execute the BUILD phase; first read incoming and split rows into
+   * partitions; may decide to spill some of the partitions
    *
-   * @return Returns an {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} if a termination condition is reached. Otherwise returns null.
+   * @return Returns an
+   *         {@link org.apache.drill.exec.record.RecordBatch.IterOutcome} if a
+   *         termination condition is reached. Otherwise returns null.
    * @throws SchemaChangeException
    */
   public IterOutcome executeBuildPhase() throws SchemaChangeException {
@@ -943,7 +1021,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       return null;
     }
 
-    if ( skipHashTableBuild ) { // No hash table needed - then consume all the right upstream
+    if (skipHashTableBuild) { // No hash table needed - then consume all the
+                              // right upstream
       killAndDrainRightUpstream();
       return null;
     }
@@ -953,26 +1032,25 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     {
       // Initializing build calculator
       // Limit scope of these variables to this block
-      int maxBatchSize = spilledState.isFirstCycle()? RecordBatch.MAX_BATCH_ROW_COUNT: RECORDS_PER_BATCH;
-      boolean doMemoryCalculation = canSpill && !probeSideIsEmpty.booleanValue();
+      int maxBatchSize = spilledState.isFirstCycle()
+          ? RecordBatch.MAX_BATCH_ROW_COUNT
+          : RECORDS_PER_BATCH;
+      boolean doMemoryCalculation = canSpill
+          && !probeSideIsEmpty.booleanValue();
       HashJoinMemoryCalculator calc = getCalculatorImpl();
 
       calc.initialize(doMemoryCalculation);
       buildCalc = calc.next();
 
-      buildCalc.initialize(spilledState.isFirstCycle(), true, // TODO Fix after growing hash values bug fixed
-        buildBatch,
-        probeBatch,
-        buildJoinColumns,
-        probeSideIsEmpty.booleanValue(),
-        allocator.getLimit(),
-        numPartitions,
-        RECORDS_PER_BATCH,
-        RECORDS_PER_BATCH,
-        maxBatchSize,
-        maxBatchSize,
-        batchMemoryManager.getOutputBatchSize(),
-        HashTable.DEFAULT_LOAD_FACTOR);
+      buildCalc.initialize(spilledState.isFirstCycle(), true, // TODO Fix after
+                                                              // growing hash
+                                                              // values bug
+                                                              // fixed
+          buildBatch, probeBatch, buildJoinColumns,
+          probeSideIsEmpty.booleanValue(), allocator.getLimit(), numPartitions,
+          RECORDS_PER_BATCH, RECORDS_PER_BATCH, maxBatchSize, maxBatchSize,
+          batchMemoryManager.getOutputBatchSize(),
+          HashTable.DEFAULT_LOAD_FACTOR);
 
       if (spilledState.isFirstCycle() && doMemoryCalculation) {
         // Do auto tuning
@@ -990,7 +1068,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     initializeRuntimeFilter();
 
     // Make the calculator aware of our partitions
-    HashJoinMemoryCalculator.PartitionStatSet partitionStatSet = new HashJoinMemoryCalculator.PartitionStatSet(partitions);
+    HashJoinMemoryCalculator.PartitionStatSet partitionStatSet = new HashJoinMemoryCalculator.PartitionStatSet(
+        partitions);
     buildCalc.setPartitionStatSet(partitionStatSet);
 
     boolean moreData = true;
@@ -998,22 +1077,25 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       switch (rightUpstream) {
       case NONE:
       case NOT_YET:
-      case STOP:
         moreData = false;
         continue;
 
       case OK_NEW_SCHEMA:
         if (!buildSchema.equals(buildBatch.getSchema())) {
-          throw SchemaChangeException.schemaChanged("Hash join does not support schema changes in build side.", buildSchema, buildBatch.getSchema());
+          throw SchemaChangeException.schemaChanged(
+              "Hash join does not support schema changes in build side.",
+              buildSchema, buildBatch.getSchema());
+        }
+        for (HashPartition partn : partitions) {
+          partn.updateBatches();
         }
-        for (HashPartition partn : partitions) { partn.updateBatches(); }
         // Fall through
       case OK:
         batchMemoryManager.update(buildBatch, RIGHT_INDEX, 0, true);
         int currentRecordCount = buildBatch.getRecordCount();
-        //create runtime filter
+        // create runtime filter
         if (spilledState.isFirstCycle() && enableRuntimeFilter) {
-          //create runtime filter and send out async
+          // create runtime filter and send out async
           for (BloomFilter bloomFilter : bloomFilter2buildId.keySet()) {
             int fieldId = bloomFilter2buildId.get(bloomFilter);
             for (int ind = 0; ind < currentRecordCount; ind++) {
@@ -1022,31 +1104,39 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
             }
           }
         }
-        // Special treatment (when no spill, and single partition) -- use the incoming vectors as they are (no row copy)
-        if ( numPartitions == 1 ) {
+        // Special treatment (when no spill, and single partition) -- use the
+        // incoming vectors as they are (no row copy)
+        if (numPartitions == 1) {
           partitions[0].appendBatch(buildBatch);
           break;
         }
 
         if (!spilledState.isFirstCycle()) {
-          read_right_HV_vector = (IntVector) buildBatch.getContainer().getLast();
+          read_right_HV_vector = (IntVector) buildBatch.getContainer()
+              .getLast();
         }
 
-        // For every record in the build batch, hash the key columns and keep the result
+        // For every record in the build batch, hash the key columns and keep
+        // the result
         for (int ind = 0; ind < currentRecordCount; ind++) {
-          int hashCode = spilledState.isFirstCycle() ? partitions[0].getBuildHashCode(ind)
-            : read_right_HV_vector.getAccessor().get(ind); // get the hash value from the HV column
+          int hashCode = spilledState.isFirstCycle()
+              ? partitions[0].getBuildHashCode(ind)
+              : read_right_HV_vector.getAccessor().get(ind); // get the hash
+                                                             // value from the
+                                                             // HV column
           int currPart = hashCode & spilledState.getPartitionMask();
           hashCode >>>= spilledState.getBitsInMask();
           // semi-join skips join-key-duplicate rows
-          if ( semiJoin ) {
+          if (semiJoin) {
 
           }
-          // Append the new inner row to the appropriate partition; spill (that partition) if needed
-          partitions[currPart].appendInnerRow(buildBatch.getContainer(), ind, hashCode, buildCalc);
+          // Append the new inner row to the appropriate partition; spill (that
+          // partition) if needed
+          partitions[currPart].appendInnerRow(buildBatch.getContainer(), ind,
+              hashCode, buildCalc);
         }
 
-        if ( read_right_HV_vector != null ) {
+        if (read_right_HV_vector != null) {
           read_right_HV_vector.clear();
           read_right_HV_vector = null;
         }
@@ -1061,14 +1151,15 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     if (spilledState.isFirstCycle() && enableRuntimeFilter) {
       if (bloomFilter2buildId.size() > 0) {
         int hashJoinOpId = this.popConfig.getOperatorId();
-        runtimeFilterReporter.sendOut(bloomFilters, probeFields, this.popConfig.getRuntimeFilterDef(), hashJoinOpId);
+        runtimeFilterReporter.sendOut(bloomFilters, probeFields,
+            this.popConfig.getRuntimeFilterDef(), hashJoinOpId);
       }
     }
 
     // Move the remaining current batches into their temp lists, or spill
     // them if the partition is spilled. Add the spilled partitions into
     // the spilled partitions list
-    if ( numPartitions > 1 ) { // a single partition needs no completion
+    if (numPartitions > 1) { // a single partition needs no completion
       for (HashPartition partn : partitions) {
         partn.completeAnInnerBatch(false, partn.isSpilled());
       }
@@ -1077,15 +1168,18 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     prefetchFirstProbeBatch();
 
     if (leftUpstream.isError()) {
-      // A termination condition was reached while prefetching the first build side data holding batch.
+      // A termination condition was reached while prefetching the first build
+      // side data holding batch.
       // We need to terminate.
       return leftUpstream;
     }
 
-    HashJoinMemoryCalculator.PostBuildCalculations postBuildCalc = buildCalc.next();
+    HashJoinMemoryCalculator.PostBuildCalculations postBuildCalc = buildCalc
+        .next();
     postBuildCalc.initialize(probeSideIsEmpty.booleanValue()); // probeEmpty
 
-    //  Traverse all the in-memory partitions' incoming batches, and build their hash tables
+    // Traverse all the in-memory partitions' incoming batches, and build their
+    // hash tables
 
     for (int index = 0; index < partitions.length; index++) {
       HashPartition partn = partitions[index];
@@ -1104,9 +1198,9 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
           partn.buildContainersHashTableAndHelper();
         }
       } catch (OutOfMemoryException e) {
-        String message = "Failed building hash table on partition " + index + ":\n"
-          + makeDebugString() + "\n"
-          + postBuildCalc.makeDebugString();
+        String message = "Failed building hash table on partition " + index
+            + ":\n" + makeDebugString() + "\n"
+            + postBuildCalc.makeDebugString();
         // Include debug info
         throw new OutOfMemoryException(message, e);
       }
@@ -1117,18 +1211,18 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     }
 
     for (HashPartition partn : partitions) {
-      if ( partn.isSpilled() ) {
-        HashJoinSpilledPartition sp = new HashJoinSpilledPartition(spilledState.getCycle(),
-          partn.getPartitionNum(),
-          originalPartition,
-          partn.getPartitionBatchesCount(),
-          partn.getSpillFile());
+      if (partn.isSpilled()) {
+        HashJoinSpilledPartition sp = new HashJoinSpilledPartition(
+            spilledState.getCycle(), partn.getPartitionNum(), originalPartition,
+            partn.getPartitionBatchesCount(), partn.getSpillFile());
 
         spilledState.addPartition(sp);
-        spilledInners[partn.getPartitionNum()] = sp; // for the outer to find the SP later
+        spilledInners[partn.getPartitionNum()] = sp; // for the outer to find
+                                                     // the SP later
         partn.closeWriter();
 
-        partn.updateProbeRecordsPerBatch(postBuildCalc.getProbeRecordsPerBatch());
+        partn.updateProbeRecordsPerBatch(
+            postBuildCalc.getProbeRecordsPerBatch());
       }
     }
 
@@ -1137,12 +1231,14 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
   private void setupOutputContainerSchema() {
 
-    if (buildSchema != null && ! semiJoin ) {
+    if (buildSchema != null && !semiJoin) {
       for (MaterializedField field : buildSchema) {
         MajorType inputType = field.getType();
         MajorType outputType;
-        // If left or full outer join, then the output type must be nullable. However, map types are
-        // not nullable so we must exclude them from the check below (see DRILL-2197).
+        // If left or full outer join, then the output type must be nullable.
+        // However, map types are
+        // not nullable so we must exclude them from the check below (see
+        // DRILL-2197).
         if (joinIsLeftOrFull && inputType.getMode() == DataMode.REQUIRED
             && inputType.getMinorType() != TypeProtos.MinorType.MAP) {
           outputType = Types.overrideMode(inputType, DataMode.OPTIONAL);
@@ -1150,20 +1246,24 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
           outputType = inputType;
         }
 
-        // make sure to project field with children for children to show up in the schema
+        // make sure to project field with children for children to show up in
+        // the schema
         MaterializedField projected = field.withType(outputType);
         // Add the vector to our output container
         container.addOrGet(projected);
       }
     }
 
-    if (probeSchema != null) { // a probe schema was seen (even though the probe may had no rows)
+    if (probeSchema != null) { // a probe schema was seen (even though the probe
+                               // may had no rows)
       for (VectorWrapper<?> vv : probeBatch) {
         MajorType inputType = vv.getField().getType();
         MajorType outputType;
 
-        // If right or full outer join then the output type should be optional. However, map types are
-        // not nullable so we must exclude them from the check below (see DRILL-2771, DRILL-2197).
+        // If right or full outer join then the output type should be optional.
+        // However, map types are
+        // not nullable so we must exclude them from the check below (see
+        // DRILL-2771, DRILL-2197).
         if (joinIsRightOrFull && inputType.getMode() == DataMode.REQUIRED
             && inputType.getMinorType() != TypeProtos.MinorType.MAP) {
           outputType = Types.overrideMode(inputType, DataMode.OPTIONAL);
@@ -1171,7 +1271,8 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
           outputType = inputType;
         }
 
-        ValueVector v = container.addOrGet(MaterializedField.create(vv.getField().getName(), outputType));
+        ValueVector v = container.addOrGet(
+            MaterializedField.create(vv.getField().getName(), outputType));
         if (v instanceof AbstractContainerVector) {
           vv.getValueVector().makeTransferPair(v);
           v.clear();
@@ -1183,30 +1284,36 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
   // (After the inner side was read whole) - Has that inner partition spilled
   public boolean isSpilledInner(int part) {
-    if ( spilledInners == null ) { return false; } // empty inner
+    if (spilledInners == null) {
+      return false;
+    } // empty inner
     return spilledInners[part] != null;
   }
 
   /**
-   *  The constructor
+   * The constructor
    *
    * @param popConfig
    * @param context
-   * @param left  -- probe/outer side incoming input
-   * @param right -- build/iner side incoming input
+   * @param left
+   *          -- probe/outer side incoming input
+   * @param right
+   *          -- build/iner side incoming input
    * @throws OutOfMemoryException
    */
   public HashJoinBatch(HashJoinPOP popConfig, FragmentContext context,
-      RecordBatch left, /*Probe side record batch*/
-      RecordBatch right /*Build side record batch*/
+      RecordBatch left, /* Probe side record batch */
+      RecordBatch right /* Build side record batch */
   ) throws OutOfMemoryException {
     super(popConfig, context, true, left, right);
     this.buildBatch = right;
     this.probeBatch = left;
     joinType = popConfig.getJoinType();
     semiJoin = popConfig.isSemiJoin();
-    joinIsLeftOrFull  = joinType == JoinRelType.LEFT  || joinType == JoinRelType.FULL;
-    joinIsRightOrFull = joinType == JoinRelType.RIGHT || joinType == JoinRelType.FULL;
+    joinIsLeftOrFull = joinType == JoinRelType.LEFT
+        || joinType == JoinRelType.FULL;
+    joinIsRightOrFull = joinType == JoinRelType.RIGHT
+        || joinType == JoinRelType.FULL;
     conditions = popConfig.getConditions();
     this.popConfig = popConfig;
     this.isRowKeyJoin = popConfig.isRowKeyJoin();
@@ -1222,63 +1329,81 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
     for (int i = 0; i < conditions.size(); i++) {
       SchemaPath rightPath = (SchemaPath) conditions.get(i).getRight();
-      PathSegment.NameSegment nameSegment = (PathSegment.NameSegment)rightPath.getLastSegment();
+      PathSegment.NameSegment nameSegment = (PathSegment.NameSegment) rightPath
+          .getLastSegment();
       buildJoinColumns.add(nameSegment.getPath());
       String refName = "build_side_" + i;
-      rightExpr.add(new NamedExpression(conditions.get(i).getRight(), new FieldReference(refName)));
+      rightExpr.add(new NamedExpression(conditions.get(i).getRight(),
+          new FieldReference(refName)));
     }
 
     this.allocator = oContext.getAllocator();
 
-    numPartitions = (int)context.getOptions().getOption(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR);
-    if ( numPartitions == 1 ) { //
-      disableSpilling("Spilling is disabled due to configuration setting of num_partitions to 1");
+    numPartitions = (int) context.getOptions()
+        .getOption(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR);
+    if (numPartitions == 1) { //
+      disableSpilling(
+          "Spilling is disabled due to configuration setting of num_partitions to 1");
     }
 
-    numPartitions = BaseAllocator.nextPowerOfTwo(numPartitions); // in case not a power of 2
+    numPartitions = BaseAllocator.nextPowerOfTwo(numPartitions); // in case not
+                                                                 // a power of 2
 
-    long memLimit = context.getOptions().getOption(ExecConstants.HASHJOIN_MAX_MEMORY_VALIDATOR);
+    long memLimit = context.getOptions()
+        .getOption(ExecConstants.HASHJOIN_MAX_MEMORY_VALIDATOR);
 
     if (memLimit != 0) {
       allocator.setLimit(memLimit);
     }
 
-    RECORDS_PER_BATCH = (int)context.getOptions().getOption(ExecConstants.HASHJOIN_NUM_ROWS_IN_BATCH_VALIDATOR);
-    maxBatchesInMemory = (int)context.getOptions().getOption(ExecConstants.HASHJOIN_MAX_BATCHES_IN_MEMORY_VALIDATOR);
+    RECORDS_PER_BATCH = (int) context.getOptions()
+        .getOption(ExecConstants.HASHJOIN_NUM_ROWS_IN_BATCH_VALIDATOR);
+    maxBatchesInMemory = (int) context.getOptions()
+        .getOption(ExecConstants.HASHJOIN_MAX_BATCHES_IN_MEMORY_VALIDATOR);
 
-    logger.info("Memory limit {} bytes", FileUtils.byteCountToDisplaySize(allocator.getLimit()));
+    logger.info("Memory limit {} bytes",
+        FileUtils.byteCountToDisplaySize(allocator.getLimit()));
     spillSet = new SpillSet(context, popConfig);
 
-    // Create empty partitions (in the ctor - covers the case where right side is empty)
+    // Create empty partitions (in the ctor - covers the case where right side
+    // is empty)
     partitions = new HashPartition[0];
 
     // get the output batch size from config.
-    int configuredBatchSize = (int) context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
-    double avail_mem_factor = context.getOptions().getOption(ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR);
-    int outputBatchSize = Math.min(configuredBatchSize, Integer.highestOneBit((int)(allocator.getLimit() * avail_mem_factor)));
+    int configuredBatchSize = (int) context.getOptions()
+        .getOption(ExecConstants.OUTPUT_BATCH_SIZE_VALIDATOR);
+    double avail_mem_factor = context.getOptions()
+        .getOption(ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR);
+    int outputBatchSize = Math.min(configuredBatchSize,
+        Integer.highestOneBit((int) (allocator.getLimit() * avail_mem_factor)));
 
     RecordBatchStats.logRecordBatchStats(getRecordBatchStatsContext(),
-      "configured output batch size: %d, allocated memory %d, avail mem factor %f, output batch size: %d",
-      configuredBatchSize, allocator.getLimit(), avail_mem_factor, outputBatchSize);
-
-    batchMemoryManager = new JoinBatchMemoryManager(outputBatchSize, left, right, new HashSet<>());
+        "configured output batch size: %d, allocated memory %d, avail mem factor %f, output batch size: %d",
+        configuredBatchSize, allocator.getLimit(), avail_mem_factor,
+        outputBatchSize);
 
+    batchMemoryManager = new JoinBatchMemoryManager(outputBatchSize, left,
+        right, new HashSet<>());
 
     RecordBatchStats.printConfiguredBatchSize(getRecordBatchStatsContext(),
-      configuredBatchSize);
+        configuredBatchSize);
 
-    enableRuntimeFilter = context.getOptions().getOption(ExecConstants.HASHJOIN_ENABLE_RUNTIME_FILTER) && popConfig.getRuntimeFilterDef() != null;
+    enableRuntimeFilter = context.getOptions()
+        .getOption(ExecConstants.HASHJOIN_ENABLE_RUNTIME_FILTER)
+        && popConfig.getRuntimeFilterDef() != null;
   }
 
   /**
-   * This method is called when {@link HashJoinBatch} closes. It cleans up left over spilled files that are in the spill queue, and closes the
-   * spillSet.
+   * This method is called when {@link HashJoinBatch} closes. It cleans up left
+   * over spilled files that are in the spill queue, and closes the spillSet.
    */
   private void cleanup() {
-    if ( buildSideIsEmpty.booleanValue() ) { return; } // not set up; nothing to clean
-    if ( spillSet.getWriteBytes() > 0 ) {
+    if (buildSideIsEmpty.booleanValue()) {
+      return;
+    } // not set up; nothing to clean
+    if (spillSet.getWriteBytes() > 0) {
       stats.setLongStat(Metric.SPILL_MB, // update stats - total MB spilled
-        (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
+          (int) Math.round(spillSet.getWriteBytes() / 1024.0D / 1024.0));
     }
     // clean (and deallocate) each partition, and delete its spill file
     for (HashPartition partn : partitions) {
@@ -1290,13 +1415,17 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
       HashJoinSpilledPartition sp = spilledState.getNextSpilledPartition();
       try {
         spillSet.delete(sp.innerSpillFile);
-      } catch(IOException e) {
-        logger.warn("Cleanup: Failed to delete spill file {}",sp.innerSpillFile);
+      } catch (IOException e) {
+        logger.warn("Cleanup: Failed to delete spill file {}",
+            sp.innerSpillFile);
       }
       try { // outer file is added later; may be null if cleaning prematurely
-        if ( sp.outerSpillFile != null ) { spillSet.delete(sp.outerSpillFile); }
-      } catch(IOException e) {
-        logger.warn("Cleanup: Failed to delete spill file {}",sp.outerSpillFile);
+        if (sp.outerSpillFile != null) {
+          spillSet.delete(sp.outerSpillFile);
+        }
+      } catch (IOException e) {
+        logger.warn("Cleanup: Failed to delete spill file {}",
+            sp.outerSpillFile);
       }
     }
     // Delete the currently handled (if any) spilled files
@@ -1305,6 +1434,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
 
   /**
    * This creates a string that summarizes the memory usage of the operator.
+   *
    * @return A memory dump string.
    */
   public String makeDebugString() {
@@ -1313,28 +1443,37 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     for (int partitionIndex = 0; partitionIndex < partitions.length; partitionIndex++) {
       String partitionPrefix = "Partition " + partitionIndex + ": ";
       HashPartition hashPartition = partitions[partitionIndex];
-      sb.append(partitionPrefix).append(hashPartition.makeDebugString()).append("\n");
+      sb.append(partitionPrefix).append(hashPartition.makeDebugString())
+          .append("\n");
     }
 
     return sb.toString();
   }
 
   /**
-   * Updates the {@link HashTable} and spilling stats after the original build side is processed.
+   * Updates the {@link HashTable} and spilling stats after the original build
+   * side is processed.
    *
-   * Note: this does not update all the stats. The cycleNum is updated dynamically in {@link #innerNext()} and the total bytes
-   * written is updated at close time in {@link #cleanup()}.
+   * Note: this does not update all the stats. The cycleNum is updated
+   * dynamically in {@link #innerNext()} and the total bytes written is updated
+   * at close time in {@link #cleanup()}.
    */
   private void updateStats() {
-    if ( buildSideIsEmpty.booleanValue() ) { return; } // no stats when the right side is empty
-    if (!spilledState.isFirstCycle()) { return; } // These stats are only for before processing spilled files
+    if (buildSideIsEmpty.booleanValue()) {
+      return;
+    } // no stats when the right side is empty
+    if (!spilledState.isFirstCycle()) {
+      return;
+    } // These stats are only for before processing spilled files
 
     HashTableStats htStats = new HashTableStats();
     long numSpilled = 0;
     HashTableStats newStats = new HashTableStats();
     // sum the stats from all the partitions
-    for ( HashPartition partn : partitions ) {
-      if ( partn.isSpilled() ) { numSpilled++; }
+    for (HashPartition partn : partitions) {
+      if (partn.isSpilled()) {
+        numSpilled++;
+      }
       partn.getStats(newStats);
       htStats.addStats(newStats);
     }
@@ -1344,74 +1483,95 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
     stats.setLongStat(Metric.NUM_RESIZING, htStats.numResizing);
     stats.setLongStat(Metric.RESIZING_TIME_MS, htStats.resizingTime);
     stats.setLongStat(Metric.NUM_PARTITIONS, numPartitions);
-    stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in case no spill
+    stats.setLongStat(Metric.SPILL_CYCLE, spilledState.getCycle()); // Put 0 in
+                                                                    // case no
+                                                                    // spill
     stats.setLongStat(Metric.SPILLED_PARTITIONS, numSpilled);
   }
 
   /**
-   * Get the hash table iterator that is created for the build side of the hash join if
-   * this hash join was instantiated as a row-key join.
-   * @return hash table iterator or null if this hash join was not a row-key join or if it
-   * was a row-key join but the build has not yet completed.
+   * Get the hash table iterator that is created for the build side of the hash
+   * join if this hash join was instantiated as a row-key join.
+   *
+   * @return hash table iterator or null if this hash join was not a row-key
+   *         join or if it was a row-key join but the build has not yet
+   *         completed.
    */
   @Override
   public Pair<ValueVector, Integer> nextRowKeyBatch() {
     if (buildComplete) {
-      // partition 0 because Row Key Join has only a single partition - no spilling
+      // partition 0 because Row Key Join has only a single partition - no
+      // spilling
       Pair<VectorContainer, Integer> pp = partitions[0].nextBatch();
       if (pp != null) {
         VectorWrapper<?> vw = Iterables.get(pp.getLeft(), 0);
         ValueVector vv = vw.getValueVector();
         return Pair.of(vv, pp.getRight());
       }
-    } else if(partitions == null && firstOutputBatch) { //if there is data coming to right(build) side in build Schema stage, use it.
+    } else if (partitions == null && firstOutputBatch) { // if there is data
+                                                         // coming to
+                                                         // right(build) side in
+                                                         // build Schema stage,
+                                                         // use it.
       firstOutputBatch = false;
-      if ( right.getRecordCount() > 0 ) {
+      if (right.getRecordCount() > 0) {
         VectorWrapper<?> vw = Iterables.get(right, 0);
         ValueVector vv = vw.getValueVector();
-        return Pair.of(vv, right.getRecordCount()-1);
+        return Pair.of(vv, right.getRecordCount() - 1);
       }
     }
     return null;
   }
 
-  @Override    // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public boolean hasRowKeyBatch() {
     return buildComplete;
   }
 
-  @Override   // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public BatchState getBatchState() {
     return state;
   }
 
-  @Override  // implement RowKeyJoin interface
+  @Override // implement RowKeyJoin interface
   public void setBatchState(BatchState newState) {
     state = newState;
   }
 
   @Override
-  public void killIncoming(boolean sendUpstream) {
+  protected void cancelIncoming() {
     wasKilled = true;
-    probeBatch.kill(sendUpstream);
-    buildBatch.kill(sendUpstream);
+    probeBatch.cancel();
+    buildBatch.cancel();
   }
 
   public void updateMetrics() {
-    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_BATCH_COUNT, batchMemoryManager.getNumIncomingBatches(LEFT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_BATCH_BYTES, batchMemoryManager.getAvgInputBatchSize(LEFT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_ROW_BYTES, batchMemoryManager.getAvgInputRowWidth(LEFT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_RECORD_COUNT, batchMemoryManager.getTotalInputRecords(LEFT_INDEX));
-
-    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_BATCH_COUNT, batchMemoryManager.getNumIncomingBatches(RIGHT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_BATCH_BYTES, batchMemoryManager.getAvgInputBatchSize(RIGHT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_ROW_BYTES, batchMemoryManager.getAvgInputRowWidth(RIGHT_INDEX));
-    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_RECORD_COUNT, batchMemoryManager.getTotalInputRecords(RIGHT_INDEX));
-
-    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_BATCH_COUNT, batchMemoryManager.getNumOutgoingBatches());
-    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_BATCH_BYTES, batchMemoryManager.getAvgOutputBatchSize());
-    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_ROW_BYTES, batchMemoryManager.getAvgOutputRowWidth());
-    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_RECORD_COUNT, batchMemoryManager.getTotalOutputRecords());
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_BATCH_COUNT,
+        batchMemoryManager.getNumIncomingBatches(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_BATCH_BYTES,
+        batchMemoryManager.getAvgInputBatchSize(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_AVG_INPUT_ROW_BYTES,
+        batchMemoryManager.getAvgInputRowWidth(LEFT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.LEFT_INPUT_RECORD_COUNT,
+        batchMemoryManager.getTotalInputRecords(LEFT_INDEX));
+
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_BATCH_COUNT,
+        batchMemoryManager.getNumIncomingBatches(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_BATCH_BYTES,
+        batchMemoryManager.getAvgInputBatchSize(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_AVG_INPUT_ROW_BYTES,
+        batchMemoryManager.getAvgInputRowWidth(RIGHT_INDEX));
+    stats.setLongStat(HashJoinBatch.Metric.RIGHT_INPUT_RECORD_COUNT,
+        batchMemoryManager.getTotalInputRecords(RIGHT_INDEX));
+
+    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_BATCH_COUNT,
+        batchMemoryManager.getNumOutgoingBatches());
+    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_BATCH_BYTES,
+        batchMemoryManager.getAvgOutputBatchSize());
+    stats.setLongStat(HashJoinBatch.Metric.AVG_OUTPUT_ROW_BYTES,
+        batchMemoryManager.getAvgOutputRowWidth());
+    stats.setLongStat(HashJoinBatch.Metric.OUTPUT_RECORD_COUNT,
+        batchMemoryManager.getTotalOutputRecords());
   }
 
   @Override
@@ -1427,41 +1587,59 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> implem
   @Override
   public void close() {
     if (!spilledState.isFirstCycle()) { // spilling happened
-      // In case closing due to cancellation, BaseRootExec.close() does not close the open
-      // SpilledRecordBatch "scanners" as it only knows about the original left/right ops.
-      killIncoming(false);
+      // In case closing due to cancellation, BaseRootExec.close() does not
+      // close the open
+      // SpilledRecordBatch "scanners" as it only knows about the original
+      // left/right ops.
+      // TODO: Code that was here didn't actually close the "scanners"
     }
 
     updateMetrics();
 
     RecordBatchStats.logRecordBatchStats(getRecordBatchStatsContext(),
-      "incoming aggregate left: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX), batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX), batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
+        "incoming aggregate left: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
+        batchMemoryManager
+            .getNumIncomingBatches(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager
+            .getAvgInputBatchSize(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager
+            .getAvgInputRowWidth(JoinBatchMemoryManager.LEFT_INDEX),
+        batchMemoryManager
+            .getTotalInputRecords(JoinBatchMemoryManager.LEFT_INDEX));
 
     RecordBatchStats.logRecordBatchStats(getRecordBatchStatsContext(),
-      "incoming aggregate right: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
-      batchMemoryManager.getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX), batchMemoryManager.getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
-      batchMemoryManager.getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX), batchMemoryManager.getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
+        "incoming aggregate right: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
+        batchMemoryManager
+            .getNumIncomingBatches(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager
+            .getAvgInputBatchSize(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager
+            .getAvgInputRowWidth(JoinBatchMemoryManager.RIGHT_INDEX),
+        batchMemoryManager
+            .getTotalInputRecords(JoinBatchMemoryManager.RIGHT_INDEX));
 
     RecordBatchStats.logRecordBatchStats(getRecordBatchStatsContext(),
-      "outgoing aggregate: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
-      batchMemoryManager.getNumOutgoingBatches(), batchMemoryManager.getAvgOutputBatchSize(),
-      batchMemoryManager.getAvgOutputRowWidth(), batchMemoryManager.getTotalOutputRecords());
+        "outgoing aggregate: batch count : %d, avg bytes : %d,  avg row bytes : %d, record count : %d",
+        batchMemoryManager.getNumOutgoingBatches(),
+        batchMemoryManager.getAvgOutputBatchSize(),
+        batchMemoryManager.getAvgOutputRowWidth(),
+        batchMemoryManager.getTotalOutputRecords());
 
     cleanup();
     super.close();
   }
 
   public HashJoinProbe setupHashJoinProbe() {
-    //  No real code generation !!
+    // No real code generation !!
     return new HashJoinProbeTemplate();
   }
 
   @Override
   public void dump() {
-    logger.error("HashJoinBatch[container={}, left={}, right={}, leftOutcome={}, rightOutcome={}, joinType={}, hashJoinProbe={}," +
-            " rightExpr={}, canSpill={}, buildSchema={}, probeSchema={}]", container, left, right, leftUpstream, rightUpstream,
-        joinType, hashJoinProbe, rightExpr, canSpill, buildSchema, probeSchema);
+    logger.error(
+        "HashJoinBatch[container={}, left={}, right={}, leftOutcome={}, rightOutcome={}, joinType={}, hashJoinProbe={},"
+            + " rightExpr={}, canSpill={}, buildSchema={}, probeSchema={}]",
+        container, left, right, leftUpstream, rightUpstream, joinType,
+        hashJoinProbe, rightExpr, canSpill, buildSchema, probeSchema);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
index 368bff6..dea15d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
@@ -54,12 +54,14 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     this.semiJoin = semiJoin;
   }
 
+  @Override
   public void initialize(boolean doMemoryCalculation) {
     Preconditions.checkState(!initialized);
     initialized = true;
     this.doMemoryCalculation = doMemoryCalculation;
   }
 
+  @Override
   public BuildSidePartitioning next() {
     Preconditions.checkState(initialized);
 
@@ -154,8 +156,6 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
   }
 
   /**
-   * <h1>Basic Functionality</h1>
-   * <p>
    * At this point we need to reserve memory for the following:
    * <ol>
    *   <li>An incoming batch</li>
@@ -165,7 +165,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
    * If we run out of room and need to start spilling, we need to specify which partitions
    * need to be spilled.
    * </p>
-   * <h1>Life Cycle</h1>
+   * <h4>Life Cycle</h4>
    * <p>
    *   <ul>
    *     <li><b>Step 0:</b> Call {@link #initialize(boolean, boolean, RecordBatch, RecordBatch, Set, boolean, long, int, int, int, int, int, int, double)}.
@@ -189,9 +189,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     private int maxBatchNumRecordsBuild;
     private int maxBatchNumRecordsProbe;
     private long memoryAvailable;
-    private boolean probeEmpty;
     private long maxBuildBatchSize;
-    private long maxProbeBatchSize;
     private long maxOutputBatchSize;
     private int initialPartitions;
     private int partitions;
@@ -307,7 +305,6 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       this.reserveHash = reserveHash;
       this.keySizes = Preconditions.checkNotNull(keySizes);
       this.memoryAvailable = memoryAvailable;
-      this.probeEmpty = probeEmpty;
       this.buildSizePredictor = buildSizePredictor;
       this.probeSizePredictor = probeSizePredictor;
       this.initialPartitions = initialPartitions;
@@ -348,7 +345,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     }
 
     /**
-     * This method calculates the amount of memory we need to reserve while partitioning. It also
+     * Calculates the amount of memory we need to reserve while partitioning. It also
      * calculates the size of a partition batch.
      */
     private void calculateMemoryUsage()
@@ -361,13 +358,13 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
         partitionProbeBatchSize = probeSizePredictor.predictBatchSize(recordsPerPartitionBatchProbe, reserveHash);
       }
 
-      maxOutputBatchSize = (long) ((double)outputBatchSize * fragmentationFactor * safetyFactor);
+      maxOutputBatchSize = (long) (outputBatchSize * fragmentationFactor * safetyFactor);
 
       long probeReservedMemory = 0;
 
       for (partitions = initialPartitions;; partitions /= 2) {
         // The total amount of memory to reserve for incomplete batches across all partitions
-        long incompletePartitionsBatchSizes = ((long) partitions) * partitionBuildBatchSize;
+        long incompletePartitionsBatchSizes = (partitions) * partitionBuildBatchSize;
         // We need to reserve all the space for incomplete batches, and the incoming batch as well as the
         // probe batch we sniffed.
         reservedMemory = incompletePartitionsBatchSizes + maxBuildBatchSize;
@@ -448,7 +445,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
 
       if (reserveHash) {
         // Include the hash sizes for the batch
-        consumedMemory += ((long) IntVector.VALUE_WIDTH) * partitionStatsSet.getNumInMemoryRecords();
+        consumedMemory += (IntVector.VALUE_WIDTH) * partitionStatsSet.getNumInMemoryRecords();
       }
 
       consumedMemory += RecordBatchSizer.multiplyByFactor(partitionStatsSet.getConsumedMemory(), fragmentationFactor);
@@ -541,14 +538,13 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
   }
 
   /**
-   * <h1>Basic Functionality</h1>
    * <p>
    *   In this state, we need to make sure there is enough room to spill probe side batches, if
    *   spilling is necessary. If there is not enough room, we have to evict build side partitions.
    *   If we don't have to evict build side partitions in this state, then we are done. If we do have
    *   to evict build side partitions then we have to recursively repeat the process.
    * </p>
-   * <h1>Lifecycle</h1>
+   * <h4>Lifecycle</h4>
    * <p>
    *   <ul>
    *     <li><b>Step 1:</b> Call {@link #initialize(boolean)}. This
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
index d40f6a3..79956a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
@@ -248,7 +248,6 @@ public class HashJoinProbeTemplate implements HashJoinProbe {
         switch (leftUpstream) {
           case NONE:
           case NOT_YET:
-          case STOP:
             recordsProcessed = 0;
             recordsToProcess = 0;
             changeToFinalProbeState();
@@ -286,6 +285,8 @@ public class HashJoinProbeTemplate implements HashJoinProbe {
             if (cycleNum > 0) {
               read_left_HV_vector = (IntVector) probeBatch.getContainer().getLast(); // Needed ?
             }
+            break;
+          default:
         }
       }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinStateCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinStateCalculator.java
index 4036438..14f9305 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinStateCalculator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinStateCalculator.java
@@ -23,7 +23,7 @@ import javax.annotation.Nullable;
  * A {@link HashJoinStateCalculator} is a piece of code that compute the memory requirements for one of the states
  * in the {@link HashJoinState} enum.
  */
-public interface HashJoinStateCalculator<T extends HashJoinStateCalculator> {
+public interface HashJoinStateCalculator<T extends HashJoinStateCalculator<?>> {
   /**
    * Signifies that the current state is complete and returns the next {@link HashJoinStateCalculator}.
    * Returns null in the case where there is no next state.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
index 503ff02..08b7c09 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
@@ -27,7 +27,6 @@ import org.apache.calcite.rel.core.JoinRelType;
  * Maintain join state.
  */
 public final class JoinStatus {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JoinStatus.class);
 
   public final RecordIterator left;
   public final RecordIterator right;
@@ -142,10 +141,6 @@ public final class JoinStatus {
    *  4. JoinOutcome.SCHEMA_CHANGED : one of the side has change in schema.
    */
   public JoinOutcome getOutcome() {
-    // on STOP, OUT_OF_MEMORY return FAILURE.
-    if (!ok || eitherMatches(IterOutcome.STOP)) {
-      return JoinOutcome.FAILURE;
-    }
     if (hasMoreData) {
       return JoinOutcome.BATCH_RETURNED;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
index c2a2b56..7e453c4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/LateralJoinBatch.java
@@ -56,7 +56,6 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA;
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 
 /**
  * RecordBatch implementation for the lateral join operator. Currently it's
@@ -352,7 +351,6 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
 
     // EMIT outcome is not expected as part of first batch from either side
     if (leftUpstream == EMIT || rightUpstream == EMIT) {
-      state = BatchState.STOP;
       throw new IllegalStateException("Unexpected IterOutcome.EMIT received either from left or right side in " +
         "buildSchema phase");
     }
@@ -392,14 +390,9 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    this.left.kill(sendUpstream);
-    // Reset the left side outcome as STOP since as part of right kill when UNNEST will ask IterOutcome of left incoming
-    // from LATERAL and based on that it can make decision if the kill is coming from downstream to LATERAL or upstream
-    // to LATERAL. Like LIMIT operator being present downstream to LATERAL or upstream to LATERAL and downstream to
-    // UNNEST.
-    leftUpstream = STOP;
-    this.right.kill(sendUpstream);
+  protected void cancelIncoming() {
+    left.cancel();
+    right.cancel();
   }
 
   /* ****************************************************************************************************************
@@ -417,7 +410,7 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
   }
 
   private boolean isTerminalOutcome(IterOutcome outcome) {
-    return (outcome == STOP || outcome == NONE);
+    return outcome == NONE;
   }
 
   /**
@@ -483,7 +476,6 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
           }
           break;
         case NONE:
-        case STOP:
           // Not using =0 since if outgoing container is empty then no point returning anything
           if (outputIndex > 0) { // can only reach here from produceOutputBatch
             processLeftBatchInFuture = true;
@@ -554,7 +546,6 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
           needNewRightBatch = false;
           break;
         case NONE:
-        case STOP:
           needNewRightBatch = false;
           break;
         case NOT_YET:
@@ -934,10 +925,7 @@ public class LateralJoinBatch extends AbstractBinaryRecordBatch<LateralJoinPOP>
 
   private boolean setBatchState(IterOutcome outcome) {
     switch(outcome) {
-      case STOP:
       case EMIT:
-        state = BatchState.STOP;
-        return false;
       case NONE:
       case NOT_YET:
         state = BatchState.DONE;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index 6eeacd8..f4ec939 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -238,8 +238,10 @@ public class MergeJoinBatch extends AbstractBinaryRecordBatch<MergeJoinPOP> {
         case FAILURE:
           status.left.clearInflightBatches();
           status.right.clearInflightBatches();
-          kill(false);
-          return IterOutcome.STOP;
+          // Should handle at the source of the error to provide a better error message.
+          throw UserException.executionError(null)
+              .message("Merge failed")
+              .build(logger);
         case NO_MORE_DATA:
           logger.debug("NO MORE DATA; returning {}",
             (status.getOutPosition() > 0 ? (first ? "OK_NEW_SCHEMA" : "OK") : (first ? "OK_NEW_SCHEMA" : "NONE")));
@@ -299,12 +301,6 @@ public class MergeJoinBatch extends AbstractBinaryRecordBatch<MergeJoinPOP> {
     rightIterator.close();
   }
 
-  @Override
-  protected void killIncoming(boolean sendUpstream) {
-    left.kill(sendUpstream);
-    right.kill(sendUpstream);
-  }
-
   private JoinWorker generateNewWorker() {
     final ClassGenerator<JoinWorker> cg = CodeGenerator.getRoot(JoinWorker.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
index 38f7aa0..a55ecd3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
@@ -177,8 +177,6 @@ public class NestedLoopJoinBatch extends AbstractBinaryRecordBatch<NestedLoopJoi
             addBatchToHyperContainer(right);
             break;
           case NONE:
-          case STOP:
-            //TODO we got a STOP, shouldn't we stop immediately ?
           case NOT_YET:
             drainRight = false;
             break;
@@ -211,7 +209,7 @@ public class NestedLoopJoinBatch extends AbstractBinaryRecordBatch<NestedLoopJoi
     if (!hasMore(rightUpstream)) {
       return;
     }
-    right.kill(true);
+    right.cancel();
     while (hasMore(rightUpstream)) {
       VectorAccessibleUtilities.clear(right);
       rightUpstream = next(HashJoinHelper.RIGHT_INPUT, right);
@@ -446,9 +444,9 @@ public class NestedLoopJoinBatch extends AbstractBinaryRecordBatch<NestedLoopJoi
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    this.left.kill(sendUpstream);
-    this.right.kill(sendUpstream);
+  protected void cancelIncoming() {
+    left.cancel();
+    right.cancel();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
index bc8c53b..3f48264 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
@@ -87,11 +87,6 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
 
     rightUpstream = next(right);
 
-    if (leftUpstream == IterOutcome.STOP || rightUpstream == IterOutcome.STOP) {
-      state = BatchState.STOP;
-      return;
-    }
-
     if (right.getRecordCount() > 0) {
       // set the hasRowKeyBatch flag such that calling next() on the left input
       // would see the correct status
@@ -138,7 +133,6 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
 
       switch(rightUpstream) {
       case NONE:
-      case STOP:
         rkJoinState = RowKeyJoinState.DONE;
         state = BatchState.DONE;
         return rightUpstream;
@@ -269,9 +263,9 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
   }
 
   @Override
-  public void killIncoming(boolean sendUpstream) {
-    left.kill(sendUpstream);
-    right.kill(sendUpstream);
+  protected void cancelIncoming() {
+    left.cancel();
+    right.cancel();
   }
 
   @Override
@@ -285,5 +279,4 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
     logger.error("RowKeyJoinBatch[container={}, left={}, right={}, hasRowKeyBatch={}, rkJoinState={}]",
         container, left, right, hasRowKeyBatch, rkJoinState);
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
index bb5b38d..f698096 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.record.AbstractSingleRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorAccessibleUtilities;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 
@@ -60,15 +61,13 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   public IterOutcome innerNext() {
     if (!first && !needMoreRecords(numberOfRecords)) {
       outgoingSv.setRecordCount(0);
-      incoming.kill(true);
-
+      incoming.cancel();
       IterOutcome upStream = next(incoming);
 
       while (upStream == IterOutcome.OK || upStream == IterOutcome.OK_NEW_SCHEMA) {
         // Clear the memory for the incoming batch
-        for (VectorWrapper<?> wrapper : incoming) {
-          wrapper.getValueVector().clear();
-        }
+        VectorAccessibleUtilities.clear(incoming);
+
         // clear memory for incoming sv (if any)
         if (incomingSv != null) {
           incomingSv.clear();
@@ -78,9 +77,7 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
       // If EMIT that means leaf operator is UNNEST, in this case refresh the limit states and return EMIT.
       if (upStream == EMIT) {
         // Clear the memory for the incoming batch
-        for (VectorWrapper<?> wrapper : incoming) {
-          wrapper.getValueVector().clear();
-        }
+        VectorAccessibleUtilities.clear(incoming);
 
         // clear memory for incoming sv (if any)
         if (incomingSv != null) {
@@ -139,15 +136,16 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
       container.buildSchema(BatchSchema.SelectionVectorMode.TWO_BYTE);
       return true;
     } else {
-     return false;
+      return false;
     }
   }
 
   /**
-   * Gets the outcome to return from super implementation and then in case of EMIT outcome it refreshes the state of
-   * operator. Refresh is done to again apply limit on all the future incoming batches which will be part of next
+   * Gets the outcome to return from super implementation and then in case of
+   * EMIT outcome it refreshes the state of operator. Refresh is done to again
+   * apply limit on all the future incoming batches which will be part of next
    * record boundary.
-   * @param hasRemainder
+   *
    * @return - IterOutcome to send downstream
    */
   @Override
@@ -201,10 +199,13 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   }
 
   /**
-   * limit call when incoming batch has number of records more than the start offset such that it can produce some
-   * output records. After first call of this method recordStartOffset should be 0 since we have already skipped the
+   * limit call when incoming batch has number of records more than the start
+   * offset such that it can produce some output records. After first call of
+   * this method recordStartOffset should be 0 since we have already skipped the
    * required number of records as part of first incoming record batch.
-   * @param inputRecordCount - number of records in incoming batch
+   *
+   * @param inputRecordCount
+   *          number of records in incoming batch
    */
   private void limit(int inputRecordCount) {
     int endRecordIndex;
@@ -240,8 +241,10 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   }
 
   /**
-   * Method which returns if more output records are needed from LIMIT operator. When numberOfRecords is set to
-   * {@link Integer#MIN_VALUE} that means there is no end bound on LIMIT, so get all the records past start offset.
+   * Method which returns if more output records are needed from LIMIT operator.
+   * When numberOfRecords is set to {@link Integer#MIN_VALUE} that means there
+   * is no end bound on LIMIT, so get all the records past start offset.
+   *
    * @return - true - more output records is expected.
    *           false - limit bound is reached and no more record is expected
    */
@@ -261,8 +264,9 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   }
 
   /**
-   * Reset the states for recordStartOffset and numberOfRecords based on the popConfig passed to the operator.
-   * This method is called for the outcome EMIT no matter if limit is reached or not.
+   * Reset the states for recordStartOffset and numberOfRecords based on the
+   * popConfig passed to the operator. This method is called for the outcome
+   * EMIT no matter if limit is reached or not.
    */
   private void refreshLimitState() {
     // Make sure startOffset is non-negative
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index ecac4ef..d25e4bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -531,12 +531,8 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
   }
 
   @Override
-  public void kill(final boolean sendUpstream) {
-    if (sendUpstream) {
-      informSenders();
-    } else {
-      close();
-    }
+  public void cancel() {
+    informSenders();
 
     for (final RawFragmentBatchProvider provider : fragProviders) {
       provider.kill(context);
@@ -587,9 +583,7 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
   }
 
   @Override
-  protected void killIncoming(final boolean sendUpstream) {
-    //No op
-  }
+  protected void cancelIncoming() { }
 
   private void checkSameSchemaAmongBatches(final RecordBatchLoader[] batchLoaders) {
     Preconditions.checkArgument(batchLoaders.length > 0, "0 batch is not allowed!");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
index 49dc42e..7314961 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
@@ -163,7 +163,6 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
           finishedLeft = !firstLeft;
           break outer;
         case NOT_YET:
-        case STOP:
           return outcome;
         case OK_NEW_SCHEMA:
           if (firstLeft) {
@@ -209,7 +208,6 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
           finishedRight = true;
           break outer;
         case NOT_YET:
-        case STOP:
           return outcome;
         case OK_NEW_SCHEMA:
           firstRight = false;
@@ -736,12 +734,6 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    left.kill(sendUpstream);
-    right.kill(sendUpstream);
-  }
-
-  @Override
   public void dump() {
     logger.error("MetadataHandlerBatch[container={}, popConfig={}]", container, popConfig);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
index 600a170..6c7d4df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
@@ -69,7 +69,6 @@ import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 
 /**
  * Responsible for handling metadata returned by incoming aggregate operators
@@ -125,7 +124,6 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
         doWorkInternal();
         // fall thru
       case NOT_YET:
-      case STOP:
         return outcome;
       default:
         throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
@@ -135,7 +133,7 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
   @Override
   public IterOutcome innerNext() {
     IterOutcome outcome = getLastKnownOutcome();
-    if (outcome != NONE && outcome != STOP) {
+    if (outcome != NONE) {
       outcome = super.innerNext();
     }
     // if incoming is exhausted, reads metadata which should be obtained from the Metastore
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionProjectorTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionProjectorTemplate.java
index 25af95a..e9e7bf4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionProjectorTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionProjectorTemplate.java
@@ -32,18 +32,16 @@ import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.vector.IntVector;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class OrderedPartitionProjectorTemplate implements OrderedPartitionProjector {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderedPartitionProjectorTemplate.class);
+  static final Logger logger = LoggerFactory.getLogger(OrderedPartitionProjectorTemplate.class);
 
   private ImmutableList<TransferPair> transfers;
-//  private VectorContainer partitionVectors;
   private int partitions;
-//  private SelectionVector2 vector2;
-//  private SelectionVector4 vector4;
   private SelectionVectorMode svMode;
   private RecordBatch outBatch;
-//  private SchemaPath outputField;
   private IntVector partitionValues;
 
   public OrderedPartitionProjectorTemplate() throws SchemaChangeException{
@@ -83,12 +81,12 @@ public abstract class OrderedPartitionProjectorTemplate implements OrderedPartit
 
     this.svMode = incoming.getSchema().getSelectionVectorMode();
     this.outBatch = outgoing;
-//    this.outputField = outputField;
     partitionValues = (IntVector) outBatch.getValueAccessorById(IntVector.class, outBatch.getValueVectorId(outputField).getFieldIds()).getValueVector();
     switch(svMode){
     case FOUR_BYTE:
     case TWO_BYTE:
       throw new UnsupportedOperationException("Selection vector not supported");
+    default:
     }
     this.transfers = ImmutableList.copyOf(transfers);
     this.partitions = partitions;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index f52554c..37b8826 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -135,6 +135,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   private final String mapKey;
   private List<VectorContainer> sampledIncomingBatches;
 
+  @SuppressWarnings("null")
   public OrderedPartitionRecordBatch(OrderedPartitionSender pop, RecordBatch incoming, FragmentContext context) throws OutOfMemoryException {
     super(pop, context);
     this.incoming = incoming;
@@ -186,7 +187,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
         switch (upstream) {
         case NONE:
         case NOT_YET:
-        case STOP:
           upstreamNone = true;
           break outer;
         default:
@@ -441,8 +441,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
@@ -504,7 +504,6 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     switch (upstream) {
     case NONE:
     case NOT_YET:
-    case STOP:
       close();
       recordCount = 0;
       return upstream;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/SampleCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/SampleCopierTemplate.java
index 4ce10c8..36baad3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/SampleCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/SampleCopierTemplate.java
@@ -23,12 +23,14 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class SampleCopierTemplate implements SampleCopier {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SampleCopierTemplate.class);
+  static final Logger logger = LoggerFactory.getLogger(SampleCopierTemplate.class);
 
   private SelectionVector4 sv4;
-  private int outputRecords = 0;
+  private int outputRecords;
 
   @Override
   public void setupCopier(FragmentContext context, SelectionVector4 sv4, VectorAccessible incoming, VectorAccessible outgoing)
@@ -42,10 +44,8 @@ public abstract class SampleCopierTemplate implements SampleCopier {
     return outputRecords;
   }
 
-
   @Override
   public boolean copyRecords(int skip, int start, int total) {
-    final int recordCount = sv4.getCount();
     int outgoingPosition = 0;
     int increment = skip > 0 ? skip : 1;
     for(int svIndex = start; svIndex < sv4.getCount() && outputRecords < total; svIndex += increment, outgoingPosition++){
@@ -60,7 +60,4 @@ public abstract class SampleCopierTemplate implements SampleCopier {
 
   public abstract void doSetup(@Named("context") FragmentContext context, @Named("incoming") VectorAccessible incoming, @Named("outgoing") VectorAccessible outgoing);
   public abstract boolean doEval(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex);
-
-
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 3b7f78a..d2eea15 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -22,6 +22,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerArray;
 
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -74,7 +75,7 @@ public class PartitionSenderRootExec extends BaseRootExec {
 
   private final AtomicIntegerArray remainingReceivers;
   private final AtomicInteger remaingReceiverCount;
-  private boolean done = false;
+  private boolean done;
   private boolean first = true;
   private final boolean closeIncoming;
 
@@ -153,7 +154,7 @@ public class PartitionSenderRootExec extends BaseRootExec {
     if (!done) {
       out = next(incoming);
     } else {
-      incoming.kill(true);
+      incoming.cancel();
       out = IterOutcome.NONE;
     }
 
@@ -171,15 +172,9 @@ public class PartitionSenderRootExec extends BaseRootExec {
             sendEmptyBatch(true);
           }
         } catch (ExecutionException e) {
-          incoming.kill(false);
-          logger.error("Error while creating partitioning sender or flushing outgoing batches", e);
-          context.getExecutorState().fail(e.getCause());
-        }
-        return false;
-
-      case STOP:
-        if (partitioner != null) {
-          partitioner.clear();
+          throw UserException.dataWriteError(e)
+            .addContext("Error while creating partitioning sender or flushing outgoing batches")
+            .build(logger);
         }
         return false;
 
@@ -198,18 +193,17 @@ public class PartitionSenderRootExec extends BaseRootExec {
             sendEmptyBatch(false);
           }
         } catch (ExecutionException e) {
-          incoming.kill(false);
-          logger.error("Error while flushing outgoing batches", e);
-          context.getExecutorState().fail(e.getCause());
-          return false;
+          throw UserException.dataWriteError(e)
+            .addContext("Error while flushing outgoing batches")
+            .build(logger);
         }
       case OK:
         try {
           partitioner.partitionBatch(incoming);
         } catch (ExecutionException e) {
-          context.getExecutorState().fail(e.getCause());
-          incoming.kill(false);
-          return false;
+          throw UserException.dataWriteError(e)
+            .addContext("Error while partitioning outgoing batches")
+            .build(logger);
         }
         VectorAccessibleUtilities.clear(incoming);
         return true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java
index 8c8cc54..5e1ffd4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/producer/ProducerConsumerBatch.java
@@ -50,7 +50,6 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
   private final BlockingDeque<RecordBatchDataWrapper> queue;
   private int recordCount;
   private BatchSchema schema;
-  private boolean stop;
   private final CountDownLatch cleanUpLatch = new CountDownLatch(1); // used to wait producer to clean up
 
   protected ProducerConsumerBatch(final ProducerConsumer popConfig, final FragmentContext context, final RecordBatch incoming) throws OutOfMemoryException {
@@ -77,8 +76,6 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
     }
     if (wrapper.finished) {
       return IterOutcome.NONE;
-    } else if (wrapper.failed) {
-      return IterOutcome.STOP;
     }
 
     recordCount = wrapper.batch.getRecordCount();
@@ -119,10 +116,8 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
 
     @Override
     public void run() {
+      boolean stop = false;
       try {
-        if (stop) {
-          return;
-        }
         outer:
         while (true) {
           final IterOutcome upstream = incoming.next();
@@ -130,9 +125,6 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
             case NONE:
               stop = true;
               break outer;
-            case STOP:
-              queue.putFirst(RecordBatchDataWrapper.failed());
-              return;
             case OK_NEW_SCHEMA:
             case OK:
               wrapper = RecordBatchDataWrapper.batch(new RecordBatchData(incoming, oContext.getAllocator()));
@@ -174,9 +166,7 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
   }
 
   @Override
-  protected void killIncoming(final boolean sendUpstream) {
-    stop = true;
-  }
+  protected void cancelIncoming() { }
 
   @Override
   public void close() {
@@ -207,30 +197,24 @@ public class ProducerConsumerBatch extends AbstractRecordBatch<ProducerConsumer>
   private static class RecordBatchDataWrapper {
     final RecordBatchData batch;
     final boolean finished;
-    final boolean failed;
 
-    RecordBatchDataWrapper(final RecordBatchData batch, final boolean finished, final boolean failed) {
+    RecordBatchDataWrapper(final RecordBatchData batch, final boolean finished) {
       this.batch = batch;
       this.finished = finished;
-      this.failed = failed;
     }
 
     public static RecordBatchDataWrapper batch(final RecordBatchData batch) {
-      return new RecordBatchDataWrapper(batch, false, false);
+      return new RecordBatchDataWrapper(batch, false);
     }
 
     public static RecordBatchDataWrapper finished() {
-      return new RecordBatchDataWrapper(null, true, false);
-    }
-
-    public static RecordBatchDataWrapper failed() {
-      return new RecordBatchDataWrapper(null, false, true);
+      return new RecordBatchDataWrapper(null, true);
     }
   }
 
   @Override
   public void dump() {
-    logger.error("ProducerConsumerBatch[container={}, recordCount={}, schema={}, stop={}]",
-        container, recordCount, schema, stop);
+    logger.error("ProducerConsumerBatch[container={}, recordCount={}, schema={}]",
+        container, recordCount, schema);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 5b2bdc5..a4613a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -68,8 +68,8 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    super.killIncoming(sendUpstream);
+  protected void cancelIncoming() {
+    super.cancelIncoming();
     hasRemainder = false;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
index b3ca591..9a96909 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/protocol/OperatorRecordBatch.java
@@ -138,7 +138,7 @@ public class OperatorRecordBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
+  public void cancel() {
     driver.cancel();
   }
 
@@ -164,11 +164,6 @@ public class OperatorRecordBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public boolean hasFailed() {
-    return lastOutcome == IterOutcome.STOP;
-  }
-
-  @Override
   public void dump() {
     logger.error("OperatorRecordBatch[batchAccessor={}, lastOutcome={}]", batchAccessor, lastOutcome);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
index 56c6246..5e859f7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/rangepartitioner/RangePartitionRecordBatch.java
@@ -69,11 +69,6 @@ public class RangePartitionRecordBatch extends AbstractSingleRecordBatch<RangePa
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
-  }
-
-  @Override
   public int getRecordCount() {
     return recordCount;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/RecordBatchData.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/RecordBatchData.java
index 4b2cfc8..6cc723c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/RecordBatchData.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/RecordBatchData.java
@@ -35,7 +35,6 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * Holds the data for a particular record batch for later manipulation.
  */
 public class RecordBatchData {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchData.class);
 
   private SelectionVector2 sv2;
   private int recordCount;
@@ -72,7 +71,7 @@ public class RecordBatchData {
 
   public List<ValueVector> getVectors() {
     List<ValueVector> vectors = Lists.newArrayList();
-    for (VectorWrapper w : container) {
+    for (VectorWrapper<?> w : container) {
       vectors.add(w.getValueVector());
     }
     return vectors;
@@ -98,5 +97,4 @@ public class RecordBatchData {
     }
     container.clear();
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
index 98f6327..747959f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
@@ -107,8 +107,6 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
         break outer;
       case NOT_YET:
         throw new UnsupportedOperationException();
-      case STOP:
-        return upstream;
       case OK_NEW_SCHEMA:
         // only change in the case that the schema truly changes.  Artificial schema changes are ignored.
         if (!incoming.getSchema().equals(schema)) {
@@ -205,8 +203,8 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
index 7a88dc9..967209b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
@@ -342,7 +342,6 @@ public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMe
         case NONE:
           break outer;
         case NOT_YET:
-        case STOP:
           return outcome;
         case OK_NEW_SCHEMA:
           if (first) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
index 63bcb86..2039e79 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
@@ -84,12 +84,6 @@ public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    left.kill(sendUpstream);
-    right.kill(sendUpstream);
-  }
-
-  @Override
   protected void buildSchema() {
     if (! prefetchFirstBatchFromBothSides()) {
       state = BatchState.DONE;
@@ -377,9 +371,6 @@ public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
               batchMemoryManager.getRecordBatchSizer(topStatus.inputIndex),
               getRecordBatchStatsContext());
             return Pair.of(outcome, topStatus);
-          case STOP:
-            batchStatusStack.pop();
-            return Pair.of(outcome, topStatus);
           case NONE:
             batchStatusStack.pop();
             if (batchStatusStack.isEmpty()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
index 65d66ce..5e6ef47 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unnest/UnnestRecordBatch.java
@@ -145,7 +145,7 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
+  protected void cancelIncoming() {
     //
     // In some cases we need to return a predetermined state from a call to next. These are:
     // 1) Kill is called due to an error occurring in the processing of the query. IterOutcome should be NONE
@@ -154,11 +154,7 @@ public class UnnestRecordBatch extends AbstractTableFunctionRecordBatch<UnnestPO
     // special handling is needed in that case.
     //
     Preconditions.checkNotNull(lateral);
-    // Do not call kill on incoming. Lateral Join has the responsibility for killing incoming
-    Preconditions.checkState(context.getExecutorState().isFailed() ||
-      lateral.getLeftOutcome() == IterOutcome.STOP, "Kill received by unnest with unexpected state. " +
-      "Neither the LateralOutcome is STOP nor executor state is failed");
-      logger.debug("Kill received. Stopping all processing");
+    logger.debug("Cancel received. Stopping all processing");
     state = BatchState.DONE;
     hasRemainder = false; // whatever the case, we need to stop processing the current row.
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
index b94c551..bfbad64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
@@ -118,10 +118,8 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    if (sendUpstream) {
-      informSenders();
-    }
+  public void cancel() {
+    informSenders();
     fragProvider.kill(context);
   }
 
@@ -293,9 +291,4 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
   public void dump() {
     logger.error("UnorderedReceiverBatch[batchLoader={}, schema={}]", batchLoader, schema);
   }
-
-  @Override
-  public boolean hasFailed() {
-    return lastOutcome == IterOutcome.STOP;
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
index 66e8b01..1fb44f5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
@@ -112,7 +112,6 @@ public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMap
     switch (upStream) {
       case NONE:
       case NOT_YET:
-      case STOP:
         return upStream;
       case OK_NEW_SCHEMA:
         first = false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
index 7e795dd..e857364 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.physical.impl.validate;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA;
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 
 import java.util.Iterator;
 
@@ -120,7 +119,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
     this(incoming, false);
   }
 
-
   public void enableBatchValidation(boolean option) {
     validateBatches = option;
   }
@@ -184,8 +182,8 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  public void cancel() {
+    incoming.cancel();
   }
 
   @Override
@@ -227,7 +225,7 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
                 instNum, batchTypeName, exceptionState, batchState));
       }
       // (Note:  This could use validationState.)
-      if ((!isRepeatable && batchState == NONE) || batchState == STOP) {
+      if (!isRepeatable && batchState == NONE) {
         throw new IllegalStateException(
             String.format(
                 "next() [on #%d, %s] called again after it returned %s."
@@ -271,12 +269,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
             validationState = ValidationState.TERMINAL;
           }
           break;
-        case STOP:
-          // STOP is allowed at any time, except if already terminated (checked
-          // above).
-          // STOP moves to terminal high-level state.
-          validationState = ValidationState.TERMINAL;
-          break;
         case NOT_YET:
           // NOT_YET is allowed at any time, except if
           // terminated (checked above).
@@ -286,7 +278,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
           throw new AssertionError(
               "Unhandled new " + IterOutcome.class.getSimpleName() + " value "
               + batchState);
-          //break;
       }
 
       // Validate schema when available.
@@ -385,11 +376,6 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
   public RecordBatch getIncoming() { return incoming; }
 
   @Override
-  public boolean hasFailed() {
-    return exceptionState != null || batchState == STOP;
-  }
-
-  @Override
   public void dump() {
     logger.error("IteratorValidatorBatchIterator[container={}, instNum={}, batchTypeName={}, lastSchema={}, "
            + "lastNewSchema={}]", getContainer(), instNum, batchTypeName, lastSchema, lastNewSchema);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java
index ba8c06b..5873779 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/window/WindowFrameRecordBatch.java
@@ -66,8 +66,6 @@ public class WindowFrameRecordBatch extends AbstractRecordBatch<WindowPOP> {
   private boolean noMoreBatches; // true when downstream returns NONE
   private BatchSchema schema;
 
-  private boolean shouldStop; // true if we received an early termination request
-
   public WindowFrameRecordBatch(WindowPOP popConfig, FragmentContext context,
       RecordBatch incoming) throws OutOfMemoryException {
     super(popConfig, context);
@@ -88,21 +86,6 @@ public class WindowFrameRecordBatch extends AbstractRecordBatch<WindowPOP> {
       return IterOutcome.NONE;
     }
 
-    if (shouldStop) {
-      if (!noMoreBatches) {
-        IterOutcome upstream = next(incoming);
-        while (upstream == IterOutcome.OK || upstream == IterOutcome.OK_NEW_SCHEMA) {
-          // Clear the memory for the incoming batch
-          for (VectorWrapper<?> wrapper : incoming) {
-            wrapper.getValueVector().clear();
-          }
-          upstream = next(incoming);
-        }
-      }
-
-      return IterOutcome.NONE;
-    }
-
     // keep saving incoming batches until the first unprocessed batch can be
     // processed, or upstream == NONE
     while (!noMoreBatches && !canDoWork()) {
@@ -114,7 +97,6 @@ public class WindowFrameRecordBatch extends AbstractRecordBatch<WindowPOP> {
           noMoreBatches = true;
           break;
         case NOT_YET:
-        case STOP:
           cleanup();
           return upstream;
         case OK_NEW_SCHEMA:
@@ -228,11 +210,7 @@ public class WindowFrameRecordBatch extends AbstractRecordBatch<WindowPOP> {
       state = BatchState.DONE;
       container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
       return;
-    case STOP:
-      state = BatchState.STOP;
-      return;
     default:
-      break;
     }
 
     try {
@@ -405,9 +383,8 @@ public class WindowFrameRecordBatch extends AbstractRecordBatch<WindowPOP> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    shouldStop = true;
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 17a634d..42ea36c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -21,7 +21,6 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.EMIT;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK;
 import static org.apache.drill.exec.record.RecordBatch.IterOutcome.OK_NEW_SCHEMA;
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
@@ -333,9 +332,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       container.buildSchema(SelectionVectorMode.NONE);
       container.setRecordCount(0);
       break;
-    case STOP:
-      state = BatchState.STOP;
-      break;
     case NONE:
       state = BatchState.DONE;
       break;
@@ -408,10 +404,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         // all batches have been read at this record boundary
         break loop;
 
-      case STOP:
-        // Something went wrong.
-        return STOP;
-
       default:
         break;
       }
@@ -450,7 +442,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     }
     switch (lastKnownOutcome) {
     case NONE:
-    case STOP:
       return lastKnownOutcome;
 
     case OK_NEW_SCHEMA:
@@ -499,8 +490,8 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
   }
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    incoming.cancel();
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MergeSortWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MergeSortWrapper.java
index 01c0f0b..104e3fe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MergeSortWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/MergeSortWrapper.java
@@ -266,7 +266,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
     } else {
       int index = 0;
       for (VectorWrapper<?> w : inputDataContainer) {
-        HyperVectorWrapper wrapper = (HyperVectorWrapper<?>) container.getValueVector(index++);
+        HyperVectorWrapper<?> wrapper = (HyperVectorWrapper<?>) container.getValueVector(index++);
         wrapper.updateVectorList(w.getValueVectors());
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
index b2b4d4c..2e5881b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/TupleState.java
@@ -631,6 +631,7 @@ public abstract class TupleState extends ContainerState
       super(events, vectorCache, projectionSet);
     }
 
+    @Override
     public void bindColumnState(ColumnState colState) {
       super.bindColumnState(colState);
       writer().bindListener(this);
@@ -701,6 +702,7 @@ public abstract class TupleState extends ContainerState
       this.offsets = offsets;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public T vector() {
       return vector;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index ee10a15..de8f499 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -31,15 +31,17 @@ import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.PositiveLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.drill.exec.server.options.TypeValidators.MinRangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.MaxRangeDoubleValidator;
 import org.apache.calcite.plan.Context;
 
 public class PlannerSettings implements Context{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlannerSettings.class);
+  private static final Logger logger = LoggerFactory.getLogger(PlannerSettings.class);
 
-  private int numEndPoints = 0;
-  private boolean useDefaultCosting = false; // True: use default Optiq costing, False: use Drill costing
+  private int numEndPoints;
+  private boolean useDefaultCosting; // True: use default Optiq costing, False: use Drill costing
   private boolean forceSingleMode;
 
   public static final int MAX_BROADCAST_THRESHOLD = Integer.MAX_VALUE;
@@ -48,21 +50,26 @@ public class PlannerSettings implements Context{
   // initial off heap memory allocation (1M)
   private static final long INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES = 1024 * 1024;
   // default off heap memory for planning (256M)
+  @SuppressWarnings("unused")
   private static final long DEFAULT_MAX_OFF_HEAP_ALLOCATION_IN_BYTES = 256 * 1024 * 1024;
   // max off heap memory for planning (16G)
   private static final long MAX_OFF_HEAP_ALLOCATION_IN_BYTES = 16l * 1024 * 1024 * 1024;
 
   public static final OptionValidator CONSTANT_FOLDING = new BooleanValidator("planner.enable_constant_folding",
       new OptionDescription("If one side of a filter condition is a constant expression, constant folding evaluates the expression in the planning phase and replaces the expression with the constant value. For example, Drill can rewrite WHERE age + 5 < 42 as WHERE age < 37."));
-  public static final OptionValidator EXCHANGE = new BooleanValidator("planner.disable_exchanges",
+  public static final String DISABLE_EXCHANGE_OPTION = "planner.disable_exchanges";
+  public static final OptionValidator EXCHANGE = new BooleanValidator(DISABLE_EXCHANGE_OPTION,
       new OptionDescription("Toggles the state of hashing to a random exchange."));
-  public static final OptionValidator HASHAGG = new BooleanValidator("planner.enable_hashagg",
+  public static final String ENABLE_HASH_AGG_OPTION = "planner.enable_hashagg";
+  public static final OptionValidator HASHAGG = new BooleanValidator(ENABLE_HASH_AGG_OPTION,
       new OptionDescription("Enable hash aggregation; otherwise, Drill does a sort-based aggregation. Writes to disk. Enable is recommended."));
-  public static final OptionValidator STREAMAGG = new BooleanValidator("planner.enable_streamagg",
+  public static final String ENABLE_STREAM_AGG_OPTION = "planner.enable_streamagg";
+  public static final OptionValidator STREAMAGG = new BooleanValidator(ENABLE_STREAM_AGG_OPTION,
       new OptionDescription("Sort-based operation. Writes to disk."));
   public static final OptionValidator TOPN = new BooleanValidator("planner.enable_topn",
       new OptionDescription("Generates the topN plan for queries with the ORDER BY and LIMIT clauses."));
-  public static final OptionValidator HASHJOIN = new BooleanValidator("planner.enable_hashjoin",
+  public static final String ENABLE_HASH_JOIN_OPTION = "planner.enable_hashjoin";
+  public static final OptionValidator HASHJOIN = new BooleanValidator(ENABLE_HASH_JOIN_OPTION,
       new OptionDescription("Enable the memory hungry hash join. Drill assumes that a query will have adequate memory to complete and tries to use the fastest operations possible to complete the planned inner, left, right, or full outer joins using a hash table. Does not write to disk. Disabling hash join allows Drill to manage arbitrarily large data in a small memory footprint."));
   public static final OptionValidator SEMIJOIN = new BooleanValidator("planner.enable_semijoin",
           new OptionDescription("Enable the semi join optimization. Planner removes the distinct processing below the hash join and sets the semi join flag in hash join."));
@@ -481,6 +488,7 @@ public class PlannerSettings implements Context{
     return options.getOption(STATISTICS_MULTICOL_NDV_ADJUST_FACTOR);
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public <T> T unwrap(Class<T> clazz) {
     if(clazz == PlannerSettings.class){
@@ -489,6 +497,4 @@ public class PlannerSettings implements Context{
       return null;
     }
   }
-
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
index 49cbb46..c426dd2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
@@ -74,10 +74,6 @@ public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> exte
   }
 
   protected boolean verifyOutcomeToSetBatchState(IterOutcome leftOutcome, IterOutcome rightOutcome) {
-    if (leftOutcome == IterOutcome.STOP || rightOutcome == IterOutcome.STOP) {
-      state = BatchState.STOP;
-      return false;
-    }
 
     if (checkForEarlyFinish(leftOutcome, rightOutcome)) {
       state = BatchState.DONE;
@@ -86,7 +82,6 @@ public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> exte
 
     // EMIT outcome is not expected as part of first batch from either side
     if (leftOutcome == IterOutcome.EMIT || rightOutcome == IterOutcome.EMIT) {
-      state = BatchState.STOP;
       throw new IllegalStateException("Unexpected IterOutcome.EMIT received either from left or right side in " +
         "buildSchema phase");
     }
@@ -135,4 +130,10 @@ public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> exte
     stats.setLongStat(Metric.AVG_OUTPUT_ROW_BYTES, batchMemoryManager.getAvgOutputRowWidth());
     stats.setLongStat(Metric.OUTPUT_RECORD_COUNT, batchMemoryManager.getTotalOutputRecords());
   }
+
+  @Override
+  protected void cancelIncoming() {
+    left.cancel();
+    right.cancel();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index 3a54c5f..6e36303 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -88,8 +88,6 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     FIRST,
     /** The first data batch has already been returned. */
     NOT_FIRST,
-    /** The query most likely failed, we need to propagate STOP to the root. */
-    STOP,
     /** All work is done, no more data to be sent. */
     DONE
   }
@@ -141,7 +139,6 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
         logger.debug("Number of records in received batch: {}", b.getRecordCount());
         break;
       default:
-        break;
     }
 
     return next;
@@ -158,9 +155,6 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
             case DONE:
               lastOutcome = IterOutcome.NONE;
               break;
-            case STOP:
-              lastOutcome = IterOutcome.STOP;
-              break;
             default:
               state = BatchState.FIRST;
               lastOutcome = IterOutcome.OK_NEW_SCHEMA;
@@ -196,11 +190,11 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
   protected void buildSchema() { }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    killIncoming(sendUpstream);
+  public void cancel() {
+    cancelIncoming();
   }
 
-  protected abstract void killIncoming(boolean sendUpstream);
+  protected abstract void cancelIncoming();
 
   @Override
   public void close() {
@@ -244,11 +238,6 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     return container;
   }
 
-  @Override
-  public boolean hasFailed() {
-    return lastOutcome == IterOutcome.STOP;
-  }
-
   public RecordBatchStatsContext getRecordBatchStatsContext() {
     return batchStatsContext;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
index 5997a34..926ae3f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractUnaryRecordBatch.java
@@ -44,9 +44,8 @@ public abstract class AbstractUnaryRecordBatch<T extends PhysicalOperator> exten
   protected abstract RecordBatch getIncoming();
 
   @Override
-  protected void killIncoming(boolean sendUpstream) {
-    final RecordBatch incoming = getIncoming();
-    incoming.kill(sendUpstream);
+  protected void cancelIncoming() {
+    getIncoming().cancel();
   }
 
   @Override
@@ -80,7 +79,6 @@ public abstract class AbstractUnaryRecordBatch<T extends PhysicalOperator> exten
         }
         return upstream;
       case NOT_YET:
-      case STOP:
         if (state == BatchState.FIRST) {
           container.buildSchema(SelectionVectorMode.NONE);
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
index 16c6383..f882731 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
@@ -41,6 +41,7 @@ public class MajorTypeSerDe {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MajorTypeSerDe.class);
 
 
+  @SuppressWarnings("serial")
   public static class De extends StdDeserializer<MajorType> {
 
     public De() {
@@ -55,6 +56,7 @@ public class MajorTypeSerDe {
 
   }
 
+  @SuppressWarnings("serial")
   public static class Se extends StdSerializer<MajorType> {
 
     public Se() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index bdb8341..5306dce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -86,9 +86,10 @@ public interface RecordBatch extends VectorAccessible {
    * </p>
    * <p>
    *   For abnormal termination, the sequence is truncated (before the
-   *   {@code NONE}) and ends with {@link #STOP}.  That is, the sequence begins
+   *   {@code NONE}) and ends with an exception.  That is, the sequence begins
    *   with a subsequence that is some prefix of a normal-completion sequence
-   *   and that does not contain {@code NONE}, and ends with {@code STOP}.
+   *   and that does not contain {@code NONE}, and ends with the
+   *   caller throwing a (query-fatal) exception.
    * </p>
    * <p>
    *   The normal-completion return sequence is matched by the following
@@ -97,14 +98,17 @@ public interface RecordBatch extends VectorAccessible {
    *     ( NOT_YET*  OK_NEW_SCHEMA
    *       NOT_YET*  OK )*
    *     )+
-   *     NOT_YET*    NONE</pre>
+   *     NOT_YET*    &lt;exception></pre>
    * </p>
    * <h4>Obsolete Outcomes</h4>
    *
-   * The former <tt>OUT_OF_MEMORY</tt> state was never really used.
+   * The former {@code OUT_OF_MEMORY} state was never really used.
    * It is now handled by calling
    * {@link FragmentContext#requestMemory()}
    * at the point that the operator realizes it is short on memory.
+   * <p>
+   * The former {@code STOP} state was replaced with a "fail fast"
+   * approach that throws an exception when an error is detected.
    */
   enum IterOutcome {
     /**
@@ -162,21 +166,6 @@ public interface RecordBatch extends VectorAccessible {
     OK_NEW_SCHEMA(false),
 
     /**
-     * Non-completion (abnormal) termination.
-     * <p>
-     *   The call to {@link #next()}
-     *   reports that the query has terminated other than by normal completion,
-     *   and that the caller must not call any of the schema-access or
-     *   data-access methods nor call {@code next()} again.
-     * </p>
-     * <p>
-     *   The caller can consume its QueryContext to understand the current state
-     *   of things.
-     * </p>
-     */
-    STOP(true),
-
-    /**
      * No data yet.
      * <p>
      *   The call to {@link #next()}
@@ -255,10 +244,17 @@ public interface RecordBatch extends VectorAccessible {
   BatchSchema getSchema();
 
   /**
-   * Informs child nodes that this query should be terminated.  Child nodes
-   * should use the QueryContext to determine what has happened.
+   * Informs child operators that no more data is needed. Only called
+   * for "normal" cancellation to avoid unnecessary compute in any worker
+   * threads. For the error case, the fragment
+   * executor will call close() on each child automatically.
+   * <p>
+   * The operator which triggers the cancel MUST send a <code>NONE</code>
+   * status downstream, or throw an exception. It is not legal to
+   * call <code>next()</code> on an operator after calling its
+   * <code>cancel()</code> method.
    */
-  void kill(boolean sendUpstream);
+  void cancel();
 
   VectorContainer getOutgoingContainer();
 
@@ -273,7 +269,7 @@ public interface RecordBatch extends VectorAccessible {
    * Gets the value vector type and ID for the given schema path.  The
    * TypedFieldId should store a fieldId which is the same as the ordinal
    * position of the field within the Iterator provided this class's
-   * implementation of Iterable<ValueVector>.
+   * implementation of {@code Iterable<ValueVector>}.
    *
    * @param path
    *          The path where the vector should be located.
@@ -315,14 +311,4 @@ public interface RecordBatch extends VectorAccessible {
    * Perform dump of this batch's state to logs.
    */
   void dump();
-
-  /**
-   * Use this method to see if the batch has failed. Currently used when logging {@code RecordBatch}'s
-   * state using {@link #dump()} method.
-   *
-   * @return {@code true} if either {@link org.apache.drill.exec.record.RecordBatch.IterOutcome#STOP}
-   * was returned by its or child's {@link #next()} invocation or there was an {@code Exception} thrown
-   * during execution of the batch; {@code false} otherwise
-   */
-  boolean hasFailed();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchMemoryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchMemoryManager.java
index 9bd905f..29f4fc8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchMemoryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchMemoryManager.java
@@ -35,10 +35,10 @@ public class RecordBatchMemoryManager {
   // the incoming rows become wide, then less (than planned) would fit into the remaining current allocated memory)
   private int currentOutgoingMaxRowCount = MAX_NUM_ROWS;
   private int outgoingRowWidth;
-  private int outputBatchSize;
-  private RecordBatchSizer[] sizer;
-  private BatchStats[] inputBatchStats;
-  private BatchStats outputBatchStats;
+  private final int outputBatchSize;
+  private final RecordBatchSizer[] sizer;
+  private final BatchStats[] inputBatchStats;
+  private final BatchStats outputBatchStats;
 
   // By default, we expect one input batch stream and one output batch stream.
   // Some operators can get multiple input batch streams i.e. for example
@@ -323,7 +323,7 @@ public class RecordBatchMemoryManager {
   public void allocateVectors(VectorContainer container, int recordCount) {
     // Allocate memory for the vectors.
     // This will iteratively allocate memory for all nested columns underneath.
-    for (VectorWrapper w : container) {
+    for (VectorWrapper<?> w : container) {
       RecordBatchSizer.ColumnSize colSize = getColumnSize(w.getField().getName());
       colSize.allocateVector(w.getValueVector(), recordCount);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
index 3db9814..4f5f02c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
@@ -145,7 +145,7 @@ public class RecordBatchSizer {
     /**
      * Child columns if this is a map column.
      */
-    private Map<String, ColumnSize> children = CaseInsensitiveMap.newHashMap();
+    private final Map<String, ColumnSize> children = CaseInsensitiveMap.newHashMap();
 
     /**
      * Returns true if there is an accurate std size. Otherwise it returns false.
@@ -249,6 +249,7 @@ public class RecordBatchSizer {
           case VARDECIMAL:
             stdNetSize = 4 + 8;
             break;
+          default:
         }
       } catch (Exception e) {
         stdNetSize = 0;
@@ -638,17 +639,17 @@ public class RecordBatchSizer {
 
   // This keeps information for only top level columns. Information for nested
   // columns can be obtained from children of topColumns.
-  private Map<String, ColumnSize> columnSizes = new QuoteInsensitiveMap(CaseInsensitiveMap.newHashMap());
+  private final Map<String, ColumnSize> columnSizes = new QuoteInsensitiveMap(CaseInsensitiveMap.newHashMap());
 
   /**
    * This field is used by the convenience method {@link #columnsList()}.
    */
-  private List<ColumnSize> columnSizesList = new ArrayList<>();
+  private final List<ColumnSize> columnSizesList = new ArrayList<>();
 
   /**
    * Number of records (rows) in the batch.
    */
-  private int rowCount;
+  private final int rowCount;
   /**
    * Actual batch size summing all buffers used to store data
    * for the batch.
@@ -677,7 +678,7 @@ public class RecordBatchSizer {
 
   private int avgDensity;
 
-  private Set<BufferLedger> ledgers = Sets.newIdentityHashSet();
+  private final Set<BufferLedger> ledgers = Sets.newIdentityHashSet();
 
   private long netBatchSize;
 
@@ -963,7 +964,7 @@ public class RecordBatchSizer {
   }
 
   public void allocateVectors(VectorContainer container, int recordCount) {
-    for (VectorWrapper w : container) {
+    for (VectorWrapper<?> w : container) {
       ColumnSize colSize = columnSizes.get(w.getField().getName());
       colSize.allocateVector(w.getValueVector(), recordCount);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordIterator.java
index 91c1b5c..6d4e745 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordIterator.java
@@ -191,7 +191,6 @@ public class RecordIterator implements VectorAccessible {
       nextBatch();
       switch (lastOutcome) {
         case NONE:
-        case STOP:
           // No more data, disallow reads unless reset is called.
           outerPosition = nextOuterPosition;
           lastBatchRead = true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemalessBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemalessBatch.java
index e4278ba..d41bffa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemalessBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemalessBatch.java
@@ -29,11 +29,8 @@ import java.util.Iterator;
  * Empty batch without schema and data.
  */
 public class SchemalessBatch implements CloseableRecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemalessBatch.class);
 
-  public SchemalessBatch() {
-    logger.debug("Empty schemaless batch is created");
-  }
+  public SchemalessBatch() { }
 
   @Override
   public FragmentContext getContext() {
@@ -63,8 +60,7 @@ public class SchemalessBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-  }
+  public void cancel() { }
 
   @Override
   public VectorContainer getOutgoingContainer() {
@@ -109,12 +105,5 @@ public class SchemalessBatch implements CloseableRecordBatch {
   public VectorContainer getContainer() { return null; }
 
   @Override
-  public boolean hasFailed() {
-    return false;
-  }
-
-  @Override
-  public void dump() {
-    logger.error("SchemalessBatch[]");
-  }
+  public void dump() { }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleRecordBatch.java
index 09200f2..bcd8bea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SimpleRecordBatch.java
@@ -55,8 +55,7 @@ public class SimpleRecordBatch implements RecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-  }
+  public void cancel() { }
 
   @Override
   public SelectionVector2 getSelectionVector2() {
@@ -107,9 +106,4 @@ public class SimpleRecordBatch implements RecordBatch {
   public void dump() {
     logger.error("SimpleRecordBatch[container=" + container + "]");
   }
-
-  @Override
-  public boolean hasFailed() {
-    return false;
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessibleComplexWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessibleComplexWriter.java
index d989ddc..1100cea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessibleComplexWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorAccessibleComplexWriter.java
@@ -24,7 +24,6 @@ import org.apache.drill.exec.vector.complex.impl.ComplexWriterImpl;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
 
 public class VectorAccessibleComplexWriter extends MapVector {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorAccessibleComplexWriter.class);
 
   private final VectorContainer vc;
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java b/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
index fcdd32b..e32c97e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestCorrelation.java
@@ -24,7 +24,6 @@ import org.junit.experimental.categories.Category;
 
 @Category({SqlTest.class, OperatorTest.class})
 public class TestCorrelation extends PlanTestBase {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestCorrelation.class);
 
   @Test  // DRILL-2962
   public void testScalarAggCorrelatedSubquery() throws Exception {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
index a2d0ef6..5273e98 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
@@ -29,6 +29,7 @@ import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -172,15 +173,18 @@ public class TestExampleQueries extends BaseTestQuery {
 
   @Test
   public void testJoinMerge() throws Exception {
-    test("alter session set `planner.enable_hashjoin` = false");
-    test("select count(*) \n" +
-        "  from (select l.l_orderkey as x, c.c_custkey as y \n" +
-        "  from cp.`tpch/lineitem.parquet` l \n" +
-        "    left outer join cp.`tpch/customer.parquet` c \n" +
-        "      on l.l_orderkey = c.c_custkey) as foo\n" +
-        "  where x < 10000\n" +
-        "");
-    test("alter session set `planner.enable_hashjoin` = true");
+    alterSession(PlannerSettings.ENABLE_HASH_JOIN_OPTION, false);
+    try {
+      test("select count(*) \n" +
+          "  from (select l.l_orderkey as x, c.c_custkey as y \n" +
+          "  from cp.`tpch/lineitem.parquet` l \n" +
+          "    left outer join cp.`tpch/customer.parquet` c \n" +
+          "      on l.l_orderkey = c.c_custkey) as foo\n" +
+          "  where x < 10000\n" +
+          "");
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_HASH_JOIN_OPTION);
+    }
   }
 
   @Test
@@ -293,15 +297,19 @@ public class TestExampleQueries extends BaseTestQuery {
   @Test
   @Ignore("DRILL-3004")
   public void testJoin() throws Exception {
-    test("alter session set `planner.enable_hashjoin` = false");
-    test("SELECT\n" +
-        "  nations.N_NAME,\n" +
-        "  regions.R_NAME\n" +
-        "FROM\n" +
-        "  cp.`tpch/nation.parquet` nations\n" +
-        "JOIN\n" +
-        "  cp.`tpch/region.parquet` regions\n" +
-        "  on nations.N_REGIONKEY = regions.R_REGIONKEY where 1 = 0");
+    try {
+      alterSession(PlannerSettings.ENABLE_HASH_JOIN_OPTION, false);
+      test("SELECT\n" +
+          "  nations.N_NAME,\n" +
+          "  regions.R_NAME\n" +
+          "FROM\n" +
+          "  cp.`tpch/nation.parquet` nations\n" +
+          "JOIN\n" +
+          "  cp.`tpch/region.parquet` regions\n" +
+          "  on nations.N_REGIONKEY = regions.R_REGIONKEY where 1 = 0");
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_HASH_JOIN_OPTION);
+    }
   }
 
 
@@ -509,7 +517,12 @@ public class TestExampleQueries extends BaseTestQuery {
   public void testOrderByDiffColumnsInSubqAndOuter() throws Exception {
     String query = "select n.n_nationkey from  (select n_nationkey, n_regionkey from cp.`tpch/nation.parquet` order by n_regionkey) n  order by n.n_nationkey";
     // set slice_target = 1 to force exchanges
-    test("alter session set `planner.slice_target` = 1; " + query);
+    try {
+      alterSession(ExecConstants.SLICE_TARGET, 1);
+      test(query);
+    } finally {
+      resetSessionOption(ExecConstants.SLICE_TARGET);
+    }
   }
 
   @Test // DRILL-1846  (this tests issue with UnionExchange)
@@ -517,7 +530,12 @@ public class TestExampleQueries extends BaseTestQuery {
   public void testLimitInSubqAndOrderByOuter() throws Exception {
     String query = "select t2.n_nationkey from (select n_nationkey, n_regionkey from cp.`tpch/nation.parquet` t1 group by n_nationkey, n_regionkey limit 10) t2 order by t2.n_nationkey";
     // set slice_target = 1 to force exchanges
-    test("alter session set `planner.slice_target` = 1; " + query);
+    try {
+      alterSession(ExecConstants.SLICE_TARGET, 1);
+      test(query);
+    } finally {
+      resetSessionOption(ExecConstants.SLICE_TARGET);
+    }
   }
 
   @Test // DRILL-1788
@@ -531,7 +549,12 @@ public class TestExampleQueries extends BaseTestQuery {
   public void test2PhaseAggAfterOrderBy() throws Exception {
     String query = "select count(*) from (select o_custkey from cp.`tpch/orders.parquet` order by o_custkey)";
     // set slice_target = 1 to force exchanges and 2-phase aggregation
-    test("alter session set `planner.slice_target` = 1; " + query);
+    try {
+      alterSession(ExecConstants.SLICE_TARGET, 1);
+      test(query);
+    } finally {
+      resetSessionOption(ExecConstants.SLICE_TARGET);
+    }
   }
 
   @Test // DRILL-1867
@@ -578,18 +601,27 @@ public class TestExampleQueries extends BaseTestQuery {
   @Test
   public void testMultipleCountDistinctWithGroupBy() throws Exception {
     String query = "select n_regionkey, count(distinct n_nationkey), count(distinct n_name) from cp.`tpch/nation.parquet` group by n_regionkey;";
-    String hashagg_only = "alter session set `planner.enable_hashagg` = true; " +
-        "alter session set `planner.enable_streamagg` = false;";
-    String streamagg_only = "alter session set `planner.enable_hashagg` = false; " +
-        "alter session set `planner.enable_streamagg` = true;";
 
     // hash agg and streaming agg with default slice target (single phase aggregate)
-    test(hashagg_only + query);
-    test(streamagg_only + query);
-
     // hash agg and streaming agg with lower slice target (multiphase aggregate)
-    test("alter session set `planner.slice_target` = 1; " + hashagg_only + query);
-    test("alter session set `planner.slice_target` = 1; " + streamagg_only + query);
+    try {
+      alterSession(PlannerSettings.ENABLE_HASH_AGG_OPTION, true);
+      alterSession(PlannerSettings.ENABLE_STREAM_AGG_OPTION, false);
+      test(query);
+      alterSession(ExecConstants.SLICE_TARGET, 1);
+      test(query);
+      resetSessionOption(ExecConstants.SLICE_TARGET);
+
+      alterSession(PlannerSettings.ENABLE_HASH_AGG_OPTION, false);
+      alterSession(PlannerSettings.ENABLE_STREAM_AGG_OPTION, true);
+      test(query);
+      alterSession(ExecConstants.SLICE_TARGET, 1);
+      test(query);
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_HASH_AGG_OPTION);
+      resetSessionOption(PlannerSettings.ENABLE_STREAM_AGG_OPTION);
+      resetSessionOption(ExecConstants.SLICE_TARGET);
+    }
   }
 
   @Test // DRILL-2019
@@ -826,12 +858,15 @@ public class TestExampleQueries extends BaseTestQuery {
 
   @Test // DRILL-2221
   public void createJsonWithEmptyList() throws Exception {
-    final String tableName = "jsonWithEmptyList";
-    test("USE dfs.tmp");
-    test("ALTER SESSION SET `store.format`='json'");
-    test("CREATE TABLE %s AS SELECT * FROM cp.`store/json/record_with_empty_list.json`", tableName);
-    test("SELECT COUNT(*) FROM %s", tableName);
-    test("ALTER SESSION SET `store.format`='parquet'");
+    try {
+      final String tableName = "jsonWithEmptyList";
+      test("USE dfs.tmp");
+      alterSession(ExecConstants.OUTPUT_FORMAT_OPTION, "json");
+      test("CREATE TABLE %s AS SELECT * FROM cp.`store/json/record_with_empty_list.json`", tableName);
+      test("SELECT COUNT(*) FROM %s", tableName);
+    } finally {
+      resetSessionOption(ExecConstants.OUTPUT_FORMAT_OPTION);
+    }
   }
 
   @Test // DRILL-2914
@@ -993,13 +1028,18 @@ public class TestExampleQueries extends BaseTestQuery {
         "where 1 = 0";
 
 
-    testBuilder()
-        .sqlQuery(query)
-        .expectsEmptyResultSet()
-        .optionSettingQueriesForTestQuery("ALTER SESSION SET `planner.enable_hashjoin` = false; " +
-            "ALTER SESSION SET `planner.disable_exchanges` = true")
-        .build()
-        .run();
+    try {
+      setSessionOption(PlannerSettings.DISABLE_EXCHANGE_OPTION, true);
+      setSessionOption(PlannerSettings.ENABLE_HASH_JOIN_OPTION, false);
+      testBuilder()
+          .sqlQuery(query)
+          .expectsEmptyResultSet()
+          .build()
+          .run();
+    } finally {
+      resetSessionOption(PlannerSettings.DISABLE_EXCHANGE_OPTION);
+      resetSessionOption(PlannerSettings.ENABLE_HASH_JOIN_OPTION);
+    }
   }
 
   @Test
@@ -1031,8 +1071,6 @@ public class TestExampleQueries extends BaseTestQuery {
         "create table mytable4  partition by (r_regionkey, r_comment) as " +
         "  select  r.* from cp.`tpch/nation.parquet` n, cp.`tpch/region.parquet` r " +
         "  where n.n_regionkey = r.r_regionkey");
-
-
   }
 
   @Test // DRILL-3210
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
index e84bb27..4717993 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
@@ -25,7 +25,6 @@ import org.junit.experimental.categories.Category;
 
 @Category({SlowTest.class})
 public class TestTpchLimit0 extends BaseTestQuery {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchLimit0.class);
 
   private void testLimitZero(String fileName) throws Exception {
     String query = getFile(fileName);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSeparatePlanningTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSeparatePlanningTest.java
index a4d10b6..90e9e00 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSeparatePlanningTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSeparatePlanningTest.java
@@ -47,7 +47,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /**
  * Class to test different planning use cases (separate from query execution)
- *
+ * (Though, despite the above, this test does execute queries.)
  */
 @Category({SlowTest.class, PlannerTest.class})
 public class DrillSeparatePlanningTest extends ClusterTest {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
index 054cdee..6e1ce9b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
@@ -160,14 +160,9 @@ public class MockRecordBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
+  public void cancel() {
     if (!limitWithUnnest) {
       isDone = true;
-      container.clear();
-      container.setRecordCount(0);
-      if (sv2 != null) {
-        sv2.clear();
-      }
     }
   }
 
@@ -254,7 +249,6 @@ public class MockRecordBatch implements CloseableRecordBatch {
         ++currentContainerIndex;
         return currentOutcome;
       case NONE:
-      case STOP:
         isDone = true;
       case NOT_YET:
         container.setRecordCount(0);
@@ -286,11 +280,6 @@ public class MockRecordBatch implements CloseableRecordBatch {
   }
 
   @Override
-  public boolean hasFailed() {
-    return false;
-  }
-
-  @Override
   public void dump() { }
 
   public static class Builder {
@@ -324,8 +313,6 @@ public class MockRecordBatch implements CloseableRecordBatch {
     }
 
     public Builder terminateWithError(IterOutcome errorOutcome) {
-      Preconditions.checkArgument(errorOutcome != IterOutcome.STOP);
-
       iterOutcomes.add(errorOutcome);
       return this;
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index 8f0d677..eb48f08 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -105,7 +105,6 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector> {
   public boolean next() {
     switch (incoming.next()) {
     case NONE:
-    case STOP:
       return false;
     default:
       return true;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
index 335c5e8..17f529f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
@@ -88,6 +88,7 @@ public class TestBroadcastExchange extends PopUnitTestBase {
         }
         b.release();
       }
+      // Nothing done with count?
     }
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
index ae2a0db..9c2cff8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
@@ -62,8 +62,8 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
   private static BatchSchema batchSchemaLeft;
 
   // Input containers -- where row count is not set for the 2nd container !!
-  private List<VectorContainer> uninitialized2ndInputContainersRight = new ArrayList<>(5);
-  private List<VectorContainer> uninitialized2ndInputContainersLeft = new ArrayList<>(5);
+  private final List<VectorContainer> uninitialized2ndInputContainersRight = new ArrayList<>(5);
+  private final List<VectorContainer> uninitialized2ndInputContainersLeft = new ArrayList<>(5);
 
   private RowSet.SingleRowSet emptyInputRowSetRight;
   private RowSet.SingleRowSet emptyInputRowSetLeft;
@@ -191,16 +191,6 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
   }
 
   @Test
-  public void testHashJoinStopOutcomeUninitRightSide() {
-    testHashJoinOutcomes(UninitializedSide.Right, RecordBatch.IterOutcome.STOP, RecordBatch.IterOutcome.STOP);
-  }
-
-  @Test
-  public void testHashJoinStopOutcomeUninitLeftSide() {
-    testHashJoinOutcomes(UninitializedSide.Left, RecordBatch.IterOutcome.STOP, RecordBatch.IterOutcome.STOP);
-  }
-
-  @Test
   public void testHashJoinNoneOutcomeUninitRightSide() {
     testHashJoinOutcomes(UninitializedSide.Right, RecordBatch.IterOutcome.NONE, RecordBatch.IterOutcome.NONE);
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestLateralJoinCorrectness.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestLateralJoinCorrectness.java
index 0e64600..b3add2f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestLateralJoinCorrectness.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestLateralJoinCorrectness.java
@@ -154,7 +154,7 @@ public class TestLateralJoinCorrectness extends SubOperatorTest {
    * @return
    */
   private boolean isTerminal(RecordBatch.IterOutcome outcome) {
-    return (outcome == RecordBatch.IterOutcome.NONE || outcome == RecordBatch.IterOutcome.STOP);
+    return (outcome == RecordBatch.IterOutcome.NONE);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitBatchEmitOutcome.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitBatchEmitOutcome.java
index c743582..c93f5cb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitBatchEmitOutcome.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitBatchEmitOutcome.java
@@ -33,9 +33,8 @@ import static org.junit.Assert.assertTrue;
 public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
 
   /**
-   * Test to show empty batch with both OK_NEW_SCHEMA and EMIT outcome is not ignored by Limit and is pass through to
-   * the downstream operator.
-   * @throws Throwable
+   * Test to show empty batch with both OK_NEW_SCHEMA and EMIT outcome is not
+   * ignored by Limit and is pass through to the downstream operator.
    */
   @Test
   public void testLimitEmptyBatchEmitOutcome() throws Throwable {
@@ -52,6 +51,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 1);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
@@ -63,9 +63,8 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
   }
 
   /**
-   * Test to validate limit considers all the data until it sees EMIT outcome and return output batch with data that
-   * meets the limit criteria.
-   * @throws Throwable
+   * Test to validate limit considers all the data until it sees EMIT outcome
+   * and return output batch with data that meets the limit criteria.
    */
   @Test
   public void testLimitNonEmptyBatchEmitOutcome() throws Throwable {
@@ -82,6 +81,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 1);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
@@ -94,9 +94,9 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
   }
 
   /**
-   * Test to show that once a limit number of records is produced using first set of batches then on getting a batch
-   * with EMIT outcome, the limit state is again refreshed and applied to next set of batches with data.
-   * @throws Throwable
+   * Test to show that once a limit number of records is produced using first
+   * set of batches then on getting a batch with EMIT outcome, the limit state
+   * is again refreshed and applied to next set of batches with data.
    */
   @Test
   public void testLimitResetsAfterFirstEmitOutcome() throws Throwable {
@@ -119,6 +119,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 1);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
@@ -134,10 +135,10 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
   }
 
   /**
-   * Test to show that when the limit number of records is found with first incoming batch, then next empty incoming
-   * batch with OK outcome is ignored, but the empty EMIT outcome batch is not ignored. Empty incoming batch with
+   * Test to show that when the limit number of records is found with first
+   * incoming batch, then next empty incoming batch with OK outcome is ignored,
+   * but the empty EMIT outcome batch is not ignored. Empty incoming batch with
    * EMIT outcome produces empty output batch with EMIT outcome.
-   * @throws Throwable
    */
   @Test
   public void testLimitNonEmptyFirst_EmptyOKEmitOutcome() throws Throwable {
@@ -157,6 +158,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 1);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
@@ -169,15 +171,18 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
   }
 
   /**
-   * Test to show that limit refreshes it's state after seeing first EMIT outcome and works on data batches following
-   * it as new set's of incoming batch and apply the limits rule from fresh on those. So for first set of batches with
-   * OK_NEW_SCHEMA and EMIT outcome but total number of records received being less than limit condition, it still
-   * produces an output with that many records (in this case 1 even though limit number of records is 2).
-   *
-   * After seeing EMIT, it refreshes it's state and operate on next input batches to again return limit number of
-   * records. So for 3rd batch with 2 records but with EMIT outcome it produces an output batch with 2 records not
-   * with 1 since state is refreshed.
-   * @throws Throwable
+   * Test to show that limit refreshes it's state after seeing first EMIT
+   * outcome and works on data batches following it as new set's of incoming
+   * batch and apply the limits rule from fresh on those. So for first set of
+   * batches with OK_NEW_SCHEMA and EMIT outcome but total number of records
+   * received being less than limit condition, it still produces an output with
+   * that many records (in this case 1 even though limit number of records is
+   * 2).
+   * <p>
+   * After seeing EMIT, it refreshes it's state and operate on next input
+   * batches to again return limit number of records. So for 3rd batch with 2
+   * records but with EMIT outcome it produces an output batch with 2 records
+   * not with 1 since state is refreshed.
    */
   @Test
   public void testMultipleLimitWithEMITOutcome() throws Throwable {
@@ -201,6 +206,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 2);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
@@ -219,9 +225,8 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
   }
 
   /**
-   * Test shows that limit operates on multiple input batches until it finds limit number of records or it sees an
-   * EMIT outcome to refresh it's state.
-   * @throws Throwable
+   * Test shows that limit operates on multiple input batches until it finds
+   * limit number of records or it sees an EMIT outcome to refresh it's state.
    */
   @Test
   public void testLimitNonEmptyFirst_NonEmptyOK_EmptyBatchEmitOutcome() throws Throwable {
@@ -246,6 +251,7 @@ public class TestLimitBatchEmitOutcome extends BaseTestOpBatchEmitOutcome {
     mockInputBatch.useUnnestKillHandlingForLimit(true);
 
     final Limit limitConf = new Limit(null, 0, 2);
+    @SuppressWarnings("resource")
     final LimitRecordBatch limitBatch = new LimitRecordBatch(limitConf, operatorFixture.getFragmentContext(),
       mockInputBatch);
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
index d116e25..63cf2b2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/protocol/TestOperatorRecordBatch.java
@@ -63,7 +63,6 @@ import org.slf4j.LoggerFactory;
 
 @Category(RowSetTests.class)
 public class TestOperatorRecordBatch extends SubOperatorTest {
-
   private static final Logger logger = LoggerFactory.getLogger(TestOperatorRecordBatch.class);
 
   /**
@@ -265,7 +264,7 @@ public class TestOperatorRecordBatch extends SubOperatorTest {
     opExec.nextCalls = 2;
 
     try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
-      opBatch.kill(false);
+      opBatch.cancel();
       assertFalse(opExec.buildSchemaCalled);
       assertEquals(0, opExec.nextCount);
       assertFalse(opExec.cancelCalled);
@@ -287,7 +286,7 @@ public class TestOperatorRecordBatch extends SubOperatorTest {
     try (OperatorRecordBatch opBatch = makeOpBatch(opExec)) {
       assertEquals(IterOutcome.OK_NEW_SCHEMA, opBatch.next());
       assertEquals(IterOutcome.OK, opBatch.next());
-      opBatch.kill(false);
+      opBatch.cancel();
       assertTrue(opExec.cancelCalled);
     } catch (Exception e) {
       fail();
@@ -310,7 +309,7 @@ public class TestOperatorRecordBatch extends SubOperatorTest {
       assertEquals(IterOutcome.OK, opBatch.next());
       assertEquals(IterOutcome.OK, opBatch.next());
       assertEquals(IterOutcome.NONE, opBatch.next());
-      opBatch.kill(false);
+      opBatch.cancel();
 
       // Already hit EOF, so fail won't be passed along.
 
@@ -342,7 +341,7 @@ public class TestOperatorRecordBatch extends SubOperatorTest {
       fail();
     }
     assertTrue(opExec.closeCalled);
-    opBatch.kill(false);
+    opBatch.cancel();
     assertFalse(opExec.cancelCalled);
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/MockLateralJoinBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/MockLateralJoinBatch.java
index adfb107..e37b375 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/MockLateralJoinBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/MockLateralJoinBatch.java
@@ -139,7 +139,6 @@ public class MockLateralJoinBatch implements LateralContract, CloseableRecordBat
         }
         return currentOutcome;
       case NONE:
-      case STOP:
         isDone = true;
         return currentOutcome;
       case NOT_YET:
@@ -163,15 +162,10 @@ public class MockLateralJoinBatch implements LateralContract, CloseableRecordBat
   }
 
   @Override
-  public boolean hasFailed() {
-    return false;
-  }
+  public void dump() { }
 
   @Override
-  public void dump() {
-  }
-
-  @Override public int getRecordCount() {
+  public int getRecordCount() {
     return 0;
   }
 
@@ -190,23 +184,28 @@ public class MockLateralJoinBatch implements LateralContract, CloseableRecordBat
     return context;
   }
 
-  @Override public BatchSchema getSchema() {
+  @Override
+  public BatchSchema getSchema() {
     return null;
   }
 
-  @Override public void kill(boolean sendUpstream) {
-    unnest.kill(sendUpstream);
+  @Override
+  public void cancel() {
+    unnest.cancel();
   }
 
-  @Override public VectorContainer getOutgoingContainer() {
+  @Override
+  public VectorContainer getOutgoingContainer() {
     return null;
   }
 
-  @Override public TypedFieldId getValueVectorId(SchemaPath path) {
+  @Override
+  public TypedFieldId getValueVectorId(SchemaPath path) {
     return null;
   }
 
-  @Override public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
+  @Override
+  public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
     return null;
   }
 
@@ -242,7 +241,8 @@ public class MockLateralJoinBatch implements LateralContract, CloseableRecordBat
   @Override
   public VectorContainer getContainer() { return null; }
 
-  @Override public Iterator<VectorWrapper<?>> iterator() {
+  @Override
+  public Iterator<VectorWrapper<?>> iterator() {
     return null;
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestCorrectness.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestCorrectness.java
index bec4683..b4c41cf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestCorrectness.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestCorrectness.java
@@ -567,7 +567,7 @@ import static org.junit.Assert.assertTrue;
         batchesProcessed++;
         if (batchesProcessed == execKill) {
           lateralJoinBatch.getContext().getExecutorState().fail(new DrillException("Testing failure of execution."));
-          lateralJoinBatch.kill(true);
+          lateralJoinBatch.cancel();
         }
         // else nothing to do
       }
@@ -716,8 +716,7 @@ import static org.junit.Assert.assertTrue;
   }
 
   private boolean isTerminal(RecordBatch.IterOutcome outcome) {
-    return (outcome == RecordBatch.IterOutcome.NONE || outcome == RecordBatch.IterOutcome.STOP);
+    return (outcome == RecordBatch.IterOutcome.NONE);
   }
-
 }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
index 0baf0a0..3540ef1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
@@ -812,10 +812,9 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
   }
 
   private boolean isTerminal(RecordBatch.IterOutcome outcome) {
-    return (outcome == RecordBatch.IterOutcome.NONE || outcome == RecordBatch.IterOutcome.STOP);
+    return (outcome == RecordBatch.IterOutcome.NONE);
   }
 
-
   /**
    *     Run a plan like the following for various input batches :
    *             Lateral1
@@ -921,7 +920,7 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
           batchesProcessed++;
           if (batchesProcessed == execKill) {
             lateralJoinBatch1.getContext().getExecutorState().fail(new DrillException("Testing failure of execution."));
-            lateralJoinBatch1.kill(true);
+            lateralJoinBatch1.cancel();
           }
           // else nothing to do
         }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
index 5875a40..1ec6739 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/filter/BloomFilterTest.java
@@ -79,7 +79,7 @@ public class BloomFilterTest extends SubOperatorTest {
     }
 
     @Override
-    public void kill(boolean sendUpstream) {
+    public void cancel() {
     }
 
     @Override
@@ -118,13 +118,7 @@ public class BloomFilterTest extends SubOperatorTest {
     }
 
     @Override
-    public void dump() {
-    }
-
-    @Override
-    public boolean hasFailed() {
-      return false;
-    }
+    public void dump() { }
   }
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index 07d8e03..ecc3918 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -145,8 +145,7 @@ public class PhysicalOpUnitTestBase extends ExecTest {
             lastResultOutcome = operator.next();
             needToGrabNext = false;
           }
-          if (lastResultOutcome == RecordBatch.IterOutcome.NONE
-            || lastResultOutcome == RecordBatch.IterOutcome.STOP) {
+          if (lastResultOutcome == RecordBatch.IterOutcome.NONE) {
             return false;
           } else {
             return true;