You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/01/24 18:35:15 UTC

[01/11] drill git commit: DRILL-5851: Empty table during a join operation with a non empty table produces cast exception.

Repository: drill
Updated Branches:
  refs/heads/master a2a56af3e -> e791ed62b


DRILL-5851: Empty table during a join operation with a non empty table produces cast exception.

close apache/drill#1059


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

Branch: refs/heads/master
Commit: 7d1e81a5ac0d8ca0b6519dd2c90bf3804c3221b1
Parents: a2a56af
Author: Hanumath Maduri <hm...@laptop-npjh4dhd.corp.maprtech.com>
Authored: Mon Oct 9 13:08:13 2017 -0700
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 14:39:53 2018 -0800

----------------------------------------------------------------------
 .../exec/physical/impl/join/HashJoinBatch.java  | 21 ++++++-
 .../impl/join/HashJoinProbeTemplate.java        |  6 +-
 .../exec/physical/impl/join/MergeJoinBatch.java |  3 +-
 .../impl/union/UnionAllRecordBatch.java         |  1 +
 .../IteratorValidatorBatchIterator.java         |  1 +
 .../exec/record/AbstractBinaryRecordBatch.java  | 11 +++-
 .../exec/physical/impl/join/JoinTestBase.java   | 63 +++++++++++++++++++
 .../impl/join/TestHashJoinAdvanced.java         | 23 ++++++-
 .../impl/join/TestMergeJoinAdvanced.java        | 25 ++++++--
 .../physical/impl/join/TestNestedLoopJoin.java  | 64 ++++++++++++++------
 10 files changed, 185 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
----------------------------------------------------------------------
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 7e2859e..7b679c0 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
@@ -181,7 +181,9 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       hyperContainer = new ExpandableHyperContainer(vectors);
       hjHelper.addNewBatch(0);
       buildBatchIndex++;
-      setupHashTable();
+      if (isFurtherProcessingRequired(rightUpstream) && this.right.getRecordCount() > 0) {
+        setupHashTable();
+      }
       hashJoinProbe = setupHashJoinProbe();
       // Build the container schema and set the counts
       for (final VectorWrapper<?> w : container) {
@@ -212,7 +214,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       }
 
       // Store the number of records projected
-      if (!hashTable.isEmpty() || joinType != JoinRelType.INNER) {
+      if ((hashTable != null && !hashTable.isEmpty()) || joinType != JoinRelType.INNER) {
 
         // Allocate the memory for the vectors in the output container
         allocateVectors();
@@ -305,11 +307,15 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     //Setup the underlying hash table
 
     // skip first batch if count is zero, as it may be an empty schema batch
-    if (right.getRecordCount() == 0) {
+    if (isFurtherProcessingRequired(rightUpstream) && right.getRecordCount() == 0) {
       for (final VectorWrapper<?> w : right) {
         w.clear();
       }
       rightUpstream = next(right);
+      if (isFurtherProcessingRequired(rightUpstream) &&
+          right.getRecordCount() > 0 && hashTable == null) {
+        setupHashTable();
+      }
     }
 
     boolean moreData = true;
@@ -535,4 +541,13 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     }
     super.close();
   }
+
+  /**
+   * This method checks to see if join processing should be continued further.
+   * @param upStream up stream operator status.
+   * @@return true if up stream status is OK or OK_NEW_SCHEMA otherwise false.
+   */
+  private boolean isFurtherProcessingRequired(IterOutcome upStream) {
+    return upStream == IterOutcome.OK || upStream == IterOutcome.OK_NEW_SCHEMA;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinProbeTemplate.java
----------------------------------------------------------------------
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 3cdce2f..5c6371a 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
@@ -136,7 +136,9 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
           case OK_NEW_SCHEMA:
             if (probeBatch.getSchema().equals(probeSchema)) {
               doSetup(outgoingJoinBatch.getContext(), buildBatch, probeBatch, outgoingJoinBatch);
-              hashTable.updateBatches();
+              if (hashTable != null) {
+                hashTable.updateBatches();
+              }
             } else {
               throw SchemaChangeException.schemaChanged("Hash join does not support schema changes in probe side.",
                   probeSchema,
@@ -155,7 +157,7 @@ public abstract class HashJoinProbeTemplate implements HashJoinProbe {
 
       // Check if we need to drain the next row in the probe side
       if (getNextRecord) {
-        if (hashTable != null) {
+        if (hashTable != null && !hashTable.isEmpty()) {
           probeIndex = hashTable.containsKey(recordsProcessed, true);
         }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
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 6d804c5..8ad3f84 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
@@ -45,7 +45,6 @@ import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.MergeJoinPOP;
 import org.apache.drill.exec.physical.impl.common.Comparator;
-import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
@@ -54,6 +53,7 @@ import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
@@ -500,5 +500,4 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     }
     return materializedExpr;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
----------------------------------------------------------------------
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 761e272..1d1ecb0 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
@@ -78,6 +78,7 @@ public class UnionAllRecordBatch extends AbstractBinaryRecordBatch<UnionAll> {
 
   protected void buildSchema() throws SchemaChangeException {
     if (! prefetchFirstBatchFromBothSides()) {
+      state = BatchState.DONE;
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
----------------------------------------------------------------------
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 a8ee0de..ac6a462 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
@@ -137,6 +137,7 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
     switch (batchState) {
     case OK:
     case OK_NEW_SCHEMA:
+    case NONE:
       return;
     default:
       throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractBinaryRecordBatch.java
----------------------------------------------------------------------
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 1137922..1ce5fde 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
@@ -65,11 +65,20 @@ public abstract class AbstractBinaryRecordBatch<T extends PhysicalOperator> exte
       return false;
     }
 
-    if (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.NONE) {
+    if (checkForEarlyFinish()) {
       state = BatchState.DONE;
       return false;
     }
 
     return true;
   }
+
+  /*
+   * Checks for the operator specific early terminal condition.
+   * @return true if the further processing can stop.
+   *         false if the further processing is needed.
+   */
+  protected boolean checkForEarlyFinish() {
+    return (leftUpstream == IterOutcome.NONE && rightUpstream == IterOutcome.NONE);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/JoinTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/JoinTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/JoinTestBase.java
new file mode 100644
index 0000000..6d55a3b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/JoinTestBase.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.join;
+
+import org.apache.drill.categories.OperatorTest;
+import org.apache.drill.PlanTestBase;
+import org.junit.experimental.categories.Category;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+
+@Category(OperatorTest.class)
+public class JoinTestBase extends PlanTestBase {
+
+  private static final String testEmptyJoin = "select count(*) as cnt from cp.`employee.json` emp %s join dfs.`dept.json` " +
+          "as dept on dept.manager = emp.`last_name`";
+
+  /**
+   * This method runs a join query with one of the table generated as an
+   * empty json file.
+   * @param testDir in which the empty json file is generated.
+   * @param joinType to be executed.
+   * @param joinPattern to look for the pattern in the successful run.
+   * @param result number of the output rows.
+   */
+  public void testJoinWithEmptyFile(File testDir, String joinType,
+                         String joinPattern, long result) throws Exception {
+    buildFile("dept.json", new String[0], testDir);
+    String query = String.format(testEmptyJoin, joinType);
+    testPlanMatchingPatterns(query, new String[]{joinPattern}, new String[]{});
+    testBuilder()
+            .sqlQuery(query)
+            .unOrdered()
+            .baselineColumns("cnt")
+            .baselineValues(result)
+            .build().run();
+  }
+
+  private void buildFile(String fileName, String[] data, File testDir) throws IOException {
+    try(PrintWriter out = new PrintWriter(new FileWriter(new File(testDir, fileName)))) {
+      for (String line : data) {
+        out.println(line);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinAdvanced.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinAdvanced.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinAdvanced.java
index 49aefe6..8110476 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinAdvanced.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinAdvanced.java
@@ -19,20 +19,22 @@
 package org.apache.drill.exec.physical.impl.join;
 
 
-import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 
+
 @Category(OperatorTest.class)
-public class TestHashJoinAdvanced extends BaseTestQuery {
+public class TestHashJoinAdvanced extends JoinTestBase {
+
+  private static final String HJ_PATTERN = "HashJoin";
+
 
   // Have to disable merge join, if this testcase is to test "HASH-JOIN".
   @BeforeClass
@@ -160,4 +162,19 @@ public class TestHashJoinAdvanced extends BaseTestQuery {
       .baselineValues("1", "2", "1", null, "a")
       .go();
   }
+
+  @Test
+  public void testHashLeftJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "left outer", HJ_PATTERN, 1155L);
+  }
+
+  @Test
+  public void testHashInnerJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "inner", HJ_PATTERN, 0L);
+  }
+
+  @Test
+  public void testHashRightJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "right outer", HJ_PATTERN, 0L);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
index 310b331..488e60a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinAdvanced.java
@@ -17,16 +17,15 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
-import org.apache.drill.test.BaseTestQuery;
-import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.test.TestTools;
+import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
@@ -38,13 +37,16 @@ import java.nio.file.Paths;
 import java.util.Random;
 
 @Category(OperatorTest.class)
-public class TestMergeJoinAdvanced extends BaseTestQuery {
+public class TestMergeJoinAdvanced extends JoinTestBase {
   private static final String LEFT = "merge-join-left.json";
   private static final String RIGHT = "merge-join-right.json";
+  private static final String MJ_PATTERN = "MergeJoin";
+
 
   private static File leftFile;
   private static File rightFile;
 
+
   @Rule
   public final TestRule TIMEOUT = TestTools.getTimeoutRule(120000); // Longer timeout than usual.
 
@@ -253,4 +255,19 @@ public class TestMergeJoinAdvanced extends BaseTestQuery {
       .baselineValues(6000*800L)
       .go();
   }
+
+  @Test
+  public void testMergeLeftJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(),"left outer", MJ_PATTERN, 1155L);
+  }
+
+  @Test
+  public void testMergeInnerJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "inner", MJ_PATTERN, 0L);
+  }
+
+  @Test
+  public void testMergeRightJoinWithEmptyTable() throws Exception {
+    testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "right outer", MJ_PATTERN, 0L);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7d1e81a5/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
index a67a484..092a1a7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
@@ -19,21 +19,19 @@
 package org.apache.drill.exec.physical.impl.join;
 
 import org.apache.drill.categories.OperatorTest;
-import org.apache.drill.PlanTestBase;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
 import java.nio.file.Paths;
-
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertThat;
 
 @Category(OperatorTest.class)
-public class TestNestedLoopJoin extends PlanTestBase {
+public class TestNestedLoopJoin extends JoinTestBase {
 
-  private static String nlpattern = "NestedLoopJoin";
+  private static final String NLJ_PATTERN = "NestedLoopJoin";
 
   private static final String DISABLE_HJ = "alter session set `planner.enable_hashjoin` = false";
   private static final String ENABLE_HJ = "alter session set `planner.enable_hashjoin` = true";
@@ -86,30 +84,30 @@ public class TestNestedLoopJoin extends PlanTestBase {
 
   @Test
   public void testNlJoinExists_1_planning() throws Exception {
-    testPlanMatchingPatterns(testNlJoinExists_1, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(testNlJoinExists_1, new String[]{NLJ_PATTERN}, new String[]{});
   }
 
   @Test
   public void testNlJoinNotIn_1_planning() throws Exception {
-    testPlanMatchingPatterns(testNlJoinNotIn_1, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(testNlJoinNotIn_1, new String[]{NLJ_PATTERN}, new String[]{});
   }
 
   @Test
   public void testNlJoinInequality_1() throws Exception {
-    testPlanMatchingPatterns(testNlJoinInequality_1, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(testNlJoinInequality_1, new String[]{NLJ_PATTERN}, new String[]{});
   }
 
   @Test
   public void testNlJoinInequality_2() throws Exception {
     test(DISABLE_NLJ_SCALAR);
-    testPlanMatchingPatterns(testNlJoinInequality_2, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(testNlJoinInequality_2, new String[]{NLJ_PATTERN}, new String[]{});
     test(ENABLE_NLJ_SCALAR);
   }
 
   @Test
   public void testNlJoinInequality_3() throws Exception {
     test(DISABLE_NLJ_SCALAR);
-    testPlanMatchingPatterns(testNlJoinInequality_3, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(testNlJoinInequality_3, new String[]{NLJ_PATTERN}, new String[]{});
     test(ENABLE_NLJ_SCALAR);
   }
 
@@ -118,7 +116,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     String query = "select total1, total2 from "
         + "(select sum(l_quantity) as total1 from cp.`tpch/lineitem.parquet` where l_suppkey between 100 and 200), "
         + "(select sum(l_quantity) as total2 from cp.`tpch/lineitem.parquet` where l_suppkey between 200 and 300)  ";
-    testPlanMatchingPatterns(query, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN}, new String[]{});
   }
 
   @Test // equality join and scalar right input, hj and mj disabled
@@ -128,7 +126,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
         + "                        where n_nationkey < 10)";
     test(DISABLE_HJ);
     test(DISABLE_MJ);
-    testPlanMatchingPatterns(query, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN}, new String[]{});
     test(ENABLE_HJ);
     test(ENABLE_MJ);
   }
@@ -141,7 +139,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     test("alter session set `planner.slice_target` = 1");
     test(DISABLE_HJ);
     test(DISABLE_MJ);
-    testPlanMatchingPatterns(query, new String[]{nlpattern, "BroadcastExchange"}, new String[]{});
+    testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN, "BroadcastExchange"}, new String[]{});
     test(ENABLE_HJ);
     test(ENABLE_MJ);
     test("alter session set `planner.slice_target` = 100000");
@@ -154,7 +152,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     test(DISABLE_HJ);
     test(DISABLE_MJ);
     test(DISABLE_NLJ_SCALAR);
-    testPlanMatchingPatterns(query, new String[]{nlpattern}, new String[]{});
+    testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN}, new String[]{});
     test(ENABLE_HJ);
     test(ENABLE_MJ);
     test(ENABLE_NLJ_SCALAR);
@@ -169,7 +167,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     test(DISABLE_HJ);
     test(DISABLE_MJ);
     test(DISABLE_NLJ_SCALAR);
-    testPlanMatchingPatterns(query, new String[]{nlpattern, "BroadcastExchange"}, new String[]{});
+    testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN, "BroadcastExchange"}, new String[]{});
     test(ENABLE_HJ);
     test(ENABLE_MJ);
     test(ENABLE_NLJ_SCALAR);
@@ -274,7 +272,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     try {
       test(DISABLE_NLJ_SCALAR);
       String query = String.format(testNlJoinBetween, "INNER");
-      testPlanMatchingPatterns(query, new String[]{nlpattern}, new String[]{});
+      testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN}, new String[]{});
       testBuilder()
           .sqlQuery(query)
           .ordered()
@@ -292,7 +290,7 @@ public class TestNestedLoopJoin extends PlanTestBase {
     try {
       test(DISABLE_NLJ_SCALAR);
       String query = String.format(testNlJoinBetween, "LEFT");
-      testPlanMatchingPatterns(query, new String[]{nlpattern}, new String[]{});
+      testPlanMatchingPatterns(query, new String[]{NLJ_PATTERN}, new String[]{});
       testBuilder()
           .sqlQuery(query)
           .ordered()
@@ -327,10 +325,40 @@ public class TestNestedLoopJoin extends PlanTestBase {
     try {
       test(DISABLE_NLJ_SCALAR);
       test(DISABLE_JOIN_OPTIMIZATION);
-      testPlanMatchingPatterns(testNlJoinWithLargeRightInput, new String[]{nlpattern}, new String[]{});
+      testPlanMatchingPatterns(testNlJoinWithLargeRightInput, new String[]{NLJ_PATTERN}, new String[]{});
     } finally {
       test(RESET_HJ);
       test(RESET_JOIN_OPTIMIZATION);
     }
   }
+
+  @Test
+  public void testNestedLeftJoinWithEmptyTable() throws Exception {
+    try {
+      alterSession(PlannerSettings.NLJOIN_FOR_SCALAR.getOptionName(), false);
+      testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "left outer", NLJ_PATTERN, 1155L);
+    } finally {
+      resetSessionOption(PlannerSettings.HASHJOIN.getOptionName());
+    }
+  }
+
+  @Test
+  public void testNestedInnerJoinWithEmptyTable() throws Exception {
+    try {
+      alterSession(PlannerSettings.NLJOIN_FOR_SCALAR.getOptionName(), false);
+      testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "inner", NLJ_PATTERN, 0L);
+    } finally {
+      resetSessionOption(PlannerSettings.HASHJOIN.getOptionName());
+    }
+  }
+
+  @Test
+  public void testNestRightJoinWithEmptyTable() throws Exception {
+    try {
+      alterSession(PlannerSettings.NLJOIN_FOR_SCALAR.getOptionName(), false);
+      testJoinWithEmptyFile(dirTestWatcher.getRootDir(), "right outer", NLJ_PATTERN, 0L);
+    } finally {
+      resetSessionOption(PlannerSettings.HASHJOIN.getOptionName());
+    }
+  }
 }


[05/11] drill git commit: DRILL-5879: Improved SQL Pattern Contains Performance

Posted by am...@apache.org.
DRILL-5879: Improved SQL Pattern Contains Performance

close apache/drill#1072


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/2420b35d
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/2420b35d
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/2420b35d

Branch: refs/heads/master
Commit: 2420b35d716a35a10be91ea82ec17bdeb392a77c
Parents: 48623ea
Author: Salim Achouche <sa...@gmail.com>
Authored: Wed Dec 13 14:24:45 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 17:37:17 2018 -0800

----------------------------------------------------------------------
 .../expr/fn/impl/SqlPatternContainsMatcher.java | 274 ++++++++++++++++++-
 .../exec/expr/fn/impl/TestSqlPatterns.java      | 112 +++++++-
 2 files changed, 363 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2420b35d/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SqlPatternContainsMatcher.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SqlPatternContainsMatcher.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SqlPatternContainsMatcher.java
index 04f5dac..ec95349 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SqlPatternContainsMatcher.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/SqlPatternContainsMatcher.java
@@ -19,44 +19,292 @@ package org.apache.drill.exec.expr.fn.impl;
 
 import io.netty.buffer.DrillBuf;
 
-public class SqlPatternContainsMatcher extends AbstractSqlPatternMatcher {
+/** SQL Pattern Contains implementation */
+public final class SqlPatternContainsMatcher extends AbstractSqlPatternMatcher {
+  private final MatcherFcn matcherFcn;
 
   public SqlPatternContainsMatcher(String patternString) {
     super(patternString);
+
+    // Pattern matching is 1) a CPU intensive operation and 2) pattern and input dependent. The conclusion is
+    // that there is no single implementation that can do it all well. So, we use multiple implementations
+    // chosen based on the pattern length.
+    if (patternLength == 0) {
+      matcherFcn = new MatcherZero();
+    } else if (patternLength == 1) {
+      matcherFcn = new MatcherOne();
+    } else if (patternLength == 2) {
+      matcherFcn = new MatcherTwo();
+    } else if (patternLength == 3) {
+      matcherFcn = new MatcherThree();
+    } else if (patternLength < 10) {
+      matcherFcn = new MatcherN();
+    } else {
+      matcherFcn = new BoyerMooreMatcher();
+    }
   }
 
   @Override
   public int match(int start, int end, DrillBuf drillBuf) {
+    return matcherFcn.match(start, end, drillBuf);
+  }
+
+  //--------------------------------------------------------------------------
+  // Inner Data Structure
+  // --------------------------------------------------------------------------
+
+  /** Abstract matcher class to allow us pick the most efficient implementation */
+  private abstract class MatcherFcn {
+    protected final byte[] patternArray;
+
+    protected MatcherFcn() {
+      assert patternByteBuffer.hasArray();
+
+      patternArray = patternByteBuffer.array();
+    }
+
+    /**
+     * @return 1 if the pattern was matched; 0 otherwise
+     */
+    protected abstract int match(int start, int end, DrillBuf drillBuf);
+  }
+
+  /** Handles patterns with length zero */
+  private final class MatcherZero extends MatcherFcn {
 
-    if (patternLength == 0) { // Everything should match for null pattern string
+    private MatcherZero() {
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected final int match(int start, int end, DrillBuf drillBuf) {
       return 1;
     }
+  }
+
+  /** Handles patterns with length one */
+  private final class MatcherOne extends MatcherFcn {
+    final byte firstPatternByte;
+
+    private MatcherOne() {
+      firstPatternByte  = patternArray[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected final int match(int start, int end, DrillBuf drillBuf) {
+      final int lengthToProcess = end - start;
+
+      // simplePattern string has meta characters i.e % and _ and escape characters removed.
+      // so, we can just directly compare.
+      for (int idx = 0; idx < lengthToProcess; idx++) {
+        byte inputByte = drillBuf.getByte(start + idx);
+
+        if (firstPatternByte != inputByte) {
+          continue;
+        }
+        return 1;
+      }
+      return 0;
+    }
+  }
+
+  /** Handles patterns with length two */
+  private final class MatcherTwo extends MatcherFcn {
+    final byte firstPatternByte;
+    final byte secondPatternByte;
+
+    private MatcherTwo() {
+      firstPatternByte  = patternArray[0];
+      secondPatternByte = patternArray[1];
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected final int match(int start, int end, DrillBuf drillBuf) {
+      final int lengthToProcess = end - start - 1;
+
+      // simplePattern string has meta characters i.e % and _ and escape characters removed.
+      // so, we can just directly compare.
+      for (int idx = 0; idx < lengthToProcess; idx++) {
+        final byte firstInByte = drillBuf.getByte(start + idx);
 
-    final int txtLength = end - start;
+        if (firstPatternByte != firstInByte) {
+          continue;
+        } else {
+          final byte secondInByte = drillBuf.getByte(start + idx + 1);
 
-    // no match if input string length is less than pattern length
-    if (txtLength < patternLength) {
+          if (secondInByte == secondPatternByte) {
+            return 1;
+          }
+        }
+      }
       return 0;
     }
+  }
 
+  /** Handles patterns with length three */
+  private final class MatcherThree extends MatcherFcn {
+    final byte firstPatternByte;
+    final byte secondPatternByte;
+    final byte thirdPatternByte;
 
-    final int outerEnd = txtLength - patternLength;
+    private MatcherThree() {
+      firstPatternByte   = patternArray[0];
+      secondPatternByte  = patternArray[1];
+      thirdPatternByte   = patternArray[2];
+    }
 
-    outer:
-    for (int txtIndex = 0; txtIndex <= outerEnd; txtIndex++) {
+    /** {@inheritDoc} */
+    @Override
+    protected final int match(int start, int end, DrillBuf drillBuf) {
+      final int lengthToProcess = end - start - 2;
 
       // simplePattern string has meta characters i.e % and _ and escape characters removed.
       // so, we can just directly compare.
-      for (int patternIndex = 0; patternIndex < patternLength; patternIndex++) {
-        if (patternByteBuffer.get(patternIndex) != drillBuf.getByte(start + txtIndex + patternIndex)) {
-          continue outer;
+      for (int idx = 0; idx < lengthToProcess; idx++) {
+        final byte inputByte = drillBuf.getByte(start + idx);
+
+        if (firstPatternByte != inputByte) {
+          continue;
+        } else {
+          final byte secondInByte = drillBuf.getByte(start + idx + 1);
+          final byte thirdInByte  = drillBuf.getByte(start + idx + 2);
+
+          if (secondInByte == secondPatternByte && thirdInByte == thirdPatternByte) {
+            return 1;
+          }
         }
       }
+      return 0;
+    }
+  }
 
-      return 1;
+  /** Handles patterns with arbitrary length */
+  private final class MatcherN extends MatcherFcn {
+    final byte firstPatternByte;
+
+    private MatcherN() {
+      firstPatternByte = patternArray[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected final int match(int start, int end, DrillBuf drillBuf) {
+      final int lengthToProcess = end - start - patternLength + 1;
+      int patternIndex          = 0;
+
+      // simplePattern string has meta characters i.e % and _ and escape characters removed.
+      // so, we can just directly compare.
+      for (int idx = 0; idx < lengthToProcess; idx++) {
+        final byte inputByte = drillBuf.getByte(start + idx);
+
+        if (firstPatternByte == inputByte) {
+          for (patternIndex = 1; patternIndex < patternLength; ++patternIndex) {
+            final byte currInByte   = drillBuf.getByte(start + idx + patternIndex);
+            final byte currPattByte = patternArray[patternIndex];
+
+            if (currInByte != currPattByte) {
+              break;
+            }
+          }
+
+          if (patternIndex == patternLength) {
+            return 1;
+          }
+        }
+      }
+      return 0;
+    }
+  }
+
+  /**
+   * Boyer-Moore matcher algorithm; excellent for large patterns and for prefix patterns which appear
+   * frequently in the input.
+   */
+  private final class BoyerMooreMatcher extends MatcherFcn {
+    private final int[] offsetTable;
+    private final int[] characterTable;
+
+    private BoyerMooreMatcher() {
+      super();
+
+      this.offsetTable    = makeOffsetTable();
+      this.characterTable = makeCharTable();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected int match(int start, int end, DrillBuf drillBuf)  {
+      final int inputLength = end - start;
+
+      for (int idx1 = patternLength - 1, idx2; idx1 < inputLength;) {
+        for (idx2 = patternLength - 1; patternArray[idx2] == drillBuf.getByte(start + idx1); --idx1, --idx2) {
+          if (idx2 == 0) {
+            return 1;
+          }
+        }
+        // idx1 += pattern.length - idx2; // For naive method
+        idx1 += Math.max(offsetTable[patternLength - 1 - idx2], characterTable[drillBuf.getByte(start + idx1) & 0xFF]);
+      }
+      return 0;
+    }
+
+    /** Build the jump table based on the mismatched character information **/
+    private int[] makeCharTable() {
+      final int TABLE_SIZE = 256; // This implementation is based on byte comparison
+      int[] resultTable    = new int[TABLE_SIZE];
+
+      for (int idx = 0; idx < resultTable.length; ++idx) {
+        resultTable[idx] = patternLength;
+      }
+
+      for (int idx = 0; idx < patternLength - 1; ++idx) {
+        final int patternValue    = ((int) patternArray[idx]) & 0xFF;
+        resultTable[patternValue] = patternLength - 1 - idx;
+      }
+
+      return resultTable;
+    }
+
+    /** Builds the scan offset based on which mismatch occurs. **/
+    private int[] makeOffsetTable() {
+      int[] resultTable      = new int[patternLength];
+      int lastPrefixPosition = patternLength;
+
+      for (int idx = patternLength - 1; idx >= 0; --idx) {
+        if (isPrefix(idx + 1)) {
+          lastPrefixPosition = idx + 1;
+        }
+        resultTable[patternLength - 1 - idx] = lastPrefixPosition - idx + patternLength - 1;
+      }
+
+      for (int idx = 0; idx < patternLength - 1; ++idx) {
+        int suffixLen          = suffixLength(idx);
+        resultTable[suffixLen] = patternLength - 1 - idx + suffixLen;
+      }
+
+      return resultTable;
     }
 
-    return  0;
+    /** Checks whether needle[pos:end] is a prefix of pattern **/
+    private boolean isPrefix(int pos) {
+      for (int idx1 = pos, idx2 = 0; idx1 < patternLength; ++idx1, ++idx2) {
+        if (patternArray[idx1] != patternArray[idx2]) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    /** Computes the maximum length of the substring ends at "pos" and is a suffix **/
+    private int suffixLength(int pos) {
+      int result = 0;
+      for (int idx1 = pos, idx2 = patternLength - 1; idx1 >= 0 && patternArray[idx1] == patternArray[idx2]; --idx1, --idx2) {
+        result += 1;
+      }
+      return result;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/2420b35d/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestSqlPatterns.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestSqlPatterns.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestSqlPatterns.java
index 2eecb54..7d85719 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestSqlPatterns.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestSqlPatterns.java
@@ -17,23 +17,25 @@
  */
 
 package org.apache.drill.exec.expr.fn.impl;
-import io.netty.buffer.DrillBuf;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.RootAllocatorFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
 import java.nio.CharBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.Charset;
 import java.nio.charset.CharsetEncoder;
+import java.util.ArrayList;
+import java.util.List;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.DrillBuf;
 
 public class TestSqlPatterns {
   BufferAllocator allocator;
@@ -446,10 +448,100 @@ public class TestSqlPatterns {
     assertEquals(1, sqlPatternComplex.match(0, byteBuffer.limit(), drillBuf)); // should match
   }
 
+  @Test
+  public void testSqlPatternContainsMUltipleMatchers() {
+
+    final String longASCIIString = "Drill supports a variety of NoSQL databases and file systems, including HBase, MongoDB, MapR-DB, HDFS, MapR-FS, Amazon S3, Azure Blob Storage, Google Cloud Storage, Swift, "
+      + "NAS and local files. A single query can join data from multiple datastores. For example, you can join a user profile collection in MongoDB with a directory of event logs in Hadoop.";
+    final String emptyString     = "";
+    final String unicodeString   = "¤EÀsÆW°ê»Ú®i¶T¤¤¤ß3¼Ó®i¶TÆU2~~";
+
+    final List<SQLPatternTestParams> tests = new ArrayList<SQLPatternTestParams>();
+
+    // Tests for Matcher ZERO
+    tests.add(new SQLPatternTestParams(longASCIIString, "", true));
+    tests.add(new SQLPatternTestParams(emptyString,     "", true));
+    tests.add(new SQLPatternTestParams(unicodeString,   "", true));
+
+    // Tests for Matcher ONE
+    tests.add(new SQLPatternTestParams(longASCIIString, "N", true));
+    tests.add(new SQLPatternTestParams(longASCIIString, "&", false));
+    tests.add(new SQLPatternTestParams(emptyString,     "N", false));
+
+    // Tests for Matcher TWO
+    tests.add(new SQLPatternTestParams(longASCIIString, "SQ", true));
+    tests.add(new SQLPatternTestParams(longASCIIString, "eT", false));
+    tests.add(new SQLPatternTestParams("A",             "SQ", false));
+    tests.add(new SQLPatternTestParams(emptyString,     "SQ", false));
+    tests.add(new SQLPatternTestParams(unicodeString,   "¶",  true));
+    tests.add(new SQLPatternTestParams(unicodeString,   "AT", false));
+
+    // Tests for Matcher THREE
+    tests.add(new SQLPatternTestParams(longASCIIString, "SQL", true));
+    tests.add(new SQLPatternTestParams(longASCIIString, "cas", false));
+    tests.add(new SQLPatternTestParams("S",             "SQL", false));
+    tests.add(new SQLPatternTestParams(emptyString,     "SQL", false));
+    tests.add(new SQLPatternTestParams(unicodeString,   "¶T", true));
+    tests.add(new SQLPatternTestParams(unicodeString,   "¶A", false));
+
+    // Tests for Matcher for patterns of length: 3 < length < 10
+    tests.add(new SQLPatternTestParams(longASCIIString, "MongoDB", true));
+    tests.add(new SQLPatternTestParams(longASCIIString, "MongoDz", false));
+    tests.add(new SQLPatternTestParams("Mon",           "MongoDB", false));
+    tests.add(new SQLPatternTestParams(emptyString,     "MongoDB", false));
+    tests.add(new SQLPatternTestParams(unicodeString,   "®i¶", true));
+    tests.add(new SQLPatternTestParams(unicodeString,   "®x¶", false));
+
+    // Tests for Matcher for patterns of length >= 10
+    tests.add(new SQLPatternTestParams(longASCIIString, "multiple datastores", true));
+    tests.add(new SQLPatternTestParams(longASCIIString, "multiple datastorb",  false));
+    tests.add(new SQLPatternTestParams("multiple",      "multiple datastores", false));
+    tests.add(new SQLPatternTestParams(emptyString,     "multiple datastores", false));
+    tests.add(new SQLPatternTestParams(unicodeString,   "¶T¤¤¤ß3¼", true));
+    tests.add(new SQLPatternTestParams(unicodeString,   "¶T¤¤¤ßz¼", false));
+
+    for (SQLPatternTestParams test : tests) {
+      setDrillBuf(test.inputString);
+
+      RegexpUtil.SqlPatternInfo patternInfo = new RegexpUtil.SqlPatternInfo(RegexpUtil.SqlPatternType.CONTAINS, "", test.patternString);
+      SqlPatternMatcher sqlPatternContains  = SqlPatternFactory.getSqlPatternMatcher(patternInfo);
+      int eval                              = sqlPatternContains.match(0, byteBuffer.limit(), drillBuf);
+      int expectedEval                      = test.shouldMatch ? 1 : 0;
+
+      if (eval != expectedEval) {
+        System.err.format("test failed; params=%s%n", test);
+      }
+
+      assertEquals(expectedEval, eval);
+    }
+  }
+
+
   @After
   public void cleanup() {
     drillBuf.close();
     allocator.close();
   }
+
+  // -------------
+  // Inner Classes
+  // -------------
+
+  /** Container class to hold SQL pattern test data */
+  private static class SQLPatternTestParams {
+    private final String inputString;
+    private final String patternString;
+    private final boolean shouldMatch;
+
+    private SQLPatternTestParams(String inputString, String patternString, boolean shouldMatch) {
+      this.inputString   = inputString;
+      this.patternString = patternString;
+      this.shouldMatch   = shouldMatch;
+    }
+
+    public String toString() {
+      return "input=["+inputString+"], pattern=["+patternString+"], should-match=["+shouldMatch+"]..";
+    }
+  }
 }
 


[03/11] drill git commit: DRILL-5868: Support SQL syntax highlighting of queries

Posted by am...@apache.org.
DRILL-5868: Support SQL syntax highlighting of queries

Based on the commit for DRILL-5981 (PR #1043), this commit further leverages the Ace JavaScript library with customizations specific to Drill.

This commit introduces the following to the Query Editor (including the Edit Query tab within an existing profile to rerunning the query).
1. Syntax highlighting (This is supported for submitted query profiles
2. Autocomplete supported in editors
3. Specifying Drill specific keywords and functions in visible autocomplete
4. Key snippets (template SQLs) allowing for rapid writing of syntax:
  i. Query System Tables
  ii. CView, CTAS and CTempTAS
  iii. Alter Session
  iv. Explain and Select * queries

NOTE: The lists for #3 and #4 are not exhaustive. As more features are added to Drill, these lists can be expanded.
Updates based on review comments

1. Disabled warning message
2. Extended reserved keyword list
3. Fixed bugs
List:
Update 1: Bug fix for when Impersonation is enabled
Update 2: Remove the duplicate editor that might get injected
Update 3: Removed trailing whitespaces in Javascripts

close apache/drill#1084


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/4c8a55b8
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/4c8a55b8
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/4c8a55b8

Branch: refs/heads/master
Commit: 4c8a55b825b91ca26073f9ba059e18cdaa08670d
Parents: a2058fd
Author: Kunal Khatua <kk...@maprtech.com>
Authored: Mon Jan 22 12:01:32 2018 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 14:42:23 2018 -0800

----------------------------------------------------------------------
 .../src/main/resources/rest/profile/profile.ftl |  83 +++++++-
 .../src/main/resources/rest/query/query.ftl     |  43 +++-
 .../rest/static/js/ace-code-editor/ace.js       |   3 +-
 .../js/ace-code-editor/ext-language_tools.js    |   4 +
 .../static/js/ace-code-editor/mode-snippets.js  | 199 +++++++++++++++++++
 .../rest/static/js/ace-code-editor/mode-sql.js  | 135 ++++++++++++-
 .../static/js/ace-code-editor/snippets/sql.js   |  48 +++++
 .../js/ace-code-editor/theme-sqlserver.js       | 138 +++++++++++++
 .../src/main/resources/rest/storage/update.ftl  |   6 +-
 9 files changed, 649 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/profile/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
index 1f38d2e..7f4474c 100644
--- a/exec/java-exec/src/main/resources/rest/profile/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -19,6 +19,13 @@
     <script src="/static/js/jquery.form.js"></script>
     <script src="/static/js/querySubmission.js"></script>
 </#if>
+  <!-- Ace Libraries for Syntax Formatting -->
+  <script src="/static/js/ace-code-editor/ace.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/mode-sql.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/ext-language_tools.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/theme-sqlserver.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/snippets/sql.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/mode-snippets.js" type="text/javascript" charset="utf-8"></script>
 
 <script>
     var globalconfig = {
@@ -65,7 +72,7 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
   </ul>
   <div id="query-content" class="tab-content">
     <div id="query-query" class="tab-pane">
-      <p><pre>${model.getProfile().query}</pre></p>
+      <p><pre id="query-text" name="query-text"  style="background-color: #f5f5f5;">${model.getProfile().query}</pre></p>
     </div>
     <div id="query-physical" class="tab-pane">
       <p><pre>${model.profile.plan}</pre></p>
@@ -84,9 +91,8 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
         </#if>
 
         <form role="form" id="queryForm" action="/query" method="POST">
-          <div class="form-group">
-            <textarea class="form-control" id="query" name="query" style="font-family: Courier;">${model.getProfile().query}</textarea>
-          </div>
+          <div id="query-editor" class="form-group">${model.getProfile().query}</div>
+          <input class="form-control" id="query" name="query" type="hidden" value="${model.getProfile().query}"/>
           <div class="form-group">
             <div class="radio-inline">
               <label>
@@ -364,6 +370,75 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
   </div>
   <div class="page-header">
   </div> <br>
+
+    <script>
+    //Configuration for Query Viewer in Profile
+    ace.require("ace/ext/language_tools");
+    var viewer = ace.edit("query-text");
+    viewer.setAutoScrollEditorIntoView(true);
+    viewer.setOption("minLines", 3);
+    viewer.setOption("maxLines", 20);
+    viewer.renderer.setShowGutter(false);
+    viewer.renderer.setOption('showLineNumbers', false);
+    viewer.renderer.setOption('showPrintMargin', false);
+    viewer.renderer.setOption("vScrollBarAlwaysVisible", true);
+    viewer.renderer.setOption("hScrollBarAlwaysVisible", true);
+    viewer.renderer.setScrollMargin(10, 10, 10, 10);
+    viewer.getSession().setMode("ace/mode/sql");
+    viewer.setTheme("ace/theme/sqlserver");
+    //CSS Formatting
+    document.getElementById('query-query').style.fontSize='13px';
+    document.getElementById('query-query').style.fontFamily='courier';
+    document.getElementById('query-query').style.lineHeight='1.5';
+    document.getElementById('query-query').style.width='98%';
+    document.getElementById('query-query').style.margin='auto';
+    document.getElementById('query-query').style.backgroundColor='#f5f5f5';
+    viewer.resize();
+    viewer.setReadOnly(true);
+    viewer.setOptions({
+      enableBasicAutocompletion: false,
+      enableSnippets: false,
+      enableLiveAutocompletion: false
+    });
+  </script>
+
+  <script>
+    //Configuration for Query Editor in Profile
+    ace.require("ace/ext/language_tools");
+    var editor = ace.edit("query-editor");
+    //Hidden text input for form-submission
+    var queryText = $('input[name="query"]');
+    editor.getSession().on("change", function () {
+      queryText.val(editor.getSession().getValue());
+    });
+    editor.setAutoScrollEditorIntoView(true);
+    editor.setOption("maxLines", 16);
+    editor.setOption("minLines", 10);
+    editor.renderer.setShowGutter(true);
+    editor.renderer.setOption('showLineNumbers', true);
+    editor.renderer.setOption('showPrintMargin', false);
+    editor.renderer.setOption("vScrollBarAlwaysVisible", true);
+    editor.renderer.setOption("hScrollBarAlwaysVisible", true);;
+    editor.renderer.setScrollMargin(10, 10, 10, 10);
+    editor.getSession().setMode("ace/mode/sql");
+    editor.getSession().setTabSize(4);
+    editor.getSession().setUseSoftTabs(true);
+    editor.setTheme("ace/theme/sqlserver");
+    editor.$blockScrolling = "Infinity";
+    //CSS Formatting
+    document.getElementById('query-editor').style.fontSize='13px';
+    document.getElementById('query-editor').style.fontFamily='courier';
+    document.getElementById('query-editor').style.lineHeight='1.5';
+    document.getElementById('query-editor').style.width='98%';
+    document.getElementById('query-editor').style.margin='auto';
+    document.getElementById('query-editor').style.backgroundColor='#ffffff';
+    editor.setOptions({
+      enableSnippets: true,
+      enableBasicAutocompletion: true,
+      enableLiveAutocompletion: false
+    });
+  </script>
+
 </#macro>
 
 <@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/query/query.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/query/query.ftl b/exec/java-exec/src/main/resources/rest/query/query.ftl
index 93194d0..6a08914 100644
--- a/exec/java-exec/src/main/resources/rest/query/query.ftl
+++ b/exec/java-exec/src/main/resources/rest/query/query.ftl
@@ -15,6 +15,13 @@
       <script src="/static/js/jquery.form.js"></script>
       <script src="/static/js/querySubmission.js"></script>
     </#if>
+  <!-- Ace Libraries for Syntax Formatting -->
+  <script src="/static/js/ace-code-editor/ace.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/mode-sql.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/ext-language_tools.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/theme-sqlserver.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/snippets/sql.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/mode-snippets.js" type="text/javascript" charset="utf-8"></script>
 </#macro>
 
 <#macro page_body>
@@ -57,13 +64,47 @@
     </div>
     <div class="form-group">
       <label for="query">Query</label>
-      <textarea class="form-control" id="query" rows="5" name="query" style="font-family: Courier;"></textarea>
+      <div id="query-editor-format"></div>
+      <input class="form-control" type="hidden" id="query" name="query"/>
     </div>
 
     <button class="btn btn-default" type=<#if model?? && model>"button" onclick="doSubmitQueryWithUserName()"<#else>"submit"</#if>>
       Submit
     </button>
   </form>
+
+  <script>
+    ace.require("ace/ext/language_tools");
+    var editor = ace.edit("query-editor-format");
+    var queryText = $('input[name="query"]');
+    //Hidden text input for form-submission
+    editor.getSession().on("change", function () {
+      queryText.val(editor.getSession().getValue());
+    });
+    editor.setAutoScrollEditorIntoView(true);
+    editor.setOption("maxLines", 25);
+    editor.setOption("minLines", 12);
+    editor.renderer.setShowGutter(true);
+    editor.renderer.setOption('showLineNumbers', true);
+    editor.renderer.setOption('showPrintMargin', false);
+    editor.getSession().setMode("ace/mode/sql");
+    editor.getSession().setTabSize(4);
+    editor.getSession().setUseSoftTabs(true);
+    editor.setTheme("ace/theme/sqlserver");
+    editor.$blockScrolling = "Infinity";
+    //CSS Formatting
+    document.getElementById('query-editor-format').style.fontSize='13px';
+    document.getElementById('query-editor-format').style.fontFamily='courier';
+    document.getElementById('query-editor-format').style.lineHeight='1.5';
+    document.getElementById('query-editor-format').style.width='98%';
+    document.getElementById('query-editor-format').style.margin='auto';
+    editor.setOptions({
+      enableSnippets: true,
+      enableBasicAutocompletion: true,
+      enableLiveAutocompletion: false
+    });
+  </script>
+
 </#macro>
 
 <@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ace.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ace.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ace.js
index 0d70ea5..7f50978 100644
--- a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ace.js
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ace.js
@@ -10,5 +10,4 @@
                     for (var key in a) if (a.hasOwnProperty(key))
                         window.ace[key] = a[key];
                 });
-            })();
-        
\ No newline at end of file
+            })();
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ext-language_tools.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ext-language_tools.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ext-language_tools.js
new file mode 100644
index 0000000..aa7dc72
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/ext-language_tools.js
@@ -0,0 +1,4 @@
+ace.define("ace/snippets",["require","exports","module","ace/lib/oop","ace/lib/event_emitter","ace/lib/lang","ace/range","ace/anchor","ace/keyboard/hash_handler","ace/tokenizer","ace/lib/dom","ace/editor"],function(e,t,n){"use strict";var r=e("./lib/oop"),i=e("./lib/event_emitter").EventEmitter,s=e("./lib/lang"),o=e("./range").Range,u=e("./anchor").Anchor,a=e("./keyboard/hash_handler").HashHandler,f=e("./tokenizer").Tokenizer,l=o.comparePoints,c=function(){this.snippetMap={},this.snippetNameMap={}};(function(){r.implement(this,i),this.getTokenizer=function(){function e(e,t,n){return e=e.substr(1),/^\d+$/.test(e)&&!n.inFormatString?[{tabstopId:parseInt(e,10)}]:[{text:e}]}function t(e){return"(?:[^\\\\"+e+"]|\\\\.)"}return c.$tokenizer=new f({start:[{regex:/:/,onMatch:function(e,t,n){return n.length&&n[0].expectIf?(n[0].expectIf=!1,n[0].elseBranch=n[0],[n[0]]):":"}},{regex:/\\./,onMatch:function(e,t,n){var r=e[1];return r=="}"&&n.length?e=r:"`$\\".indexOf(r)!=-1?e=r:n.inFormatString&&
 (r=="n"?e="\n":r=="t"?e="\n":"ulULE".indexOf(r)!=-1&&(e={changeCase:r,local:r>"a"})),[e]}},{regex:/}/,onMatch:function(e,t,n){return[n.length?n.shift():e]}},{regex:/\$(?:\d+|\w+)/,onMatch:e},{regex:/\$\{[\dA-Z_a-z]+/,onMatch:function(t,n,r){var i=e(t.substr(1),n,r);return r.unshift(i[0]),i},next:"snippetVar"},{regex:/\n/,token:"newline",merge:!1}],snippetVar:[{regex:"\\|"+t("\\|")+"*\\|",onMatch:function(e,t,n){n[0].choices=e.slice(1,-1).split(",")},next:"start"},{regex:"/("+t("/")+"+)/(?:("+t("/")+"*)/)(\\w*):?",onMatch:function(e,t,n){var r=n[0];return r.fmtString=e,e=this.splitRegex.exec(e),r.guard=e[1],r.fmt=e[2],r.flag=e[3],""},next:"start"},{regex:"`"+t("`")+"*`",onMatch:function(e,t,n){return n[0].code=e.splice(1,-1),""},next:"start"},{regex:"\\?",onMatch:function(e,t,n){n[0]&&(n[0].expectIf=!0)},next:"start"},{regex:"([^:}\\\\]|\\\\.)*:?",token:"",next:"start"}],formatString:[{regex:"/("+t("/")+"+)/",token:"regex"},{regex:"",onMatch:function(e,t,n){n.inFormatString=!0},next:
 "start"}]}),c.prototype.getTokenizer=function(){return c.$tokenizer},c.$tokenizer},this.tokenizeTmSnippet=function(e,t){return this.getTokenizer().getLineTokens(e,t).tokens.map(function(e){return e.value||e})},this.$getDefaultValue=function(e,t){if(/^[A-Z]\d+$/.test(t)){var n=t.substr(1);return(this.variables[t[0]+"__"]||{})[n]}if(/^\d+$/.test(t))return(this.variables.__||{})[t];t=t.replace(/^TM_/,"");if(!e)return;var r=e.session;switch(t){case"CURRENT_WORD":var i=r.getWordRange();case"SELECTION":case"SELECTED_TEXT":return r.getTextRange(i);case"CURRENT_LINE":return r.getLine(e.getCursorPosition().row);case"PREV_LINE":return r.getLine(e.getCursorPosition().row-1);case"LINE_INDEX":return e.getCursorPosition().column;case"LINE_NUMBER":return e.getCursorPosition().row+1;case"SOFT_TABS":return r.getUseSoftTabs()?"YES":"NO";case"TAB_SIZE":return r.getTabSize();case"FILENAME":case"FILEPATH":return"";case"FULLNAME":return"Ace"}},this.variables={},this.getVariableValue=function(e,t){return 
 this.variables.hasOwnProperty(t)?this.variables[t](e,t)||"":this.$getDefaultValue(e,t)||""},this.tmStrFormat=function(e,t,n){var r=t.flag||"",i=t.guard;i=new RegExp(i,r.replace(/[^gi]/,""));var s=this.tokenizeTmSnippet(t.fmt,"formatString"),o=this,u=e.replace(i,function(){o.variables.__=arguments;var e=o.resolveVariables(s,n),t="E";for(var r=0;r<e.length;r++){var i=e[r];if(typeof i=="object"){e[r]="";if(i.changeCase&&i.local){var u=e[r+1];u&&typeof u=="string"&&(i.changeCase=="u"?e[r]=u[0].toUpperCase():e[r]=u[0].toLowerCase(),e[r+1]=u.substr(1))}else i.changeCase&&(t=i.changeCase)}else t=="U"?e[r]=i.toUpperCase():t=="L"&&(e[r]=i.toLowerCase())}return e.join("")});return this.variables.__=null,u},this.resolveVariables=function(e,t){function o(t){var n=e.indexOf(t,r+1);n!=-1&&(r=n)}var n=[];for(var r=0;r<e.length;r++){var i=e[r];if(typeof i=="string")n.push(i);else{if(typeof i!="object")continue;if(i.skip)o(i);else{if(i.processed<r)continue;if(i.text){var s=this.getVariableValue(t,i.
 text);s&&i.fmtString&&(s=this.tmStrFormat(s,i)),i.processed=r,i.expectIf==null?s&&(n.push(s),o(i)):s?i.skip=i.elseBranch:o(i)}else i.tabstopId!=null?n.push(i):i.changeCase!=null&&n.push(i)}}}return n},this.insertSnippetForSelection=function(e,t){function f(e){var t=[];for(var n=0;n<e.length;n++){var r=e[n];if(typeof r=="object"){if(a[r.tabstopId])continue;var i=e.lastIndexOf(r,n-1);r=t[i]||{tabstopId:r.tabstopId}}t[n]=r}return t}var n=e.getCursorPosition(),r=e.session.getLine(n.row),i=e.session.getTabString(),s=r.match(/^\s*/)[0];n.column<s.length&&(s=s.slice(0,n.column)),t=t.replace(/\r/g,"");var o=this.tokenizeTmSnippet(t);o=this.resolveVariables(o,e),o=o.map(function(e){return e=="\n"?e+s:typeof e=="string"?e.replace(/\t/g,i):e});var u=[];o.forEach(function(e,t){if(typeof e!="object")return;var n=e.tabstopId,r=u[n];r||(r=u[n]=[],r.index=n,r.value="");if(r.indexOf(e)!==-1)return;r.push(e);var i=o.indexOf(e,t+1);if(i===-1)return;var s=o.slice(t+1,i),a=s.some(function(e){return type
 of e=="object"});a&&!r.value?r.value=s:s.length&&(!r.value||typeof r.value!="string")&&(r.value=s.join(""))}),u.forEach(function(e){e.length=0});var a={};for(var l=0;l<o.length;l++){var c=o[l];if(typeof c!="object")continue;var p=c.tabstopId,d=o.indexOf(c,l+1);if(a[p]){a[p]===c&&(a[p]=null);continue}var v=u[p],m=typeof v.value=="string"?[v.value]:f(v.value);m.unshift(l+1,Math.max(0,d-l)),m.push(c),a[p]=c,o.splice.apply(o,m),v.indexOf(c)===-1&&v.push(c)}var g=0,y=0,b="";o.forEach(function(e){if(typeof e=="string"){var t=e.split("\n");t.length>1?(y=t[t.length-1].length,g+=t.length-1):y+=e.length,b+=e}else e.start?e.end={row:g,column:y}:e.start={row:g,column:y}});var w=e.getSelectionRange(),E=e.session.replace(w,b),S=new h(e),x=e.inVirtualSelectionMode&&e.selection.index;S.addTabstops(u,w.start,E,x)},this.insertSnippet=function(e,t){var n=this;if(e.inVirtualSelectionMode)return n.insertSnippetForSelection(e,t);e.forEachSelection(function(){n.insertSnippetForSelection(e,t)},null,{keepOr
 der:!0}),e.tabstopManager&&e.tabstopManager.tabNext()},this.$getScope=function(e){var t=e.session.$mode.$id||"";t=t.split("/").pop();if(t==="html"||t==="php"){t==="php"&&!e.session.$mode.inlinePhp&&(t="html");var n=e.getCursorPosition(),r=e.session.getState(n.row);typeof r=="object"&&(r=r[0]),r.substring&&(r.substring(0,3)=="js-"?t="javascript":r.substring(0,4)=="css-"?t="css":r.substring(0,4)=="php-"&&(t="php"))}return t},this.getActiveScopes=function(e){var t=this.$getScope(e),n=[t],r=this.snippetMap;return r[t]&&r[t].includeScopes&&n.push.apply(n,r[t].includeScopes),n.push("_"),n},this.expandWithTab=function(e,t){var n=this,r=e.forEachSelection(function(){return n.expandSnippetForSelection(e,t)},null,{keepOrder:!0});return r&&e.tabstopManager&&e.tabstopManager.tabNext(),r},this.expandSnippetForSelection=function(e,t){var n=e.getCursorPosition(),r=e.session.getLine(n.row),i=r.substring(0,n.column),s=r.substr(n.column),o=this.snippetMap,u;return this.getActiveScopes(e).some(functio
 n(e){var t=o[e];return t&&(u=this.findMatchingSnippet(t,i,s)),!!u},this),u?t&&t.dryRun?!0:(e.session.doc.removeInLine(n.row,n.column-u.replaceBefore.length,n.column+u.replaceAfter.length),this.variables.M__=u.matchBefore,this.variables.T__=u.matchAfter,this.insertSnippetForSelection(e,u.content),this.variables.M__=this.variables.T__=null,!0):!1},this.findMatchingSnippet=function(e,t,n){for(var r=e.length;r--;){var i=e[r];if(i.startRe&&!i.startRe.test(t))continue;if(i.endRe&&!i.endRe.test(n))continue;if(!i.startRe&&!i.endRe)continue;return i.matchBefore=i.startRe?i.startRe.exec(t):[""],i.matchAfter=i.endRe?i.endRe.exec(n):[""],i.replaceBefore=i.triggerRe?i.triggerRe.exec(t)[0]:"",i.replaceAfter=i.endTriggerRe?i.endTriggerRe.exec(n)[0]:"",i}},this.snippetMap={},this.snippetNameMap={},this.register=function(e,t){function o(e){return e&&!/^\^?\(.*\)\$?$|^\\b$/.test(e)&&(e="(?:"+e+")"),e||""}function u(e,t,n){return e=o(e),t=o(t),n?(e=t+e,e&&e[e.length-1]!="$"&&(e+="$")):(e+=t,e&&e[0]!="
 ^"&&(e="^"+e)),new RegExp(e)}function a(e){e.scope||(e.scope=t||"_"),t=e.scope,n[t]||(n[t]=[],r[t]={});var o=r[t];if(e.name){var a=o[e.name];a&&i.unregister(a),o[e.name]=e}n[t].push(e),e.tabTrigger&&!e.trigger&&(!e.guard&&/^\w/.test(e.tabTrigger)&&(e.guard="\\b"),e.trigger=s.escapeRegExp(e.tabTrigger));if(!e.trigger&&!e.guard&&!e.endTrigger&&!e.endGuard)return;e.startRe=u(e.trigger,e.guard,!0),e.triggerRe=new RegExp(e.trigger,"",!0),e.endRe=u(e.endTrigger,e.endGuard,!0),e.endTriggerRe=new RegExp(e.endTrigger,"",!0)}var n=this.snippetMap,r=this.snippetNameMap,i=this;e||(e=[]),e&&e.content?a(e):Array.isArray(e)&&e.forEach(a),this._signal("registerSnippets",{scope:t})},this.unregister=function(e,t){function i(e){var i=r[e.scope||t];if(i&&i[e.name]){delete i[e.name];var s=n[e.scope||t],o=s&&s.indexOf(e);o>=0&&s.splice(o,1)}}var n=this.snippetMap,r=this.snippetNameMap;e.content?i(e):Array.isArray(e)&&e.forEach(i)},this.parseSnippetFile=function(e){e=e.replace(/\r/g,"");var t=[],n={},r=/^
 #.*|^({[\s\S]*})\s*$|^(\S+) (.*)$|^((?:\n*\t.*)+)/gm,i;while(i=r.exec(e)){if(i[1])try{n=JSON.parse(i[1]),t.push(n)}catch(s){}if(i[4])n.content=i[4].replace(/^\t/gm,""),t.push(n),n={};else{var o=i[2],u=i[3];if(o=="regex"){var a=/\/((?:[^\/\\]|\\.)*)|$/g;n.guard=a.exec(u)[1],n.trigger=a.exec(u)[1],n.endTrigger=a.exec(u)[1],n.endGuard=a.exec(u)[1]}else o=="snippet"?(n.tabTrigger=u.match(/^\S*/)[0],n.name||(n.name=u)):n[o]=u}}return t},this.getSnippetByName=function(e,t){var n=this.snippetNameMap,r;return this.getActiveScopes(t).some(function(t){var i=n[t];return i&&(r=i[e]),!!r},this),r}}).call(c.prototype);var h=function(e){if(e.tabstopManager)return e.tabstopManager;e.tabstopManager=this,this.$onChange=this.onChange.bind(this),this.$onChangeSelection=s.delayedCall(this.onChangeSelection.bind(this)).schedule,this.$onChangeSession=this.onChangeSession.bind(this),this.$onAfterExec=this.onAfterExec.bind(this),this.attach(e)};(function(){this.attach=function(e){this.index=0,this.ranges=[]
 ,this.tabstops=[],this.$openTabstops=null,this.selectedTabstop=null,this.editor=e,this.editor.on("change",this.$onChange),this.editor.on("changeSelection",this.$onChangeSelection),this.editor.on("changeSession",this.$onChangeSession),this.editor.commands.on("afterExec",this.$onAfterExec),this.editor.keyBinding.addKeyboardHandler(this.keyboardHandler)},this.detach=function(){this.tabstops.forEach(this.removeTabstopMarkers,this),this.ranges=null,this.tabstops=null,this.selectedTabstop=null,this.editor.removeListener("change",this.$onChange),this.editor.removeListener("changeSelection",this.$onChangeSelection),this.editor.removeListener("changeSession",this.$onChangeSession),this.editor.commands.removeListener("afterExec",this.$onAfterExec),this.editor.keyBinding.removeKeyboardHandler(this.keyboardHandler),this.editor.tabstopManager=null,this.editor=null},this.onChange=function(e){var t=e,n=e.action[0]=="r",r=e.start,i=e.end,s=r.row,o=i.row,u=o-s,a=i.column-r.column;n&&(u=-u,a=-a);if(!
 this.$inChange&&n){var f=this.selectedTabstop,c=f&&!f.some(function(e){return l(e.start,r)<=0&&l(e.end,i)>=0});if(c)return this.detach()}var h=this.ranges;for(var p=0;p<h.length;p++){var d=h[p];if(d.end.row<r.row)continue;if(n&&l(r,d.start)<0&&l(i,d.end)>0){this.removeRange(d),p--;continue}d.start.row==s&&d.start.column>r.column&&(d.start.column+=a),d.end.row==s&&d.end.column>=r.column&&(d.end.column+=a),d.start.row>=s&&(d.start.row+=u),d.end.row>=s&&(d.end.row+=u),l(d.start,d.end)>0&&this.removeRange(d)}h.length||this.detach()},this.updateLinkedFields=function(){var e=this.selectedTabstop;if(!e||!e.hasLinkedRanges)return;this.$inChange=!0;var n=this.editor.session,r=n.getTextRange(e.firstNonLinked);for(var i=e.length;i--;){var s=e[i];if(!s.linked)continue;var o=t.snippetManager.tmStrFormat(r,s.original);n.replace(s,o)}this.$inChange=!1},this.onAfterExec=function(e){e.command&&!e.command.readOnly&&this.updateLinkedFields()},this.onChangeSelection=function(){if(!this.editor)return;va
 r e=this.editor.selection.lead,t=this.editor.selection.anchor,n=this.editor.selection.isEmpty();for(var r=this.ranges.length;r--;){if(this.ranges[r].linked)continue;var i=this.ranges[r].contains(e.row,e.column),s=n||this.ranges[r].contains(t.row,t.column);if(i&&s)return}this.detach()},this.onChangeSession=function(){this.detach()},this.tabNext=function(e){var t=this.tabstops.length,n=this.index+(e||1);n=Math.min(Math.max(n,1),t),n==t&&(n=0),this.selectTabstop(n),n===0&&this.detach()},this.selectTabstop=function(e){this.$openTabstops=null;var t=this.tabstops[this.index];t&&this.addTabstopMarkers(t),this.index=e,t=this.tabstops[this.index];if(!t||!t.length)return;this.selectedTabstop=t;if(!this.editor.inVirtualSelectionMode){var n=this.editor.multiSelect;n.toSingleRange(t.firstNonLinked.clone());for(var r=t.length;r--;){if(t.hasLinkedRanges&&t[r].linked)continue;n.addRange(t[r].clone(),!0)}n.ranges[0]&&n.addRange(n.ranges[0].clone())}else this.editor.selection.setRange(t.firstNonLinke
 d);this.editor.keyBinding.addKeyboardHandler(this.keyboardHandler)},this.addTabstops=function(e,t,n){this.$openTabstops||(this.$openTabstops=[]);if(!e[0]){var r=o.fromPoints(n,n);v(r.start,t),v(r.end,t),e[0]=[r],e[0].index=0}var i=this.index,s=[i+1,0],u=this.ranges;e.forEach(function(e,n){var r=this.$openTabstops[n]||e;for(var i=e.length;i--;){var a=e[i],f=o.fromPoints(a.start,a.end||a.start);d(f.start,t),d(f.end,t),f.original=a,f.tabstop=r,u.push(f),r!=e?r.unshift(f):r[i]=f,a.fmtString?(f.linked=!0,r.hasLinkedRanges=!0):r.firstNonLinked||(r.firstNonLinked=f)}r.firstNonLinked||(r.hasLinkedRanges=!1),r===e&&(s.push(r),this.$openTabstops[n]=r),this.addTabstopMarkers(r)},this),s.length>2&&(this.tabstops.length&&s.push(s.splice(2,1)[0]),this.tabstops.splice.apply(this.tabstops,s))},this.addTabstopMarkers=function(e){var t=this.editor.session;e.forEach(function(e){e.markerId||(e.markerId=t.addMarker(e,"ace_snippet-marker","text"))})},this.removeTabstopMarkers=function(e){var t=this.edito
 r.session;e.forEach(function(e){t.removeMarker(e.markerId),e.markerId=null})},this.removeRange=function(e){var t=e.tabstop.indexOf(e);e.tabstop.splice(t,1),t=this.ranges.indexOf(e),this.ranges.splice(t,1),this.editor.session.removeMarker(e.markerId),e.tabstop.length||(t=this.tabstops.indexOf(e.tabstop),t!=-1&&this.tabstops.splice(t,1),this.tabstops.length||this.detach())},this.keyboardHandler=new a,this.keyboardHandler.bindKeys({Tab:function(e){if(t.snippetManager&&t.snippetManager.expandWithTab(e))return;e.tabstopManager.tabNext(1)},"Shift-Tab":function(e){e.tabstopManager.tabNext(-1)},Esc:function(e){e.tabstopManager.detach()},Return:function(e){return!1}})}).call(h.prototype);var p={};p.onChange=u.prototype.onChange,p.setPosition=function(e,t){this.pos.row=e,this.pos.column=t},p.update=function(e,t,n){this.$insertRight=n,this.pos=e,this.onChange(t)};var d=function(e,t){e.row==0&&(e.column+=t.column),e.row+=t.row},v=function(e,t){e.row==t.row&&(e.column-=t.column),e.row-=t.row};e(
 "./lib/dom").importCssString(".ace_snippet-marker {    -moz-box-sizing: border-box;    box-sizing: border-box;    background: rgba(194, 193, 208, 0.09);    border: 1px dotted rgba(211, 208, 235, 0.62);    position: absolute;}"),t.snippetManager=new c;var m=e("./editor").Editor;(function(){this.insertSnippet=function(e,n){return t.snippetManager.insertSnippet(this,e,n)},this.expandSnippet=function(e){return t.snippetManager.expandWithTab(this,e)}}).call(m.prototype)}),ace.define("ace/autocomplete/popup",["require","exports","module","ace/virtual_renderer","ace/editor","ace/range","ace/lib/event","ace/lib/lang","ace/lib/dom"],function(e,t,n){"use strict";var r=e("../virtual_renderer").VirtualRenderer,i=e("../editor").Editor,s=e("../range").Range,o=e("../lib/event"),u=e("../lib/lang"),a=e("../lib/dom"),f=function(e){var t=new r(e);t.$maxLines=4;var n=new i(t);return n.setHighlightActiveLine(!1),n.setShowPrintMargin(!1),n.renderer.setShowGutter(!1),n.renderer.setHighlightGutterLine(!1),
 n.$mouseHandler.$focusWaitTimout=0,n.$highlightTagPending=!0,n},l=function(e){var t=a.createElement("div"),n=new f(t);e&&e.appendChild(t),t.style.display="none",n.renderer.content.style.cursor="default",n.renderer.setStyle("ace_autocomplete"),n.setOption("displayIndentGuides",!1),n.setOption("dragDelay",150);var r=function(){};n.focus=r,n.$isFocused=!0,n.renderer.$cursorLayer.restartTimer=r,n.renderer.$cursorLayer.element.style.opacity=0,n.renderer.$maxLines=8,n.renderer.$keepTextAreaAtCursor=!1,n.setHighlightActiveLine(!1),n.session.highlight(""),n.session.$searchHighlight.clazz="ace_highlight-marker",n.on("mousedown",function(e){var t=e.getDocumentPosition();n.selection.moveToPosition(t),c.start.row=c.end.row=t.row,e.stop()});var i,l=new s(-1,0,-1,Infinity),c=new s(-1,0,-1,Infinity);c.id=n.session.addMarker(c,"ace_active-line","fullLine"),n.setSelectOnHover=function(e){e?l.id&&(n.session.removeMarker(l.id),l.id=null):l.id=n.session.addMarker(l,"ace_line-hover","fullLine")},n.setSe
 lectOnHover(!1),n.on("mousemove",function(e){if(!i){i=e;return}if(i.x==e.x&&i.y==e.y)return;i=e,i.scrollTop=n.renderer.scrollTop;var t=i.getDocumentPosition().row;l.start.row!=t&&(l.id||n.setRow(t),p(t))}),n.renderer.on("beforeRender",function(){if(i&&l.start.row!=-1){i.$pos=null;var e=i.getDocumentPosition().row;l.id||n.setRow(e),p(e,!0)}}),n.renderer.on("afterRender",function(){var e=n.getRow(),t=n.renderer.$textLayer,r=t.element.childNodes[e-t.config.firstRow];if(r==t.selectedNode)return;t.selectedNode&&a.removeCssClass(t.selectedNode,"ace_selected"),t.selectedNode=r,r&&a.addCssClass(r,"ace_selected")});var h=function(){p(-1)},p=function(e,t){e!==l.start.row&&(l.start.row=l.end.row=e,t||n.session._emit("changeBackMarker"),n._emit("changeHoverMarker"))};n.getHoveredRow=function(){return l.start.row},o.addListener(n.container,"mouseout",h),n.on("hide",h),n.on("changeSelection",h),n.session.doc.getLength=function(){return n.data.length},n.session.doc.getLine=function(e){var t=n.data
 [e];return typeof t=="string"?t:t&&t.value||""};var d=n.session.bgTokenizer;return d.$tokenizeRow=function(e){var t=n.data[e],r=[];if(!t)return r;typeof t=="string"&&(t={value:t}),t.caption||(t.caption=t.value||t.name);var i=-1,s,o;for(var u=0;u<t.caption.length;u++)o=t.caption[u],s=t.matchMask&1<<u?1:0,i!==s?(r.push({type:t.className||""+(s?"completion-highlight":""),value:o}),i=s):r[r.length-1].value+=o;if(t.meta){var a=n.renderer.$size.scrollerWidth/n.renderer.layerConfig.characterWidth,f=t.meta;f.length+t.caption.length>a-2&&(f=f.substr(0,a-t.caption.length-3)+"\u2026"),r.push({type:"rightAlignedText",value:f})}return r},d.$updateOnChange=r,d.start=r,n.session.$computeWidth=function(){return this.screenWidth=0},n.$blockScrolling=Infinity,n.isOpen=!1,n.isTopdown=!1,n.autoSelect=!0,n.data=[],n.setData=function(e){n.setValue(u.stringRepeat("\n",e.length),-1),n.data=e||[],n.setRow(0)},n.getData=function(e){return n.data[e]},n.getRow=function(){return c.start.row},n.setRow=function(e
 ){e=Math.max(this.autoSelect?0:-1,Math.min(this.data.length,e)),c.start.row!=e&&(n.selection.clearSelection(),c.start.row=c.end.row=e||0,n.session._emit("changeBackMarker"),n.moveCursorTo(e||0,0),n.isOpen&&n._signal("select"))},n.on("changeSelection",function(){n.isOpen&&n.setRow(n.selection.lead.row),n.renderer.scrollCursorIntoView()}),n.hide=function(){this.container.style.display="none",this._signal("hide"),n.isOpen=!1},n.show=function(e,t,r){var s=this.container,o=window.innerHeight,u=window.innerWidth,a=this.renderer,f=a.$maxLines*t*1.4,l=e.top+this.$borderSize,c=l>o/2&&!r;c&&l+t+f>o?(a.$maxPixelHeight=l-2*this.$borderSize,s.style.top="",s.style.bottom=o-l+"px",n.isTopdown=!1):(l+=t,a.$maxPixelHeight=o-l-.2*t,s.style.top=l+"px",s.style.bottom="",n.isTopdown=!0),s.style.display="",this.renderer.$textLayer.checkForSizeChanges();var h=e.left;h+s.offsetWidth>u&&(h=u-s.offsetWidth),s.style.left=h+"px",this._signal("show"),i=null,n.isOpen=!0},n.getTextLeftOffset=function(){return thi
 s.$borderSize+this.renderer.$padding+this.$imageSize},n.$imageSize=0,n.$borderSize=1,n};a.importCssString(".ace_editor.ace_autocomplete .ace_marker-layer .ace_active-line {    background-color: #CAD6FA;    z-index: 1;}.ace_editor.ace_autocomplete .ace_line-hover {    border: 1px solid #abbffe;    margin-top: -1px;    background: rgba(233,233,253,0.4);}.ace_editor.ace_autocomplete .ace_line-hover {    position: absolute;    z-index: 2;}.ace_editor.ace_autocomplete .ace_scroller {   background: none;   border: none;   box-shadow: none;}.ace_rightAlignedText {    color: gray;    display: inline-block;    position: absolute;    right: 4px;    text-align: right;    z-index: -1;}.ace_editor.ace_autocomplete .ace_completion-highlight{    color: #000;    text-shadow: 0 0 0.01em;}.ace_editor.ace_autocomplete {    width: 280px;    z-index: 200000;    background: #fbfbfb;    color: #444;    border: 1px lightgray solid;    position: fixed;    box-shadow: 2px 3px 5px rgba(0,0,0,.2);    line-heig
 ht: 1.4;}"),t.AcePopup=l}),ace.define("ace/autocomplete/util",["require","exports","module"],function(e,t,n){"use strict";t.parForEach=function(e,t,n){var r=0,i=e.length;i===0&&n();for(var s=0;s<i;s++)t(e[s],function(e,t){r++,r===i&&n(e,t)})};var r=/[a-zA-Z_0-9\$\-\u00A2-\uFFFF]/;t.retrievePrecedingIdentifier=function(e,t,n){n=n||r;var i=[];for(var s=t-1;s>=0;s--){if(!n.test(e[s]))break;i.push(e[s])}return i.reverse().join("")},t.retrieveFollowingIdentifier=function(e,t,n){n=n||r;var i=[];for(var s=t;s<e.length;s++){if(!n.test(e[s]))break;i.push(e[s])}return i},t.getCompletionPrefix=function(e){var t=e.getCursorPosition(),n=e.session.getLine(t.row),r;return e.completers.forEach(function(e){e.identifierRegexps&&e.identifierRegexps.forEach(function(e){!r&&e&&(r=this.retrievePrecedingIdentifier(n,t.column,e))}.bind(this))}.bind(this)),r||this.retrievePrecedingIdentifier(n,t.column)}}),ace.define("ace/autocomplete",["require","exports","module","ace/keyboard/hash_handler","ace/autocompl
 ete/popup","ace/autocomplete/util","ace/lib/event","ace/lib/lang","ace/lib/dom","ace/snippets"],function(e,t,n){"use strict";var r=e("./keyboard/hash_handler").HashHandler,i=e("./autocomplete/popup").AcePopup,s=e("./autocomplete/util"),o=e("./lib/event"),u=e("./lib/lang"),a=e("./lib/dom"),f=e("./snippets").snippetManager,l=function(){this.autoInsert=!1,this.autoSelect=!0,this.exactMatch=!1,this.gatherCompletionsId=0,this.keyboardHandler=new r,this.keyboardHandler.bindKeys(this.commands),this.blurListener=this.blurListener.bind(this),this.changeListener=this.changeListener.bind(this),this.mousedownListener=this.mousedownListener.bind(this),this.mousewheelListener=this.mousewheelListener.bind(this),this.changeTimer=u.delayedCall(function(){this.updateCompletions(!0)}.bind(this)),this.tooltipTimer=u.delayedCall(this.updateDocTooltip.bind(this),50)};(function(){this.$init=function(){return this.popup=new i(document.body||document.documentElement),this.popup.on("click",function(e){this.i
 nsertMatch(),e.stop()}.bind(this)),this.popup.focus=this.editor.focus.bind(this.editor),this.popup.on("show",this.tooltipTimer.bind(null,null)),this.popup.on("select",this.tooltipTimer.bind(null,null)),this.popup.on("changeHoverMarker",this.tooltipTimer.bind(null,null)),this.popup},this.getPopup=function(){return this.popup||this.$init()},this.openPopup=function(e,t,n){this.popup||this.$init(),this.popup.autoSelect=this.autoSelect,this.popup.setData(this.completions.filtered),e.keyBinding.addKeyboardHandler(this.keyboardHandler);var r=e.renderer;this.popup.setRow(this.autoSelect?0:-1);if(!n){this.popup.setTheme(e.getTheme()),this.popup.setFontSize(e.getFontSize());var i=r.layerConfig.lineHeight,s=r.$cursorLayer.getPixelPosition(this.base,!0);s.left-=this.popup.getTextLeftOffset();var o=e.container.getBoundingClientRect();s.top+=o.top-r.layerConfig.offset,s.left+=o.left-e.renderer.scrollLeft,s.left+=r.gutterWidth,this.popup.show(s,i)}else n&&!t&&this.detach()},this.detach=function(){
 this.editor.keyBinding.removeKeyboardHandler(this.keyboardHandler),this.editor.off("changeSelection",this.changeListener),this.editor.off("blur",this.blurListener),this.editor.off("mousedown",this.mousedownListener),this.editor.off("mousewheel",this.mousewheelListener),this.changeTimer.cancel(),this.hideDocTooltip(),this.gatherCompletionsId+=1,this.popup&&this.popup.isOpen&&this.popup.hide(),this.base&&this.base.detach(),this.activated=!1,this.completions=this.base=null},this.changeListener=function(e){var t=this.editor.selection.lead;(t.row!=this.base.row||t.column<this.base.column)&&this.detach(),this.activated?this.changeTimer.schedule():this.detach()},this.blurListener=function(e){var t=document.activeElement,n=this.editor.textInput.getElement(),r=e.relatedTarget&&this.tooltipNode&&this.tooltipNode.contains(e.relatedTarget),i=this.popup&&this.popup.container;t!=n&&t.parentNode!=i&&!r&&t!=this.tooltipNode&&e.relatedTarget!=n&&this.detach()},this.mousedownListener=function(e){this
 .detach()},this.mousewheelListener=function(e){this.detach()},this.goTo=function(e){var t=this.popup.getRow(),n=this.popup.session.getLength()-1;switch(e){case"up":t=t<=0?n:t-1;break;case"down":t=t>=n?-1:t+1;break;case"start":t=0;break;case"end":t=n}this.popup.setRow(t)},this.insertMatch=function(e,t){e||(e=this.popup.getData(this.popup.getRow()));if(!e)return!1;if(e.completer&&e.completer.insertMatch)e.completer.insertMatch(this.editor,e);else{if(this.completions.filterText){var n=this.editor.selection.getAllRanges();for(var r=0,i;i=n[r];r++)i.start.column-=this.completions.filterText.length,this.editor.session.remove(i)}e.snippet?f.insertSnippet(this.editor,e.snippet):this.editor.execCommand("insertstring",e.value||e)}this.detach()},this.commands={Up:function(e){e.completer.goTo("up")},Down:function(e){e.completer.goTo("down")},"Ctrl-Up|Ctrl-Home":function(e){e.completer.goTo("start")},"Ctrl-Down|Ctrl-End":function(e){e.completer.goTo("end")},Esc:function(e){e.completer.detach()},
 Return:function(e){return e.completer.insertMatch()},"Shift-Return":function(e){e.completer.insertMatch(null,{deleteSuffix:!0})},Tab:function(e){var t=e.completer.insertMatch();if(!!t||!!e.tabstopManager)return t;e.completer.goTo("down")},PageUp:function(e){e.completer.popup.gotoPageUp()},PageDown:function(e){e.completer.popup.gotoPageDown()}},this.gatherCompletions=function(e,t){var n=e.getSession(),r=e.getCursorPosition(),i=s.getCompletionPrefix(e);this.base=n.doc.createAnchor(r.row,r.column-i.length),this.base.$insertRight=!0;var o=[],u=e.completers.length;return e.completers.forEach(function(a,f){a.getCompletions(e,n,r,i,function(n,r){!n&&r&&(o=o.concat(r)),t(null,{prefix:s.getCompletionPrefix(e),matches:o,finished:--u===0})})}),!0},this.showPopup=function(e){this.editor&&this.detach(),this.activated=!0,this.editor=e,e.completer!=this&&(e.completer&&e.completer.detach(),e.completer=this),e.on("changeSelection",this.changeListener),e.on("blur",this.blurListener),e.on("mousedown",
 this.mousedownListener),e.on("mousewheel",this.mousewheelListener),this.updateCompletions()},this.updateCompletions=function(e){if(e&&this.base&&this.completions){var t=this.editor.getCursorPosition(),n=this.editor.session.getTextRange({start:this.base,end:t});if(n==this.completions.filterText)return;this.completions.setFilter(n);if(!this.completions.filtered.length)return this.detach();if(this.completions.filtered.length==1&&this.completions.filtered[0].value==n&&!this.completions.filtered[0].snippet)return this.detach();this.openPopup(this.editor,n,e);return}var r=this.gatherCompletionsId;this.gatherCompletions(this.editor,function(t,n){var i=function(){if(!n.finished)return;return this.detach()}.bind(this),s=n.prefix,o=n&&n.matches;if(!o||!o.length)return i();if(s.indexOf(n.prefix)!==0||r!=this.gatherCompletionsId)return;this.completions=new c(o),this.exactMatch&&(this.completions.exactMatch=!0),this.completions.setFilter(s);var u=this.completions.filtered;if(!u.length)return i()
 ;if(u.length==1&&u[0].value==s&&!u[0].snippet)return i();if(this.autoInsert&&u.length==1&&n.finished)return this.insertMatch(u[0]);this.openPopup(this.editor,s,e)}.bind(this))},this.cancelContextMenu=function(){this.editor.$mouseHandler.cancelContextMenu()},this.updateDocTooltip=function(){var e=this.popup,t=e.data,n=t&&(t[e.getHoveredRow()]||t[e.getRow()]),r=null;if(!n||!this.editor||!this.popup.isOpen)return this.hideDocTooltip();this.editor.completers.some(function(e){return e.getDocTooltip&&(r=e.getDocTooltip(n)),r}),r||(r=n),typeof r=="string"&&(r={docText:r});if(!r||!r.docHTML&&!r.docText)return this.hideDocTooltip();this.showDocTooltip(r)},this.showDocTooltip=function(e){this.tooltipNode||(this.tooltipNode=a.createElement("div"),this.tooltipNode.className="ace_tooltip ace_doc-tooltip",this.tooltipNode.style.margin=0,this.tooltipNode.style.pointerEvents="auto",this.tooltipNode.tabIndex=-1,this.tooltipNode.onblur=this.blurListener.bind(this),this.tooltipNode.onclick=this.onTool
 tipClick.bind(this));var t=this.tooltipNode;e.docHTML?t.innerHTML=e.docHTML:e.docText&&(t.textContent=e.docText),t.parentNode||document.body.appendChild(t);var n=this.popup,r=n.container.getBoundingClientRect();t.style.top=n.container.style.top,t.style.bottom=n.container.style.bottom,window.innerWidth-r.right<320?(t.style.right=window.innerWidth-r.left+"px",t.style.left=""):(t.style.left=r.right+1+"px",t.style.right=""),t.style.display="block"},this.hideDocTooltip=function(){this.tooltipTimer.cancel();if(!this.tooltipNode)return;var e=this.tooltipNode;!this.editor.isFocused()&&document.activeElement==e&&this.editor.focus(),this.tooltipNode=null,e.parentNode&&e.parentNode.removeChild(e)},this.onTooltipClick=function(e){var t=e.target;while(t&&t!=this.tooltipNode){if(t.nodeName=="A"&&t.href){t.rel="noreferrer",t.target="_blank";break}t=t.parentNode}}}).call(l.prototype),l.startCommand={name:"startAutocomplete",exec:function(e){e.completer||(e.completer=new l),e.completer.autoInsert=!1
 ,e.completer.autoSelect=!0,e.completer.showPopup(e),e.completer.cancelContextMenu()},bindKey:"Ctrl-Space|Ctrl-Shift-Space|Alt-Space"};var c=function(e,t){this.all=e,this.filtered=e,this.filterText=t||"",this.exactMatch=!1};(function(){this.setFilter=function(e){if(e.length>this.filterText&&e.lastIndexOf(this.filterText,0)===0)var t=this.filtered;else var t=this.all;this.filterText=e,t=this.filterCompletions(t,this.filterText),t=t.sort(function(e,t){return t.exactMatch-e.exactMatch||t.score-e.score});var n=null;t=t.filter(function(e){var t=e.snippet||e.caption||e.value;return t===n?!1:(n=t,!0)}),this.filtered=t},this.filterCompletions=function(e,t){var n=[],r=t.toUpperCase(),i=t.toLowerCase();e:for(var s=0,o;o=e[s];s++){var u=o.value||o.caption||o.snippet;if(!u)continue;var a=-1,f=0,l=0,c,h;if(this.exactMatch){if(t!==u.substr(0,t.length))continue e}else for(var p=0;p<t.length;p++){var d=u.indexOf(i[p],a+1),v=u.indexOf(r[p],a+1);c=d>=0?v<0||d<v?d:v:v;if(c<0)continue e;h=c-a-1,h>0&&(a=
 ==-1&&(l+=10),l+=h),f|=1<<c,a=c}o.matchMask=f,o.exactMatch=l?0:1,o.score=(o.score||0)-l,n.push(o)}return n}}).call(c.prototype),t.Autocomplete=l,t.FilteredList=c}),ace.define("ace/autocomplete/text_completer",["require","exports","module","ace/range"],function(e,t,n){function s(e,t){var n=e.getTextRange(r.fromPoints({row:0,column:0},t));return n.split(i).length-1}function o(e,t){var n=s(e,t),r=e.getValue().split(i),o=Object.create(null),u=r[n];return r.forEach(function(e,t){if(!e||e===u)return;var i=Math.abs(n-t),s=r.length-i;o[e]?o[e]=Math.max(s,o[e]):o[e]=s}),o}var r=e("../range").Range,i=/[^a-zA-Z_0-9\$\-\u00C0-\u1FFF\u2C00-\uD7FF\w]+/;t.getCompletions=function(e,t,n,r,i){var s=o(t,n,r),u=Object.keys(s);i(null,u.map(function(e){return{caption:e,value:e,score:s[e],meta:"local"}}))}}),ace.define("ace/ext/language_tools",["require","exports","module","ace/snippets","ace/autocomplete","ace/config","ace/lib/lang","ace/autocomplete/util","ace/autocomplete/text_completer","ace/editor","
 ace/config"],function(e,t,n){"use strict";var r=e("../snippets").snippetManager,i=e("../autocomplete").Autocomplete,s=e("../config"),o=e("../lib/lang"),u=e("../autocomplete/util"),a=e("../autocomplete/text_completer"),f={getCompletions:function(e,t,n,r,i){if(t.$mode.completer)return t.$mode.completer.getCompletions(e,t,n,r,i);var s=e.session.getState(n.row),o=t.$mode.getCompletions(s,t,n,r);i(null,o)}},l={getCompletions:function(e,t,n,i,s){var o=r.snippetMap,u=[];r.getActiveScopes(e).forEach(function(e){var t=o[e]||[];for(var n=t.length;n--;){var r=t[n],i=r.name||r.tabTrigger;if(!i)continue;u.push({caption:i,snippet:r.content,meta:r.tabTrigger&&!r.name?r.tabTrigger+"\u21e5 ":"snippet",type:"snippet"})}},this),s(null,u)},getDocTooltip:function(e){e.type=="snippet"&&!e.docHTML&&(e.docHTML=["<b>",o.escapeHTML(e.caption),"</b>","<hr></hr>",o.escapeHTML(e.snippet)].join(""))}},c=[l,a,f];t.setCompleters=function(e){c.length=0,e&&c.push.apply(c,e)},t.addCompleter=function(e){c.push(e)},t.t
 extCompleter=a,t.keyWordCompleter=f,t.snippetCompleter=l;var h={name:"expandSnippet",exec:function(e){return r.expandWithTab(e)},bindKey:"Tab"},p=function(e,t){d(t.session.$mode)},d=function(e){var t=e.$id;r.files||(r.files={}),v(t),e.modes&&e.modes.forEach(d)},v=function(e){if(!e||r.files[e])return;var t=e.replace("mode","snippets");r.files[e]={},s.loadModule(t,function(t){t&&(r.files[e]=t,!t.snippets&&t.snippetText&&(t.snippets=r.parseSnippetFile(t.snippetText)),r.register(t.snippets||[],t.scope),t.includeScopes&&(r.snippetMap[t.scope].includeScopes=t.includeScopes,t.includeScopes.forEach(function(e){v("ace/mode/"+e)})))})},m=function(e){var t=e.editor,n=t.completer&&t.completer.activated;if(e.command.name==="backspace")n&&!u.getCompletionPrefix(t)&&t.completer.detach();else if(e.command.name==="insertstring"){var r=u.getCompletionPrefix(t);r&&!n&&(t.completer||(t.completer=new i),t.completer.autoInsert=!1,t.completer.showPopup(t))}},g=e("../editor").Editor;e("../config").defineOp
 tions(g.prototype,"editor",{enableBasicAutocompletion:{set:function(e){e?(this.completers||(this.completers=Array.isArray(e)?e:c),this.commands.addCommand(i.startCommand)):this.commands.removeCommand(i.startCommand)},value:!1},enableLiveAutocompletion:{set:function(e){e?(this.completers||(this.completers=Array.isArray(e)?e:c),this.commands.on("afterExec",m)):this.commands.removeListener("afterExec",m)},value:!1},enableSnippets:{set:function(e){e?(this.commands.addCommand(h),this.on("changeMode",p),p(null,this)):(this.commands.removeCommand(h),this.off("changeMode",p))},value:!1}})});
+                (function() {
+                    ace.require(["ace/ext/language_tools"], function() {});
+                })();
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-snippets.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-snippets.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-snippets.js
new file mode 100644
index 0000000..0195feb
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-snippets.js
@@ -0,0 +1,199 @@
+ace.define("ace/mode/folding/coffee",["require","exports","module","ace/lib/oop","ace/mode/folding/fold_mode","ace/range"], function(require, exports, module) {
+"use strict";
+
+var oop = require("../../lib/oop");
+var BaseFoldMode = require("./fold_mode").FoldMode;
+var Range = require("../../range").Range;
+
+var FoldMode = exports.FoldMode = function() {};
+oop.inherits(FoldMode, BaseFoldMode);
+
+(function() {
+
+    this.getFoldWidgetRange = function(session, foldStyle, row) {
+        var range = this.indentationBlock(session, row);
+        if (range)
+            return range;
+
+        var re = /\S/;
+        var line = session.getLine(row);
+        var startLevel = line.search(re);
+        if (startLevel == -1 || line[startLevel] != "#")
+            return;
+
+        var startColumn = line.length;
+        var maxRow = session.getLength();
+        var startRow = row;
+        var endRow = row;
+
+        while (++row < maxRow) {
+            line = session.getLine(row);
+            var level = line.search(re);
+
+            if (level == -1)
+                continue;
+
+            if (line[level] != "#")
+                break;
+
+            endRow = row;
+        }
+
+        if (endRow > startRow) {
+            var endColumn = session.getLine(endRow).length;
+            return new Range(startRow, startColumn, endRow, endColumn);
+        }
+    };
+    this.getFoldWidget = function(session, foldStyle, row) {
+        var line = session.getLine(row);
+        var indent = line.search(/\S/);
+        var next = session.getLine(row + 1);
+        var prev = session.getLine(row - 1);
+        var prevIndent = prev.search(/\S/);
+        var nextIndent = next.search(/\S/);
+
+        if (indent == -1) {
+            session.foldWidgets[row - 1] = prevIndent!= -1 && prevIndent < nextIndent ? "start" : "";
+            return "";
+        }
+        if (prevIndent == -1) {
+            if (indent == nextIndent && line[indent] == "#" && next[indent] == "#") {
+                session.foldWidgets[row - 1] = "";
+                session.foldWidgets[row + 1] = "";
+                return "start";
+            }
+        } else if (prevIndent == indent && line[indent] == "#" && prev[indent] == "#") {
+            if (session.getLine(row - 2).search(/\S/) == -1) {
+                session.foldWidgets[row - 1] = "start";
+                session.foldWidgets[row + 1] = "";
+                return "";
+            }
+        }
+
+        if (prevIndent!= -1 && prevIndent < indent)
+            session.foldWidgets[row - 1] = "start";
+        else
+            session.foldWidgets[row - 1] = "";
+
+        if (indent < nextIndent)
+            return "start";
+        else
+            return "";
+    };
+
+}).call(FoldMode.prototype);
+
+});
+
+ace.define("ace/mode/snippets",["require","exports","module","ace/lib/oop","ace/mode/text","ace/mode/text_highlight_rules","ace/mode/folding/coffee"], function(require, exports, module) {
+"use strict";
+
+var oop = require("../lib/oop");
+var TextMode = require("./text").Mode;
+var TextHighlightRules = require("./text_highlight_rules").TextHighlightRules;
+
+var SnippetHighlightRules = function() {
+
+    var builtins = "SELECTION|CURRENT_WORD|SELECTED_TEXT|CURRENT_LINE|LINE_INDEX|" +
+        "LINE_NUMBER|SOFT_TABS|TAB_SIZE|FILENAME|FILEPATH|FULLNAME";
+
+    this.$rules = {
+        "start" : [
+            {token:"constant.language.escape", regex: /\\[\$}`\\]/},
+            {token:"keyword", regex: "\\$(?:TM_)?(?:" + builtins + ")\\b"},
+            {token:"variable", regex: "\\$\\w+"},
+            {onMatch: function(value, state, stack) {
+                if (stack[1])
+                    stack[1]++;
+                else
+                    stack.unshift(state, 1);
+                return this.tokenName;
+            }, tokenName: "markup.list", regex: "\\${", next: "varDecl"},
+            {onMatch: function(value, state, stack) {
+                if (!stack[1])
+                    return "text";
+                stack[1]--;
+                if (!stack[1])
+                    stack.splice(0,2);
+                return this.tokenName;
+            }, tokenName: "markup.list", regex: "}"},
+            {token: "doc.comment", regex:/^\${2}-{5,}$/}
+        ],
+        "varDecl" : [
+            {regex: /\d+\b/, token: "constant.numeric"},
+            {token:"keyword", regex: "(?:TM_)?(?:" + builtins + ")\\b"},
+            {token:"variable", regex: "\\w+"},
+            {regex: /:/, token: "punctuation.operator", next: "start"},
+            {regex: /\//, token: "string.regex", next: "regexp"},
+            {regex: "", next: "start"}
+        ],
+        "regexp" : [
+            {regex: /\\./, token: "escape"},
+            {regex: /\[/, token: "regex.start", next: "charClass"},
+            {regex: "/", token: "string.regex", next: "format"},
+            {"token": "string.regex", regex:"."}
+        ],
+        charClass : [
+            {regex: "\\.", token: "escape"},
+            {regex: "\\]", token: "regex.end", next: "regexp"},
+            {"token": "string.regex", regex:"."}
+        ],
+        "format" : [
+            {regex: /\\[ulULE]/, token: "keyword"},
+            {regex: /\$\d+/, token: "variable"},
+            {regex: "/[gim]*:?", token: "string.regex", next: "start"},
+            {"token": "string", regex:"."}
+        ]
+    };
+};
+oop.inherits(SnippetHighlightRules, TextHighlightRules);
+
+exports.SnippetHighlightRules = SnippetHighlightRules;
+
+var SnippetGroupHighlightRules = function() {
+    this.$rules = {
+        "start" : [
+			{token: "text", regex: "^\\t", next: "sn-start"},
+			{token:"invalid", regex: /^ \s*/},
+            {token:"comment", regex: /^#.*/},
+            {token:"constant.language.escape", regex: "^regex ", next: "regex"},
+            {token:"constant.language.escape", regex: "^(trigger|endTrigger|name|snippet|guard|endGuard|tabTrigger|key)\\b"}
+        ],
+		"regex" : [
+			{token:"text", regex: "\\."},
+			{token:"keyword", regex: "/"},
+			{token:"empty", regex: "$", next: "start"}
+		]
+    };
+	this.embedRules(SnippetHighlightRules, "sn-", [
+		{token: "text", regex: "^\\t", next: "sn-start"},
+		{onMatch: function(value, state, stack) {
+			stack.splice(stack.length);
+			return this.tokenName;
+		}, tokenName: "text", regex: "^(?!\t)", next: "start"}
+	]);
+	
+};
+
+oop.inherits(SnippetGroupHighlightRules, TextHighlightRules);
+
+exports.SnippetGroupHighlightRules = SnippetGroupHighlightRules;
+
+var FoldMode = require("./folding/coffee").FoldMode;
+
+var Mode = function() {
+    this.HighlightRules = SnippetGroupHighlightRules;
+    this.foldingRules = new FoldMode();
+    this.$behaviour = this.$defaultBehaviour;
+};
+oop.inherits(Mode, TextMode);
+
+(function() {
+    this.$indentWithTabs = true;
+    this.lineCommentStart = "#";
+    this.$id = "ace/mode/snippets";
+}).call(Mode.prototype);
+exports.Mode = Mode;
+
+
+});

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-sql.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-sql.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-sql.js
index c4dadd8..992fdb6 100644
--- a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-sql.js
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/mode-sql.js
@@ -1 +1,134 @@
-ace.define("ace/mode/sql_highlight_rules",["require","exports","module","ace/lib/oop","ace/mode/text_highlight_rules"],function(e,t,n){"use strict";var r=e("../lib/oop"),i=e("./text_highlight_rules").TextHighlightRules,s=function(){var e="select|insert|update|delete|from|where|and|or|group|by|order|limit|offset|having|as|case|when|else|end|type|left|right|join|on|outer|desc|asc|union|create|table|primary|key|if|foreign|not|references|default|null|inner|cross|natural|database|drop|grant",t="true|false",n="avg|count|first|last|max|min|sum|ucase|lcase|mid|len|round|rank|now|format|coalesce|ifnull|isnull|nvl",r="int|numeric|decimal|date|varchar|char|bigint|float|double|bit|binary|text|set|timestamp|money|real|number|integer",i=this.createKeywordMapper({"support.function":n,keyword:e,"constant.language":t,"storage.type":r},"identifier",!0);this.$rules={start:[{token:"comment",regex:"--.*$"},{token:"comment",start:"/\\*",end:"\\*/"},{token:"string",regex:'".*?"'},{token:"string",regex:"'.
 *?'"},{token:"string",regex:"`.*?`"},{token:"constant.numeric",regex:"[+-]?\\d+(?:(?:\\.\\d*)?(?:[eE][+-]?\\d+)?)?\\b"},{token:i,regex:"[a-zA-Z_$][a-zA-Z0-9_$]*\\b"},{token:"keyword.operator",regex:"\\+|\\-|\\/|\\/\\/|%|<@>|@>|<@|&|\\^|~|<|>|<=|=>|==|!=|<>|="},{token:"paren.lparen",regex:"[\\(]"},{token:"paren.rparen",regex:"[\\)]"},{token:"text",regex:"\\s+"}]},this.normalizeRules()};r.inherits(s,i),t.SqlHighlightRules=s}),ace.define("ace/mode/sql",["require","exports","module","ace/lib/oop","ace/mode/text","ace/mode/sql_highlight_rules"],function(e,t,n){"use strict";var r=e("../lib/oop"),i=e("./text").Mode,s=e("./sql_highlight_rules").SqlHighlightRules,o=function(){this.HighlightRules=s,this.$behaviour=this.$defaultBehaviour};r.inherits(o,i),function(){this.lineCommentStart="--",this.$id="ace/mode/sql"}.call(o.prototype),t.Mode=o})
\ No newline at end of file
+/**
+ * Drill SQL Definition (Forked from SqlServer definition)
+ */
+
+ace.define("ace/mode/sql_highlight_rules",["require","exports","module","ace/lib/oop","ace/mode/text_highlight_rules"], function(require, exports, module) {
+"use strict";
+
+var oop = require("../lib/oop");
+var TextHighlightRules = require("./text_highlight_rules").TextHighlightRules;
+
+var SqlHighlightRules = function() {
+
+    //TODO: https://drill.apache.org/docs/reserved-keywords/
+    //e.g. Cubing operators like ROLLUP are not listed
+    //Covered: https://drill.apache.org/docs/supported-sql-commands/
+    var keywords = (
+        "select|insert|update|delete|from|where|and|or|group|by|order|limit|offset|having|as|case|" +
+        "when|else|end|type|left|right|join|on|outer|desc|asc|union|create|table|key|if|" +
+        "not|default|null|inner|database|drop|" +
+        "flatten|kvgen|columns|" +
+        "set|reset|alter|session|system|" +
+        "temporary|function|using|jar|between|distinct" +
+        "partition|view|schema|files|" +
+        "explain|plan|with|without|implementation|" +
+        "show|describe|use"
+    );
+    //Confirmed to be UnSupported as of Drill-1.12.0
+    /* cross|natural|primary|foreign|references|grant */
+
+    var builtinConstants = (
+        "true|false"
+    );
+
+    //Drill-specific
+    var builtinFunctions = (
+        //Math and Trignometric
+        "abs|cbrt|ceil|ceiling|degrees|e|exp|floor|log|log|log10|lshift|mod|negative|pi|pow|radians|rand|round|round|rshift|sign|sqrt|trunc|trunc|" +
+        "sin|cos|tan|asin|acos|atan|sinh|cosh|tanh|" +
+        //datatype conversion
+        "cast|convert_to|convert_from|string_binary|binary_string|" +
+        //time-conversion
+        "to_char|to_date|to_number|to_timestamp|to_timestamp|" +
+        "age|extract|current_date|current_time|current_timestamp|date_add|date_part|date_sub|localtime|localtimestamp|now|timeofday|unix_timestamp|" +
+        //string manipulation
+        "byte_substr|char_length|concat|ilike|initcap|length|lower|lpad|ltrim|position|regexp_replace|rpad|rtrim|strpos|substr|trim|upper|" +
+        //statistical
+        "avg|count|max|min|sum|stddev|stddev_pop|stddev_samp|variance|var_pop|var_samp|" +
+        //null-handling
+        "coalesce|nullif"
+    );
+
+    //Drill-specific
+    var dataTypes = (
+        "BIGINT|BINARY|BOOLEAN|CHAR|CHARACTER|DATE|DEC|DECIMAL|DOUBLE|FIXED16CHAR|FIXEDBINARY|FLOAT|INT|" +
+        "INTEGER|INTERVAL|INTERVALDAY|INTERVALYEAR|NUMERIC|NULL|SMALLINT|TIME|TIMESTAMP|VARBINARY|" +
+        "VAR16CHAR|VARCHAR");
+    //[Cannot supported due to space]
+    //DOUBLE PRECISION|CHARACTER VARYING;
+
+    var keywordMapper = this.createKeywordMapper({
+        "support.function": builtinFunctions,
+        "keyword": keywords,
+        "constant.language": builtinConstants,
+        "storage.type": dataTypes
+    }, "identifier", true);
+
+    this.$rules = {
+        "start" : [ {
+            token : "comment",
+            regex : "--.*$"
+        },  {
+            token : "comment",
+            start : "/\\*",
+            end : "\\*/"
+        }, {
+            token : "string",           // " string
+            regex : '".*?"'
+        }, {
+            token : "string",           // ' string
+            regex : "'.*?'"
+        }, {
+            token : "string",           // ` string (apache drill)
+            regex : "`.*?`"
+        }, {
+            token : "constant.numeric", // float
+            regex : "[+-]?\\d+(?:(?:\\.\\d*)?(?:[eE][+-]?\\d+)?)?\\b"
+        }, {
+            token : keywordMapper,
+            regex : "[a-zA-Z_$][a-zA-Z0-9_$]*\\b"
+        }, {
+            token : "keyword.operator",
+            regex : "\\+|\\-|\\/|\\/\\/|%|<@>|@>|<@|&|\\^|~|<|>|<=|=>|==|!=|<>|="
+        }, {
+            token : "paren.lparen",
+            regex : "[\\(]"
+        }, {
+            token : "paren.rparen",
+            regex : "[\\)]"
+        }, {
+            token : "text",
+            regex : "\\s+"
+        } ]
+    };
+    this.normalizeRules();
+};
+
+oop.inherits(SqlHighlightRules, TextHighlightRules);
+
+exports.SqlHighlightRules = SqlHighlightRules;
+});
+
+ace.define("ace/mode/sql",["require","exports","module","ace/lib/oop","ace/mode/text","ace/mode/sql_highlight_rules"], function(require, exports, module) {
+"use strict";
+
+var oop = require("../lib/oop");
+var TextMode = require("./text").Mode;
+var SqlHighlightRules = require("./sql_highlight_rules").SqlHighlightRules;
+
+var Mode = function() {
+    this.HighlightRules = SqlHighlightRules;
+    this.$behaviour = this.$defaultBehaviour;
+};
+oop.inherits(Mode, TextMode);
+
+(function() {
+
+    this.lineCommentStart = "--";
+
+    this.$id = "ace/mode/sql";
+}).call(Mode.prototype);
+
+exports.Mode = Mode;
+
+});

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/snippets/sql.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/snippets/sql.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/snippets/sql.js
new file mode 100644
index 0000000..3c5d7de
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/snippets/sql.js
@@ -0,0 +1,48 @@
+/**
+ * Drill SQL Syntax Snippets
+ */
+
+ace.define("ace/snippets/sql",["require","exports","module"], function(require, exports, module) {
+"use strict";
+
+exports.snippetText = "snippet info\n\
+	select * from INFORMATION_SCHEMA.${1:<tableName>};\n\
+snippet sysmem\n\
+	select * from sys.memory;\n\
+snippet sysopt\n\
+	select * from sys.options;\n\
+snippet sysbit\n\
+	select * from sys.drillbits;\n\
+snippet sysconn\n\
+	select * from sys.connections;\n\
+snippet sysprof\n\
+	select * from sys.profiles;\n\
+snippet cview\n\
+	create view ${1:[workspace]}.${2:<viewName>} ( ${3:<columnName>} )  as \n\
+	${4:<query>};\n\
+snippet ctas\n\
+	create table ${1:<tableName>} ( ${2:<columnName>} )  as \n\
+	${3:<query>};\n\
+snippet ctemp\n\
+	create temporary table ${1:<tableName>} ( ${2:<columnName>} )  as \n\
+	${3:<query>};\n\
+snippet cfnjar\n\
+	create function using jar '${1:<jarName>}.jar';\n\
+snippet alt\n\
+	alter session set `${1:<parameter>}` = ${2:<value>};\n\
+snippet reset\n\
+	alter session reset `${1:<parameter>}`;\n\
+snippet explain\n\
+	explain plan for\n\
+	${1:<query>};\n\
+snippet s*\n\
+	select *\n\
+	from ${1:<tableName>}\n\
+	where ${2:<condition>};\n\
+snippet cast\n\
+	cast(${1:<columnName>} AS ${2:<dataType>}) ${3:<alias>}\n\
+";
+
+exports.scope = "sql";
+
+});

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/theme-sqlserver.js
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/theme-sqlserver.js b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/theme-sqlserver.js
new file mode 100644
index 0000000..91f34f6
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/static/js/ace-code-editor/theme-sqlserver.js
@@ -0,0 +1,138 @@
+ace.define("ace/theme/sqlserver",["require","exports","module","ace/lib/dom"], function(require, exports, module) {
+
+exports.isDark = false;
+exports.cssClass = "ace-sqlserver";
+exports.cssText = ".ace-sqlserver .ace_gutter {\
+background: #ebebeb;\
+color: #333;\
+overflow: hidden;\
+}\
+.ace-sqlserver .ace_print-margin {\
+width: 1px;\
+background: #e8e8e8;\
+}\
+.ace-sqlserver {\
+background-color: #FFFFFF;\
+color: black;\
+}\
+.ace-sqlserver .ace_identifier {\
+color: black;\
+}\
+.ace-sqlserver .ace_keyword {\
+color: #0000FF;\
+}\
+.ace-sqlserver .ace_numeric {\
+color: black;\
+}\
+.ace-sqlserver .ace_storage {\
+color: #11B7BE;\
+}\
+.ace-sqlserver .ace_keyword.ace_operator,\
+.ace-sqlserver .ace_lparen,\
+.ace-sqlserver .ace_rparen,\
+.ace-sqlserver .ace_punctuation {\
+color: #808080;\
+}\
+.ace-sqlserver .ace_set.ace_statement {\
+color: #0000FF;\
+text-decoration: underline;\
+}\
+.ace-sqlserver .ace_cursor {\
+color: black;\
+}\
+.ace-sqlserver .ace_invisible {\
+color: rgb(191, 191, 191);\
+}\
+.ace-sqlserver .ace_constant.ace_buildin {\
+color: rgb(88, 72, 246);\
+}\
+.ace-sqlserver .ace_constant.ace_language {\
+color: #979797;\
+}\
+.ace-sqlserver .ace_constant.ace_library {\
+color: rgb(6, 150, 14);\
+}\
+.ace-sqlserver .ace_invalid {\
+background-color: rgb(153, 0, 0);\
+color: white;\
+}\
+.ace-sqlserver .ace_support.ace_function {\
+color: #FF00FF;\
+}\
+.ace-sqlserver .ace_support.ace_constant {\
+color: rgb(6, 150, 14);\
+}\
+.ace-sqlserver .ace_class {\
+color: #008080;\
+}\
+.ace-sqlserver .ace_support.ace_other {\
+color: #6D79DE;\
+}\
+.ace-sqlserver .ace_variable.ace_parameter {\
+font-style: italic;\
+color: #FD971F;\
+}\
+.ace-sqlserver .ace_comment {\
+color: #008000;\
+}\
+.ace-sqlserver .ace_constant.ace_numeric {\
+color: black;\
+}\
+.ace-sqlserver .ace_variable {\
+color: rgb(49, 132, 149);\
+}\
+.ace-sqlserver .ace_xml-pe {\
+color: rgb(104, 104, 91);\
+}\
+.ace-sqlserver .ace_support.ace_storedprocedure {\
+color: #800000;\
+}\
+.ace-sqlserver .ace_heading {\
+color: rgb(12, 7, 255);\
+}\
+.ace-sqlserver .ace_list {\
+color: rgb(185, 6, 144);\
+}\
+.ace-sqlserver .ace_marker-layer .ace_selection {\
+background: rgb(181, 213, 255);\
+}\
+.ace-sqlserver .ace_marker-layer .ace_step {\
+background: rgb(252, 255, 0);\
+}\
+.ace-sqlserver .ace_marker-layer .ace_stack {\
+background: rgb(164, 229, 101);\
+}\
+.ace-sqlserver .ace_marker-layer .ace_bracket {\
+margin: -1px 0 0 -1px;\
+border: 1px solid rgb(192, 192, 192);\
+}\
+.ace-sqlserver .ace_marker-layer .ace_active-line {\
+background: rgba(0, 0, 0, 0.07);\
+}\
+.ace-sqlserver .ace_gutter-active-line {\
+background-color: #dcdcdc;\
+}\
+.ace-sqlserver .ace_marker-layer .ace_selected-word {\
+background: rgb(250, 250, 255);\
+border: 1px solid rgb(200, 200, 250);\
+}\
+.ace-sqlserver .ace_meta.ace_tag {\
+color: #0000FF;\
+}\
+.ace-sqlserver .ace_string.ace_regex {\
+color: #FF0000;\
+}\
+.ace-sqlserver .ace_string {\
+color: #FF0000;\
+}\
+.ace-sqlserver .ace_entity.ace_other.ace_attribute-name {\
+color: #994409;\
+}\
+.ace-sqlserver .ace_indent-guide {\
+background: url(\"data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAAEAAAACCAYAAACZgbYnAAAAE0lEQVQImWP4////f4bLly//BwAmVgd1/w11/gAAAABJRU5ErkJggg==\") right repeat-y;\
+}\
+";
+
+var dom = require("../lib/dom");
+dom.importCssString(exports.cssText, exports.cssClass);
+});

http://git-wip-us.apache.org/repos/asf/drill/blob/4c8a55b8/exec/java-exec/src/main/resources/rest/storage/update.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/storage/update.ftl b/exec/java-exec/src/main/resources/rest/storage/update.ftl
index 377617a..b84bfa5 100644
--- a/exec/java-exec/src/main/resources/rest/storage/update.ftl
+++ b/exec/java-exec/src/main/resources/rest/storage/update.ftl
@@ -12,6 +12,10 @@
 <#include "*/generic.ftl">
 <#macro page_head>
   <script src="/static/js/jquery.form.js"></script>
+
+  <!-- Ace Libraries for Syntax Formatting -->
+  <script src="/static/js/ace-code-editor/ace.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/static/js/ace-code-editor/theme-eclipse.js" type="text/javascript" charset="utf-8"></script>
 </#macro>
 
 <#macro page_body>
@@ -42,8 +46,6 @@
   <br>
   <div id="message" class="hidden alert alert-info">
   </div>
-  <script src="/static/js/ace-code-editor/ace.js" type="text/javascript" charset="utf-8"></script>
-  <script src="/static/js/ace-code-editor/theme-eclipse.js" type="text/javascript" charset="utf-8"></script>
   <script>
     var editor = ace.edit("editor");
     var textarea = $('textarea[name="config"]');


[06/11] drill git commit: DRILL-6002: Avoid memory copy from direct buffer to heap while spilling to local disk

Posted by am...@apache.org.
DRILL-6002: Avoid memory copy from direct buffer to heap while spilling to local disk

close apache/drill#1058


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

Branch: refs/heads/master
Commit: d803f0c2188c679de3dacf10741005b217425a33
Parents: 2420b35
Author: Vlad Rozov <vr...@apache.org>
Authored: Wed Nov 22 14:06:13 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 17:45:50 2018 -0800

----------------------------------------------------------------------
 .../drill/exec/cache/VectorSerializer.java      | 107 +++++++++++++------
 .../impl/aggregate/HashAggTemplate.java         |  38 +++----
 .../exec/physical/impl/spill/SpillSet.java      |  99 +++++++++++++----
 .../physical/impl/xsort/managed/BatchGroup.java |  28 ++---
 .../impl/xsort/managed/SpilledRuns.java         |   2 +-
 .../exec/cache/TestBatchSerialization.java      |  34 ++++--
 6 files changed, 210 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
index eeef9e5..03ea11e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorSerializer.java
@@ -17,16 +17,30 @@
  */
 package org.apache.drill.exec.cache;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.metrics.DrillMetrics;
+import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+
+import io.netty.buffer.DrillBuf;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Serializes vector containers to an output stream or from
  * an input stream.
@@ -39,51 +53,84 @@ public class VectorSerializer {
    * objects to an output stream.
    */
 
-  public static class Writer {
+  public static class Writer implements Closeable
+  {
+    static final MetricRegistry metrics = DrillMetrics.getRegistry();
+    static final String WRITER_TIMER = MetricRegistry.name(VectorAccessibleSerializable.class, "writerTime");
 
-    private final OutputStream stream;
-    private final BufferAllocator allocator;
-    private boolean retain;
+    private final WritableByteChannel channel;
+    private final OutputStream output;
     private long timeNs;
+    private int bytesWritten;
 
-    public Writer(BufferAllocator allocator, OutputStream stream) {
-      this.allocator = allocator;
-      this.stream = stream;
-    }
-
-    public Writer retain() {
-      retain = true;
-      return this;
+    private Writer(WritableByteChannel channel) {
+      this.channel = channel;
+      output = Channels.newOutputStream(channel);
     }
 
-    public Writer write(VectorAccessible va) throws IOException {
+    public int write(VectorAccessible va) throws IOException {
       return write(va, null);
     }
 
     @SuppressWarnings("resource")
-    public Writer write(VectorAccessible va, SelectionVector2 sv2) throws IOException {
+    public int write(VectorAccessible va, SelectionVector2 sv2) throws IOException {
+      checkNotNull(va);
       WritableBatch batch = WritableBatch.getBatchNoHVWrap(
           va.getRecordCount(), va, sv2 != null);
-      return write(batch, sv2);
+      try {
+        return write(batch, sv2);
+      } finally {
+        batch.clear();
+      }
     }
 
-    public Writer write(WritableBatch batch, SelectionVector2 sv2) throws IOException {
-      VectorAccessibleSerializable vas;
-      if (sv2 == null) {
-        vas = new VectorAccessibleSerializable(batch, allocator);
-      } else {
-        vas = new VectorAccessibleSerializable(batch, sv2, allocator);
+    public int write(WritableBatch batch, SelectionVector2 sv2) throws IOException {
+      checkNotNull(batch);
+      checkNotNull(channel);
+      final Timer.Context timerContext = metrics.timer(WRITER_TIMER).time();
+
+      final DrillBuf[] incomingBuffers = batch.getBuffers();
+      final UserBitShared.RecordBatchDef batchDef = batch.getDef();
+      bytesWritten = batchDef.getSerializedSize();
+
+      /* Write the metadata to the file */
+      batchDef.writeDelimitedTo(output);
+
+      /* If we have a selection vector, dump it to file first */
+      if (sv2 != null) {
+        final int dataLength = sv2.getCount() * SelectionVector2.RECORD_SIZE;
+        ByteBuffer buffer = sv2.getBuffer(false).nioBuffer(0, dataLength);
+        while (buffer.remaining() > 0) {
+          bytesWritten += channel.write(buffer);
+        }
       }
-      if (retain) {
-        vas.writeToStreamAndRetain(stream);
-      } else {
-        vas.writeToStream(stream);
+
+      /* Dump the array of ByteBuf's associated with the value vectors */
+      for (DrillBuf buf : incomingBuffers) {
+        /* dump the buffer into the OutputStream */
+        ByteBuffer buffer = buf.nioBuffer();
+        while (buffer.remaining() > 0) {
+          bytesWritten += channel.write(buffer);
+        }
       }
-      timeNs += vas.getTimeNs();
-      return this;
+
+      timeNs += timerContext.stop();
+      return bytesWritten;
     }
 
-    public long timeNs() { return timeNs; }
+    @Override
+    public void close() throws IOException {
+      if (!channel.isOpen()) {
+        return;
+      }
+      channel.close();
+    }
+
+    public long time(TimeUnit unit) {
+      return unit.convert(timeNs, TimeUnit.NANOSECONDS);
+    }
+
+    public int getBytesWritten() { return bytesWritten; }
   }
 
   /**
@@ -111,8 +158,8 @@ public class VectorSerializer {
     public long timeNs() { return timeNs; }
   }
 
-  public static Writer writer(BufferAllocator allocator, OutputStream stream) {
-    return new Writer(allocator, stream);
+  public static Writer writer(WritableByteChannel channel) throws IOException {
+    return new Writer(channel);
   }
 
   public static Reader reader(BufferAllocator allocator, InputStream stream) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
----------------------------------------------------------------------
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 2f181fe..89ba59b 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
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.physical.impl.aggregate;
 
 import java.io.IOException;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -26,7 +25,6 @@ import java.util.concurrent.TimeUnit;
 
 import javax.inject.Named;
 
-import com.google.common.base.Stopwatch;
 import org.apache.drill.common.exceptions.RetryAfterSpillException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ExpressionPosition;
@@ -34,7 +32,7 @@ import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.cache.VectorSerializer.Writer;
 import org.apache.drill.exec.compile.sig.RuntimeOverridden;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
@@ -149,7 +147,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   // For handling spilling
   private SpillSet spillSet;
   SpilledRecordbatch newIncoming; // when reading a spilled file - work like an "incoming"
-  private OutputStream outputStream[]; // an output stream for each spilled partition
+  private Writer writers[]; // a vector writer for each spilled partition
   private int spilledBatchesCount[]; // count number of batches spilled, in each partition
   private String spillFiles[];
   private int cycleNum = 0; // primary, secondary, tertiary, etc.
@@ -454,7 +452,7 @@ public abstract class HashAggTemplate implements HashAggregator {
     htables = new HashTable[numPartitions] ;
     batchHolders = (ArrayList<BatchHolder>[]) new ArrayList<?>[numPartitions] ;
     outBatchIndex = new int[numPartitions] ;
-    outputStream = new OutputStream[numPartitions];
+    writers = new Writer[numPartitions];
     spilledBatchesCount = new int[numPartitions];
     spillFiles = new String[numPartitions];
     spilledPartitionsList = new ArrayList<SpilledPartition>();
@@ -504,7 +502,7 @@ public abstract class HashAggTemplate implements HashAggregator {
         batchHolders[i] = new ArrayList<BatchHolder>();
       }
       outBatchIndex[i] = 0;
-      outputStream[i] = null;
+      writers[i] = null;
       spilledBatchesCount[i] = 0;
       spillFiles[i] = null;
     }
@@ -792,14 +790,14 @@ public abstract class HashAggTemplate implements HashAggregator {
           }
 
           // delete any (still active) output spill file
-          if ( outputStream[i] != null && spillFiles[i] != null) {
+          if ( writers[i] != null && spillFiles[i] != null) {
             try {
-              outputStream[i].close();
-              outputStream[i] = null;
+              spillSet.close(writers[i]);
+              writers[i] = null;
               spillSet.delete(spillFiles[i]);
               spillFiles[i] = null;
             } catch(IOException e) {
-              logger.warn("Cleanup: Failed to delete spill file {}",spillFiles[i]);
+              logger.warn("Cleanup: Failed to delete spill file {}", spillFiles[i], e);
             }
           }
     }
@@ -854,7 +852,7 @@ public abstract class HashAggTemplate implements HashAggregator {
   }
 
   private boolean isSpilled(int part) {
-    return outputStream[part] != null;
+    return writers[part] != null;
   }
   /**
    * Which partition to choose for flushing out (i.e. spill or return) ?
@@ -932,7 +930,7 @@ public abstract class HashAggTemplate implements HashAggregator {
       spillFiles[part] = spillSet.getNextSpillFile(cycleNum > 0 ? Integer.toString(cycleNum) : null);
 
       try {
-        outputStream[part] = spillSet.openForOutput(spillFiles[part]);
+        writers[part] = spillSet.writer(spillFiles[part]);
       } catch (IOException ioe) {
         throw UserException.resourceError(ioe)
             .message("Hash Aggregation failed to open spill file: " + spillFiles[part])
@@ -975,17 +973,17 @@ public abstract class HashAggTemplate implements HashAggregator {
 
       outContainer.setRecordCount(numPendingOutput);
       WritableBatch batch = WritableBatch.getBatchNoHVWrap(numPendingOutput, outContainer, false);
-      VectorAccessibleSerializable outputBatch = new VectorAccessibleSerializable(batch, allocator);
-      Stopwatch watch = Stopwatch.createStarted();
       try {
-        outputBatch.writeToStream(outputStream[part]);
+        writers[part].write(batch, null);
       } catch (IOException ioe) {
         throw UserException.dataWriteError(ioe)
-            .message("Hash Aggregation failed to write to output stream: " + outputStream[part].toString())
+            .message("Hash Aggregation failed to write to output file: " + spillFiles[part])
             .build(logger);
+      } finally {
+        batch.clear();
       }
       outContainer.zeroVectors();
-      logger.trace("HASH AGG: Took {} us to spill {} records", watch.elapsed(TimeUnit.MICROSECONDS), numPendingOutput);
+      logger.trace("HASH AGG: Took {} us to spill {} records", writers[part].time(TimeUnit.MICROSECONDS), numPendingOutput);
     }
 
     spilledBatchesCount[part] += currPartition.size(); // update count of spilled batches
@@ -1048,16 +1046,14 @@ public abstract class HashAggTemplate implements HashAggregator {
           spilledPartitionsList.add(sp);
 
           reinitPartition(nextPartitionToReturn); // free the memory
-          long posn = spillSet.getPosition(outputStream[nextPartitionToReturn]);
-          spillSet.tallyWriteBytes(posn); // for the IO stats
           try {
-            outputStream[nextPartitionToReturn].close();
+            spillSet.close(writers[nextPartitionToReturn]);
           } catch (IOException ioe) {
             throw UserException.resourceError(ioe)
                 .message("IO Error while closing output stream")
                 .build(logger);
           }
-          outputStream[nextPartitionToReturn] = null;
+          writers[nextPartitionToReturn] = null;
         }
         else {
           currPartition = batchHolders[nextPartitionToReturn];

http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
index 9a6420a..2f9ab14 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/spill/SpillSet.java
@@ -18,13 +18,15 @@
 package org.apache.drill.exec.physical.impl.spill;
 
 import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.StandardOpenOption;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
@@ -32,6 +34,7 @@ import java.util.Set;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.VectorSerializer;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashAggregate;
@@ -65,7 +68,7 @@ public class SpillSet {
 
     void deleteOnExit(String fragmentSpillDir) throws IOException;
 
-    OutputStream createForWrite(String fileName) throws IOException;
+    WritableByteChannel createForWrite(String fileName) throws IOException;
 
     InputStream openForInput(String fileName) throws IOException;
 
@@ -77,10 +80,10 @@ public class SpillSet {
      * Given a manager-specific output stream, return the current write position.
      * Used to report total write bytes.
      *
-     * @param outputStream output stream created by the file manager
+     * @param channel created by the file manager
      * @return
      */
-    long getWriteBytes(OutputStream outputStream);
+    long getWriteBytes(WritableByteChannel channel);
 
     /**
      * Given a manager-specific input stream, return the current read position.
@@ -104,9 +107,17 @@ public class SpillSet {
      * nodes provide insufficient local disk space)
      */
 
+    // The buffer size is calculated as LCM of the Hadoop internal checksum buffer (9 * checksum length), where
+    // checksum length is 512 by default, and MapRFS page size that equals to 8 * 1024. The length of the transfer
+    // buffer does not affect performance of the write to hdfs or maprfs significantly once buffer length is more
+    // than 32 bytes.
+    private static final int TRANSFER_SIZE = 9 * 8 * 1024;
+
+    private final byte buffer[];
     private FileSystem fs;
 
     protected HadoopFileManager(String fsName) {
+      buffer = new byte[TRANSFER_SIZE];
       Configuration conf = new Configuration();
       conf.set(FileSystem.FS_DEFAULT_NAME_KEY, fsName);
       try {
@@ -124,8 +135,8 @@ public class SpillSet {
     }
 
     @Override
-    public OutputStream createForWrite(String fileName) throws IOException {
-      return fs.create(new Path(fileName));
+    public WritableByteChannel createForWrite(String fileName) throws IOException {
+      return new WritableByteChannelImpl(buffer, fs.create(new Path(fileName)));
     }
 
     @Override
@@ -152,10 +163,10 @@ public class SpillSet {
     }
 
     @Override
-    public long getWriteBytes(OutputStream outputStream) {
+    public long getWriteBytes(WritableByteChannel channel) {
       try {
-        return ((FSDataOutputStream) outputStream).getPos();
-      } catch (IOException e) {
+        return ((FSDataOutputStream)((WritableByteChannelImpl)channel).out).getPos();
+      } catch (Exception e) {
         // Just used for logging, not worth dealing with the exception.
         return 0;
       }
@@ -295,10 +306,8 @@ public class SpillSet {
 
     @SuppressWarnings("resource")
     @Override
-    public OutputStream createForWrite(String fileName) throws IOException {
-      return new CountingOutputStream(
-                new BufferedOutputStream(
-                    new FileOutputStream(new File(baseDir, fileName))));
+    public WritableByteChannel createForWrite(String fileName) throws IOException {
+      return FileChannel.open(new File(baseDir, fileName).toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE);
     }
 
     @SuppressWarnings("resource")
@@ -321,8 +330,13 @@ public class SpillSet {
     }
 
     @Override
-    public long getWriteBytes(OutputStream outputStream) {
-      return ((CountingOutputStream) outputStream).getCount();
+    public long getWriteBytes(WritableByteChannel channel)
+    {
+      try {
+        return ((FileChannel)channel).position();
+      } catch (Exception e) {
+        return 0;
+      }
     }
 
     @Override
@@ -331,6 +345,44 @@ public class SpillSet {
     }
   }
 
+  private static class WritableByteChannelImpl implements WritableByteChannel
+  {
+    private final byte buffer[];
+    private OutputStream out;
+
+    WritableByteChannelImpl(byte[] buffer, OutputStream out) {
+      this.buffer = buffer;
+      this.out = out;
+    }
+
+    @Override
+    public int write(ByteBuffer src) throws IOException {
+      int remaining = src.remaining();
+      int totalWritten = 0;
+      synchronized (buffer) {
+        for (int posn = 0; posn < remaining; posn += buffer.length) {
+          int len = Math.min(buffer.length, remaining - posn);
+          src.get(buffer, 0, len);
+          out.write(buffer, 0, len);
+          totalWritten += len;
+        }
+      }
+      return totalWritten;
+    }
+
+    @Override
+    public boolean isOpen()
+    {
+      return out != null;
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+      out = null;
+    }
+  }
+
   private final Iterator<String> dirs;
 
   /**
@@ -457,7 +509,7 @@ public class SpillSet {
     return fileManager.openForInput(fileName);
   }
 
-  public OutputStream openForOutput(String fileName) throws IOException {
+  public WritableByteChannel openForOutput(String fileName) throws IOException {
     return fileManager.createForWrite(fileName);
   }
 
@@ -484,8 +536,8 @@ public class SpillSet {
     return fileManager.getReadBytes(inputStream);
   }
 
-  public long getPosition(OutputStream outputStream) {
-    return fileManager.getWriteBytes(outputStream);
+  public long getPosition(WritableByteChannel channel) {
+    return fileManager.getWriteBytes(channel);
   }
 
   public void tallyReadBytes(long readLength) {
@@ -495,4 +547,13 @@ public class SpillSet {
   public void tallyWriteBytes(long writeLength) {
     writeBytes += writeLength;
   }
+
+  public VectorSerializer.Writer writer(String fileName) throws IOException {
+    return VectorSerializer.writer(openForOutput(fileName));
+  }
+
+  public void close(VectorSerializer.Writer writer) throws IOException {
+    tallyWriteBytes(writer.getBytesWritten());
+    writer.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
index d902e0d..bd2e368 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/BatchGroup.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.physical.impl.xsort.managed;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.concurrent.TimeUnit;
@@ -27,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.cache.VectorSerializer;
+import org.apache.drill.exec.cache.VectorSerializer.Writer;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.spill.SpillSet;
 import org.apache.drill.exec.record.BatchSchema;
@@ -139,13 +139,12 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
 
   public static class SpilledRun extends BatchGroup {
     private InputStream inputStream;
-    private OutputStream outputStream;
     private String path;
     private SpillSet spillSet;
     private BufferAllocator allocator;
     private int spilledBatches;
     private long batchSize;
-    private VectorSerializer.Writer writer;
+    private Writer writer;
     private VectorSerializer.Reader reader;
 
     public SpilledRun(SpillSet spillSet, String path, BufferAllocator allocator) throws IOException {
@@ -153,15 +152,13 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
       this.spillSet = spillSet;
       this.path = path;
       this.allocator = allocator;
-      outputStream = spillSet.openForOutput(path);
-      writer = VectorSerializer.writer(allocator, outputStream);
+      writer = spillSet.writer(path);
     }
 
     public void addBatch(VectorContainer newContainer) throws IOException {
-      Stopwatch watch = Stopwatch.createStarted();
       writer.write(newContainer);
       newContainer.zeroVectors();
-      logger.trace("Wrote {} records in {} us", newContainer.getRecordCount(), watch.elapsed(TimeUnit.MICROSECONDS));
+      logger.trace("Wrote {} records in {} us", newContainer.getRecordCount(), writer.time(TimeUnit.MICROSECONDS));
       spilledBatches++;
 
       // Hold onto the husk of the last added container so that we have a
@@ -249,7 +246,7 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
         ex = e;
       }
       try {
-        closeOutputStream();
+        closeWriter();
       } catch (IOException e) {
         ex = ex == null ? e : ex;
       }
@@ -280,17 +277,12 @@ public abstract class BatchGroup implements VectorAccessible, AutoCloseable {
       logger.trace("Summary: Read {} bytes from {}", readLength, path);
     }
 
-    public long closeOutputStream() throws IOException {
-      if (outputStream == null) {
-        return 0;
+    public void closeWriter() throws IOException {
+      if (writer != null) {
+        spillSet.close(writer);
+        logger.trace("Summary: Wrote {} bytes in {} us to {}", writer.getBytesWritten(), writer.time(TimeUnit.MICROSECONDS), path);
+        writer = null;
       }
-      long writeSize = spillSet.getPosition(outputStream);
-      spillSet.tallyWriteBytes(writeSize);
-      outputStream.close();
-      outputStream = null;
-      writer = null;
-      logger.trace("Summary: Wrote {} bytes to {}", writeSize, path);
-      return writeSize;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
index 3d7e63a..bbf4457 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SpilledRuns.java
@@ -178,7 +178,7 @@ public class SpilledRuns {
         newGroup.addBatch(dest);
       }
       context.injectChecked(ExternalSortBatch.INTERRUPTION_WHILE_SPILLING, IOException.class);
-      newGroup.closeOutputStream();
+      newGroup.closeWriter();
       logger.trace("Spilled {} output batches, each of {} bytes, {} records, to {}",
                    merger.getBatchCount(), merger.getEstBatchSize(),
                    spillBatchRowCount, outputFile);

http://git-wip-us.apache.org/repos/asf/drill/blob/d803f0c2/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
index a283924..bcf0618 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -18,17 +18,19 @@
 package org.apache.drill.exec.cache;
 
 import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
+import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.cache.VectorSerializer.Reader;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.cache.VectorSerializer.Reader;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.test.DirTestWatcher;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.RowSet;
@@ -40,10 +42,15 @@ import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.SchemaBuilder;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 
+import static org.junit.Assert.assertTrue;
+
 public class TestBatchSerialization extends DrillTest {
 
+  @ClassRule
+  public static final DirTestWatcher dirTestWatcher = new DirTestWatcher();
   public static OperatorFixture fixture;
 
   @BeforeClass
@@ -117,12 +124,21 @@ public class TestBatchSerialization extends DrillTest {
    */
   private void verifySerialize(SingleRowSet rowSet, SingleRowSet expected) throws IOException {
 
-    File dir = OperatorFixture.getTempDir("serial");
-    File outFile = new File(dir, "serialze.dat");
-    try (OutputStream out = new BufferedOutputStream(new FileOutputStream(outFile))) {
-      VectorSerializer.writer(fixture.allocator(), out)
-        .write(rowSet.container(), rowSet.getSv2());
+    File dir = DirTestWatcher.createTempDir(dirTestWatcher.getDir());
+    FileChannel channel = FileChannel.open(new File(dir, "serialize.dat").toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE);
+    VectorSerializer.Writer writer = VectorSerializer.writer(channel);
+    VectorContainer container = rowSet.container();
+    SelectionVector2 sv2 = rowSet.getSv2();
+    writer.write(container, sv2);
+    container.clear();
+    if (sv2 != null) {
+      sv2.clear();
     }
+    writer.close();
+
+    File outFile = new File(dir, "serialize.dat");
+    assertTrue(outFile.exists());
+    assertTrue(outFile.isFile());
 
     RowSet result;
     try (InputStream in = new BufferedInputStream(new FileInputStream(outFile))) {


[07/11] drill git commit: DRILL-6049: Misc. hygiene and code cleanup changes

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
index 4a501b8..9a7e847 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
@@ -49,7 +49,6 @@ import com.google.common.collect.Ordering;
 import com.google.common.primitives.Ints;
 
 public class MapVector extends AbstractMapVector {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class);
 
   public final static MajorType TYPE = Types.required(MinorType.MAP);
 
@@ -58,11 +57,11 @@ public class MapVector extends AbstractMapVector {
   private final Mutator mutator = new Mutator();
   private int valueCount;
 
-  public MapVector(String path, BufferAllocator allocator, CallBack callBack){
+  public MapVector(String path, BufferAllocator allocator, CallBack callBack) {
     this(MaterializedField.create(path, TYPE), allocator, callBack);
   }
 
-  public MapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack){
+  public MapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
     super(field, allocator, callBack);
   }
 
@@ -73,14 +72,14 @@ public class MapVector extends AbstractMapVector {
   transient private MapSingleCopier ephPair2;
 
   public void copyFromSafe(int fromIndex, int thisIndex, MapVector from) {
-    if(ephPair == null || ephPair.from != from) {
+    if (ephPair == null || ephPair.from != from) {
       ephPair = (MapTransferPair) from.makeTransferPair(this);
     }
     ephPair.copyValueSafe(fromIndex, thisIndex);
   }
 
   public void copyFromSafe(int fromSubIndex, int thisIndex, RepeatedMapVector from) {
-    if(ephPair2 == null || ephPair2.from != from) {
+    if (ephPair2 == null || ephPair2.from != from) {
       ephPair2 = from.makeSingularCopier(this);
     }
     ephPair2.copySafe(fromSubIndex, thisIndex);
@@ -143,9 +142,6 @@ public class MapVector extends AbstractMapVector {
 
   @Override
   public DrillBuf[] getBuffers(boolean clear) {
-    //int expectedSize = getBufferSize();
-    //int actualSize   = super.getBufferSize();
-    //Preconditions.checkArgument(expectedSize == actualSize);
     return super.getBuffers(clear);
   }
 
@@ -294,9 +290,9 @@ public class MapVector extends AbstractMapVector {
 
   @Override
   public SerializedField getMetadata() {
-    SerializedField.Builder b = getField() //
-        .getAsBuilder() //
-        .setBufferLength(getBufferSize()) //
+    SerializedField.Builder b = getField()
+        .getAsBuilder()
+        .setBufferLength(getBufferSize())
         .setValueCount(valueCount);
 
 
@@ -311,13 +307,6 @@ public class MapVector extends AbstractMapVector {
     return mutator;
   }
 
-  @Override
-  public void exchange(ValueVector other) {
-    // Exchange is used for look-ahead writers, but writers manage
-    // map member vectors directly.
-    throw new UnsupportedOperationException("Exchange() not supported for maps");
-  }
-
   public class Accessor extends BaseValueVector.BaseAccessor {
 
     @Override
@@ -357,6 +346,14 @@ public class MapVector extends AbstractMapVector {
     return getChildByOrdinal(id);
   }
 
+  /**
+   * Set the value count for the map without setting the counts for the contained
+   * vectors. Use this only when the values of the contained vectors are set
+   * elsewhere in the code.
+   *
+   * @param valueCount number of items in the map
+   */
+
   public void setMapValueCount(int valueCount) {
     this.valueCount = valueCount;
   }
@@ -402,4 +399,13 @@ public class MapVector extends AbstractMapVector {
   public void toNullable(ValueVector nullableVector) {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public void exchange(ValueVector other) {
+    super.exchange(other);
+    MapVector otherMap = (MapVector) other;
+    int temp = otherMap.valueCount;
+    otherMap.valueCount = valueCount;
+    valueCount = temp;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
index 57f1a67..270f973 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedMapVector.java
@@ -64,9 +64,7 @@ public class RepeatedMapVector extends AbstractMapVector
   private final EmptyValuePopulator emptyPopulator;
 
   public RepeatedMapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
-    super(field, allocator, callBack);
-    this.offsets = new UInt4Vector(BaseRepeatedValueVector.OFFSETS_FIELD, allocator);
-    this.emptyPopulator = new EmptyValuePopulator(offsets);
+    this(field, new UInt4Vector(BaseRepeatedValueVector.OFFSETS_FIELD, allocator), callBack);
   }
 
   public RepeatedMapVector(MaterializedField field, UInt4Vector offsets, CallBack callBack) {
@@ -150,7 +148,7 @@ public class RepeatedMapVector extends AbstractMapVector
     }
 
     long bufferSize = offsets.getBufferSizeFor(valueCount);
-    for (final ValueVector v : (Iterable<ValueVector>) this) {
+    for (final ValueVector v : this) {
       bufferSize += v.getBufferSizeFor(valueCount);
     }
 
@@ -424,9 +422,8 @@ public class RepeatedMapVector extends AbstractMapVector
 
   @Override
   public void exchange(ValueVector other) {
-    // Exchange is used for look-ahead writers, but writers manage
-    // map member vectors directly.
-    throw new UnsupportedOperationException("Exchange() not supported for maps");
+    super.exchange(other);
+    offsets.exchange(((RepeatedMapVector) other).offsets);
   }
 
   @Override
@@ -459,13 +456,13 @@ public class RepeatedMapVector extends AbstractMapVector
     assert bufOffset == buffer.writerIndex();
   }
 
-
   @Override
   public SerializedField getMetadata() {
-    SerializedField.Builder builder = getField() //
-        .getAsBuilder() //
-        .setBufferLength(getBufferSize()) //
-        // while we don't need to actually read this on load, we need it to make sure we don't skip deserialization of this vector
+    SerializedField.Builder builder = getField()
+        .getAsBuilder()
+        .setBufferLength(getBufferSize())
+        // while we don't need to actually read this on load, we need it to
+        // make sure we don't skip deserialization of this vector
         .setValueCount(accessor.getValueCount());
     builder.addChild(offsets.getMetadata());
     for (final ValueVector child : getChildren()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedValueVector.java
index 0fba292..4bcfba6 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/RepeatedValueVector.java
@@ -35,9 +35,8 @@ public interface RepeatedValueVector extends ValueVector, ContainerVectorLike {
 
   /**
    * Returns the underlying offset vector or null if none exists.
-   *
-   * TODO(DRILL-2995): eliminate exposing low-level interfaces.
    */
+
   UInt4Vector getOffsetVector();
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/StateTool.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/StateTool.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/StateTool.java
index f5ed3a0..9a736d3 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/StateTool.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/StateTool.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,13 +22,15 @@ import java.util.Arrays;
 public class StateTool {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StateTool.class);
 
+  @SuppressWarnings("unchecked")
   public static <T extends Enum<?>> void check(T currentState, T... expectedStates) {
     for (T s : expectedStates) {
       if (s == currentState) {
         return;
       }
     }
-    throw new IllegalArgumentException(String.format("Expected to be in one of these states %s but was actuall in state %s", Arrays.toString(expectedStates), currentState));
+    throw new IllegalArgumentException(
+        String.format("Expected to be in one of these states %s but was actually in state %s",
+            Arrays.toString(expectedStates), currentState));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java
index 10ac551..28e90b9 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/PromotableWriter.java
@@ -120,6 +120,7 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
     }
   }
 
+  @Override
   protected FieldWriter getWriter(MinorType type) {
     if (state == State.UNION) {
       return writer;
@@ -144,6 +145,7 @@ public class PromotableWriter extends AbstractPromotableFieldWriter {
     return writer.isEmptyMap();
   }
 
+  @Override
   protected FieldWriter getWriter() {
     return getWriter(type);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/logical/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
index 7dfe4a2..22f2b09 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpressionBase.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,19 +17,12 @@
  */
 package org.apache.drill.common.expression;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
 
 @JsonPropertyOrder({ "type" })
 public abstract class LogicalExpressionBase implements LogicalExpression {
@@ -62,11 +55,13 @@ public abstract class LogicalExpressionBase implements LogicalExpression {
     return this.getClass().getSimpleName();
   }
 
+  @Override
   @JsonIgnore
   public int getSelfCost() {
     return 0;
   }
 
+  @Override
   @JsonIgnore
   public int getCumulativeCost()  {
     int cost = this.getSelfCost();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
index 16bb255..f198620 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
@@ -28,6 +28,7 @@ public abstract class PathSegment {
   }
 
   public abstract PathSegment cloneWithNewChild(PathSegment segment);
+
   @Override
   public abstract PathSegment clone();
 
@@ -123,7 +124,6 @@ public abstract class PathSegment {
     }
   }
 
-
   public static final class NameSegment extends PathSegment {
     private final String path;
 
@@ -137,24 +137,16 @@ public abstract class PathSegment {
       this.path = n.toString();
     }
 
-    public String getPath() {
-      return path;
-    }
+    public String getPath() { return path; }
 
     @Override
-    public boolean isArray() {
-      return false;
-    }
+    public boolean isArray() { return false; }
 
     @Override
-    public boolean isNamed() {
-      return true;
-    }
+    public boolean isNamed() { return true; }
 
     @Override
-    public NameSegment getNameSegment() {
-      return this;
-    }
+    public NameSegment getNameSegment() { return this; }
 
     @Override
     public String toString() {
@@ -183,6 +175,11 @@ public abstract class PathSegment {
       return path.equalsIgnoreCase(other.path);
     }
 
+    public boolean nameEquals(String name) {
+      return path == null && name == null ||
+             path != null && path.equalsIgnoreCase(name);
+    }
+
     @Override
     public NameSegment clone() {
       NameSegment s = new NameSegment(this.path);
@@ -202,7 +199,6 @@ public abstract class PathSegment {
       }
       return s;
     }
-
   }
 
   public NameSegment getNameSegment() {
@@ -284,6 +280,7 @@ public abstract class PathSegment {
    * @param otherSeg - path segment to check if it is contained below this one.
    * @return - is this a match
    */
+
   public boolean contains(PathSegment otherSeg) {
     if (this == otherSeg) {
       return true;
@@ -309,7 +306,5 @@ public abstract class PathSegment {
     } else {
       return child.contains(otherSeg.child);
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index 8854e15..95f3dbb 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 
 import org.antlr.runtime.ANTLRStringStream;
@@ -38,12 +39,29 @@ import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
 
 public class SchemaPath extends LogicalExpressionBase {
 
+  public static final String WILDCARD = "*";
+  public static final SchemaPath STAR_COLUMN = getSimplePath(WILDCARD);
+
   private final NameSegment rootSegment;
 
+  public SchemaPath(SchemaPath path) {
+    super(path.getPosition());
+    this.rootSegment = path.rootSegment;
+  }
+
+  public SchemaPath(NameSegment rootSegment) {
+    super(ExpressionPosition.UNKNOWN);
+    this.rootSegment = rootSegment;
+  }
+
+  public SchemaPath(NameSegment rootSegment, ExpressionPosition pos) {
+    super(pos);
+    this.rootSegment = rootSegment;
+  }
+
   public static SchemaPath getSimplePath(String name) {
     return getCompoundPath(name);
   }
@@ -58,7 +76,7 @@ public class SchemaPath extends LogicalExpressionBase {
   }
 
   public PathSegment getLastSegment() {
-    PathSegment s= rootSegment;
+    PathSegment s = rootSegment;
     while (s.getChild() != null) {
       s = s.getChild();
     }
@@ -71,7 +89,6 @@ public class SchemaPath extends LogicalExpressionBase {
     this.rootSegment = new NameSegment(simpleName);
   }
 
-
   public NamePart getAsNamePart() {
     return getNamePart(rootSegment);
   }
@@ -157,20 +174,75 @@ public class SchemaPath extends LogicalExpressionBase {
     return true;
   }
 
+  /**
+   * Return whether this name refers to an array. The path must be an array if it
+   * ends with an array index; else it may or may not be an entire array.
+   *
+   * @return true if the path ends with an array index, false otherwise
+   */
 
-  public SchemaPath(SchemaPath path) {
-    super(path.getPosition());
-    this.rootSegment = path.rootSegment;
+  public boolean isArray() {
+    PathSegment seg = rootSegment;
+    while (seg != null) {
+      if (seg.isArray()) {
+        return true;
+      }
+      seg = seg.getChild();
+    }
+    return false;
   }
 
-  public SchemaPath(NameSegment rootSegment) {
-    super(ExpressionPosition.UNKNOWN);
-    this.rootSegment = rootSegment;
+  /**
+   * Determine if this is a one-part name. In general, special columns work only
+   * if they are single-part names.
+   *
+   * @return true if this is a one-part name, false if this is a multi-part
+   * name (with either map member or array index parts.)
+   */
+
+  public boolean isLeaf() {
+    return rootSegment.isLastPath();
   }
 
-  public SchemaPath(NameSegment rootSegment, ExpressionPosition pos) {
-    super(pos);
-    this.rootSegment = rootSegment;
+  /**
+   * Return if this column is the special wildcard ("*") column which means to
+   * project all table columns.
+   *
+   * @return true if the column is "*"
+   */
+
+  public boolean isWildcard() {
+    return isLeaf() && nameEquals(WILDCARD);
+  }
+
+  /**
+   * Returns if this is a simple column and the name matches the given
+   * name (ignoring case.) This does not check if the name is an entire
+   * match, only the the first (or only) part of the name matches.
+   * Also check {@link #isLeaf()} to check for a single-part name.
+   *
+   * @param name name to match
+   * @return true if this is a single-part column with that name.
+   */
+
+  public boolean nameEquals(String name) {
+    return rootSegment.nameEquals(name);
+  }
+
+  /**
+   * Return the root name: either the entire name (if one part) or
+   * the first part (if multi-part.)
+   * <ul>
+   * <li>a: returns a</li>
+   * <li>a.b: returns a</li>
+   * <li>a[10]: returns a</li>
+   * </ul>
+   *
+   * @return the root (or only) name
+   */
+
+  public String rootName() {
+    return rootSegment.getPath();
   }
 
   @Override
@@ -243,7 +315,7 @@ public class SchemaPath extends LogicalExpressionBase {
 
   @Override
   public Iterator<LogicalExpression> iterator() {
-    return Iterators.emptyIterator();
+    return Collections.emptyIterator();
   }
 
   @Override
@@ -264,6 +336,7 @@ public class SchemaPath extends LogicalExpressionBase {
     return rootSegment.getPath();
   }
 
+  @SuppressWarnings("serial")
   public static class De extends StdDeserializer<SchemaPath> {
 
     public De() {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/logical/src/main/java/org/apache/drill/common/logical/FormatPluginConfigBase.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/logical/FormatPluginConfigBase.java b/logical/src/main/java/org/apache/drill/common/logical/FormatPluginConfigBase.java
index 6b9dfec..5bdb69f 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/FormatPluginConfigBase.java
+++ b/logical/src/main/java/org/apache/drill/common/logical/FormatPluginConfigBase.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,7 +25,6 @@ import org.apache.drill.common.scanner.persistence.ScanResult;
 public abstract class FormatPluginConfigBase implements FormatPluginConfig{
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FormatPluginConfigBase.class);
 
-
   /**
    * scan for implementations of {@see FormatPlugin}.
    *
@@ -38,7 +37,7 @@ public abstract class FormatPluginConfigBase implements FormatPluginConfig{
       StringBuilder sb = new StringBuilder();
       sb.append("Found ");
       sb.append(pluginClasses.size());
-      sb.append("format plugin configuration classes:\n");
+      sb.append(" format plugin configuration classes:\n");
       for (Class<?> c : pluginClasses) {
         sb.append('\t');
         sb.append(c.getName());
@@ -54,5 +53,4 @@ public abstract class FormatPluginConfigBase implements FormatPluginConfig{
 
   @Override
   public abstract int hashCode();
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c64788c..9b2a368 100644
--- a/pom.xml
+++ b/pom.xml
@@ -31,7 +31,7 @@
   <properties>
     <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
     <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
-    <dep.junit.version>4.11</dep.junit.version>
+    <dep.junit.version>4.12</dep.junit.version>
     <dep.slf4j.version>1.7.6</dep.slf4j.version>
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index edc401c..9ef1f8d 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -518,8 +518,8 @@ public final class UserBitShared {
      */
     PCAP_SUB_SCAN(37, 37),
     /**
-    * <code>KAFKA_SUB_SCAN = 38;</code>
-    */
+     * <code>KAFKA_SUB_SCAN = 38;</code>
+     */
     KAFKA_SUB_SCAN(38, 38),
     ;
 
@@ -2223,6 +2223,36 @@ public final class UserBitShared {
        * </pre>
        */
       VALIDATION(10, 10),
+      /**
+       * <code>EXECUTION_ERROR = 11;</code>
+       *
+       * <pre>
+       * Execution exception
+       *  - Internal errors not related to bad code
+       * </pre>
+       */
+      EXECUTION_ERROR(11, 11),
+      /**
+       * <code>INTERNAL_ERROR = 12;</code>
+       *
+       * <pre>
+       * Internal exception
+       *  - Failed assertions
+       *  - Other "this should not happen" cases
+       * </pre>
+       */
+      INTERNAL_ERROR(12, 12),
+      /**
+       * <code>UNSPECIFIED_ERROR = 13;</code>
+       *
+       * <pre>
+       * Unspecified exception
+       *  - Exception caught but cause is unknown
+       * Indicates code that needs revisiting to move error reporting
+       * closer to the cause.
+       * </pre>
+       */
+      UNSPECIFIED_ERROR(13, 13),
       ;
 
       /**
@@ -2335,6 +2365,36 @@ public final class UserBitShared {
        * </pre>
        */
       public static final int VALIDATION_VALUE = 10;
+      /**
+       * <code>EXECUTION_ERROR = 11;</code>
+       *
+       * <pre>
+       * Execution exception
+       *  - Internal errors not related to bad code
+       * </pre>
+       */
+      public static final int EXECUTION_ERROR_VALUE = 11;
+      /**
+       * <code>INTERNAL_ERROR = 12;</code>
+       *
+       * <pre>
+       * Internal exception
+       *  - Failed assertions
+       *  - Other "this should not happen" cases
+       * </pre>
+       */
+      public static final int INTERNAL_ERROR_VALUE = 12;
+      /**
+       * <code>UNSPECIFIED_ERROR = 13;</code>
+       *
+       * <pre>
+       * Unspecified exception
+       *  - Exception caught but cause is unknown
+       * Indicates code that needs revisiting to move error reporting
+       * closer to the cause.
+       * </pre>
+       */
+      public static final int UNSPECIFIED_ERROR_VALUE = 13;
 
 
       public final int getNumber() { return value; }
@@ -2352,6 +2412,9 @@ public final class UserBitShared {
           case 8: return SYSTEM;
           case 9: return UNSUPPORTED_OPERATION;
           case 10: return VALIDATION;
+          case 11: return EXECUTION_ERROR;
+          case 12: return INTERNAL_ERROR;
+          case 13: return UNSPECIFIED_ERROR;
           default: return null;
         }
       }
@@ -23942,127 +24005,129 @@ public final class UserBitShared {
       "s.proto\032\022Coordination.proto\032\017SchemaDef.p" +
       "roto\"$\n\017UserCredentials\022\021\n\tuser_name\030\001 \001" +
       "(\t\"\'\n\007QueryId\022\r\n\005part1\030\001 \001(\020\022\r\n\005part2\030\002 " +
-      "\001(\020\"\255\003\n\014DrillPBError\022\020\n\010error_id\030\001 \001(\t\022(" +
+      "\001(\020\"\355\003\n\014DrillPBError\022\020\n\010error_id\030\001 \001(\t\022(" +
       "\n\010endpoint\030\002 \001(\0132\026.exec.DrillbitEndpoint" +
       "\0227\n\nerror_type\030\003 \001(\0162#.exec.shared.Drill" +
       "PBError.ErrorType\022\017\n\007message\030\004 \001(\t\0220\n\tex" +
       "ception\030\005 \001(\0132\035.exec.shared.ExceptionWra" +
       "pper\0220\n\rparsing_error\030\006 \003(\0132\031.exec.share",
-      "d.ParsingError\"\262\001\n\tErrorType\022\016\n\nCONNECTI" +
+      "d.ParsingError\"\362\001\n\tErrorType\022\016\n\nCONNECTI" +
       "ON\020\000\022\r\n\tDATA_READ\020\001\022\016\n\nDATA_WRITE\020\002\022\014\n\010F" +
       "UNCTION\020\003\022\t\n\005PARSE\020\004\022\016\n\nPERMISSION\020\005\022\010\n\004" +
       "PLAN\020\006\022\014\n\010RESOURCE\020\007\022\n\n\006SYSTEM\020\010\022\031\n\025UNSU" +
-      "PPORTED_OPERATION\020\t\022\016\n\nVALIDATION\020\n\"\246\001\n\020" +
-      "ExceptionWrapper\022\027\n\017exception_class\030\001 \001(" +
-      "\t\022\017\n\007message\030\002 \001(\t\022:\n\013stack_trace\030\003 \003(\0132" +
-      "%.exec.shared.StackTraceElementWrapper\022," +
-      "\n\005cause\030\004 \001(\0132\035.exec.shared.ExceptionWra" +
-      "pper\"\205\001\n\030StackTraceElementWrapper\022\022\n\ncla",
-      "ss_name\030\001 \001(\t\022\021\n\tfile_name\030\002 \001(\t\022\023\n\013line" +
-      "_number\030\003 \001(\005\022\023\n\013method_name\030\004 \001(\t\022\030\n\020is" +
-      "_native_method\030\005 \001(\010\"\\\n\014ParsingError\022\024\n\014" +
-      "start_column\030\002 \001(\005\022\021\n\tstart_row\030\003 \001(\005\022\022\n" +
-      "\nend_column\030\004 \001(\005\022\017\n\007end_row\030\005 \001(\005\"~\n\016Re" +
-      "cordBatchDef\022\024\n\014record_count\030\001 \001(\005\022+\n\005fi" +
-      "eld\030\002 \003(\0132\034.exec.shared.SerializedField\022" +
-      ")\n!carries_two_byte_selection_vector\030\003 \001" +
-      "(\010\"\205\001\n\010NamePart\022(\n\004type\030\001 \001(\0162\032.exec.sha" +
-      "red.NamePart.Type\022\014\n\004name\030\002 \001(\t\022$\n\005child",
-      "\030\003 \001(\0132\025.exec.shared.NamePart\"\033\n\004Type\022\010\n" +
-      "\004NAME\020\000\022\t\n\005ARRAY\020\001\"\324\001\n\017SerializedField\022%" +
-      "\n\nmajor_type\030\001 \001(\0132\021.common.MajorType\022(\n" +
-      "\tname_part\030\002 \001(\0132\025.exec.shared.NamePart\022" +
-      "+\n\005child\030\003 \003(\0132\034.exec.shared.SerializedF" +
-      "ield\022\023\n\013value_count\030\004 \001(\005\022\027\n\017var_byte_le" +
-      "ngth\030\005 \001(\005\022\025\n\rbuffer_length\030\007 \001(\005\"7\n\nNod" +
-      "eStatus\022\017\n\007node_id\030\001 \001(\005\022\030\n\020memory_footp" +
-      "rint\030\002 \001(\003\"\263\002\n\013QueryResult\0228\n\013query_stat" +
-      "e\030\001 \001(\0162#.exec.shared.QueryResult.QueryS",
-      "tate\022&\n\010query_id\030\002 \001(\0132\024.exec.shared.Que" +
-      "ryId\022(\n\005error\030\003 \003(\0132\031.exec.shared.DrillP" +
-      "BError\"\227\001\n\nQueryState\022\014\n\010STARTING\020\000\022\013\n\007R" +
-      "UNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED\020\003\022\n\n" +
-      "\006FAILED\020\004\022\032\n\026CANCELLATION_REQUESTED\020\005\022\014\n" +
-      "\010ENQUEUED\020\006\022\r\n\tPREPARING\020\007\022\014\n\010PLANNING\020\010" +
-      "\"p\n\tQueryData\022&\n\010query_id\030\001 \001(\0132\024.exec.s" +
-      "hared.QueryId\022\021\n\trow_count\030\002 \001(\005\022(\n\003def\030" +
-      "\003 \001(\0132\033.exec.shared.RecordBatchDef\"\330\001\n\tQ" +
-      "ueryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222",
-      "\n\005state\030\003 \001(\0162#.exec.shared.QueryResult." +
-      "QueryState\022\017\n\004user\030\004 \001(\t:\001-\022\'\n\007foreman\030\005" +
-      " \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014options_" +
-      "json\030\006 \001(\t\022\022\n\ntotal_cost\030\007 \001(\001\022\025\n\nqueue_" +
-      "name\030\010 \001(\t:\001-\"\242\004\n\014QueryProfile\022 \n\002id\030\001 \001" +
-      "(\0132\024.exec.shared.QueryId\022$\n\004type\030\002 \001(\0162\026" +
-      ".exec.shared.QueryType\022\r\n\005start\030\003 \001(\003\022\013\n" +
-      "\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n\004plan\030\006 \001(\t\022" +
-      "\'\n\007foreman\030\007 \001(\0132\026.exec.DrillbitEndpoint" +
-      "\0222\n\005state\030\010 \001(\0162#.exec.shared.QueryResul",
-      "t.QueryState\022\027\n\017total_fragments\030\t \001(\005\022\032\n" +
-      "\022finished_fragments\030\n \001(\005\022;\n\020fragment_pr" +
-      "ofile\030\013 \003(\0132!.exec.shared.MajorFragmentP" +
-      "rofile\022\017\n\004user\030\014 \001(\t:\001-\022\r\n\005error\030\r \001(\t\022\024" +
-      "\n\014verboseError\030\016 \001(\t\022\020\n\010error_id\030\017 \001(\t\022\022" +
-      "\n\nerror_node\030\020 \001(\t\022\024\n\014options_json\030\021 \001(\t" +
-      "\022\017\n\007planEnd\030\022 \001(\003\022\024\n\014queueWaitEnd\030\023 \001(\003\022" +
-      "\022\n\ntotal_cost\030\024 \001(\001\022\025\n\nqueue_name\030\025 \001(\t:" +
-      "\001-\"t\n\024MajorFragmentProfile\022\031\n\021major_frag" +
-      "ment_id\030\001 \001(\005\022A\n\026minor_fragment_profile\030",
-      "\002 \003(\0132!.exec.shared.MinorFragmentProfile" +
-      "\"\350\002\n\024MinorFragmentProfile\022)\n\005state\030\001 \001(\016" +
-      "2\032.exec.shared.FragmentState\022(\n\005error\030\002 " +
-      "\001(\0132\031.exec.shared.DrillPBError\022\031\n\021minor_" +
-      "fragment_id\030\003 \001(\005\0226\n\020operator_profile\030\004 " +
-      "\003(\0132\034.exec.shared.OperatorProfile\022\022\n\nsta" +
-      "rt_time\030\005 \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memor" +
-      "y_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n" +
-      "\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint\022" +
-      "\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013 ",
-      "\001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profile" +
-      "\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013op" +
-      "erator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023" +
-      "\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001" +
-      "(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003\022" +
-      "(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricValu" +
-      "e\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022\017" +
-      "\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sche" +
-      "mas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 " +
-      "\001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value\030",
-      "\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.sh" +
-      "ared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functio" +
-      "n_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmech" +
-      "anism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001(" +
-      "\0162\027.exec.shared.SaslStatus*5\n\nRpcChannel" +
-      "\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020" +
-      "\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010" +
-      "PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_ST" +
-      "ATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000" +
-      "\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014",
-      "\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022" +
-      "\032\n\026CANCELLATION_REQUESTED\020\006*\360\005\n\020CoreOper" +
-      "atorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST" +
-      "_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020" +
-      "\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH" +
-      "_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGIN" +
-      "G_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDER" +
-      "\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022" +
-      "\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTI" +
-      "ON_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREGA",
-      "TE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022" +
-      "\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026P" +
-      "ARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN" +
-      "\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SC" +
-      "AN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_S" +
-      "CAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020" +
-      "\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_" +
-      "SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_" +
-      "CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW" +
-      "\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCA",
-      "N\020$\022\021\n\rPCAP_SUB_SCAN\020%*g\n\nSaslStatus\022\020\n\014" +
-      "SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_I" +
-      "N_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_F" +
-      "AILED\020\004B.\n\033org.apache.drill.exec.protoB\r" +
-      "UserBitSharedH\001"
+      "PPORTED_OPERATION\020\t\022\016\n\nVALIDATION\020\n\022\023\n\017E" +
+      "XECUTION_ERROR\020\013\022\022\n\016INTERNAL_ERROR\020\014\022\025\n\021" +
+      "UNSPECIFIED_ERROR\020\r\"\246\001\n\020ExceptionWrapper" +
+      "\022\027\n\017exception_class\030\001 \001(\t\022\017\n\007message\030\002 \001" +
+      "(\t\022:\n\013stack_trace\030\003 \003(\0132%.exec.shared.St" +
+      "ackTraceElementWrapper\022,\n\005cause\030\004 \001(\0132\035.",
+      "exec.shared.ExceptionWrapper\"\205\001\n\030StackTr" +
+      "aceElementWrapper\022\022\n\nclass_name\030\001 \001(\t\022\021\n" +
+      "\tfile_name\030\002 \001(\t\022\023\n\013line_number\030\003 \001(\005\022\023\n" +
+      "\013method_name\030\004 \001(\t\022\030\n\020is_native_method\030\005" +
+      " \001(\010\"\\\n\014ParsingError\022\024\n\014start_column\030\002 \001" +
+      "(\005\022\021\n\tstart_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001(" +
+      "\005\022\017\n\007end_row\030\005 \001(\005\"~\n\016RecordBatchDef\022\024\n\014" +
+      "record_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exec" +
+      ".shared.SerializedField\022)\n!carries_two_b" +
+      "yte_selection_vector\030\003 \001(\010\"\205\001\n\010NamePart\022",
+      "(\n\004type\030\001 \001(\0162\032.exec.shared.NamePart.Typ" +
+      "e\022\014\n\004name\030\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.sh" +
+      "ared.NamePart\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY" +
+      "\020\001\"\324\001\n\017SerializedField\022%\n\nmajor_type\030\001 \001" +
+      "(\0132\021.common.MajorType\022(\n\tname_part\030\002 \001(\013" +
+      "2\025.exec.shared.NamePart\022+\n\005child\030\003 \003(\0132\034" +
+      ".exec.shared.SerializedField\022\023\n\013value_co" +
+      "unt\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\025\n\rbu" +
+      "ffer_length\030\007 \001(\005\"7\n\nNodeStatus\022\017\n\007node_" +
+      "id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\263\002\n\013Q",
+      "ueryResult\0228\n\013query_state\030\001 \001(\0162#.exec.s" +
+      "hared.QueryResult.QueryState\022&\n\010query_id" +
+      "\030\002 \001(\0132\024.exec.shared.QueryId\022(\n\005error\030\003 " +
+      "\003(\0132\031.exec.shared.DrillPBError\"\227\001\n\nQuery" +
+      "State\022\014\n\010STARTING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMP" +
+      "LETED\020\002\022\014\n\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\032\n\026CAN" +
+      "CELLATION_REQUESTED\020\005\022\014\n\010ENQUEUED\020\006\022\r\n\tP" +
+      "REPARING\020\007\022\014\n\010PLANNING\020\010\"p\n\tQueryData\022&\n" +
+      "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\021\n" +
+      "\trow_count\030\002 \001(\005\022(\n\003def\030\003 \001(\0132\033.exec.sha",
+      "red.RecordBatchDef\"\330\001\n\tQueryInfo\022\r\n\005quer" +
+      "y\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222\n\005state\030\003 \001(\0162#." +
+      "exec.shared.QueryResult.QueryState\022\017\n\004us" +
+      "er\030\004 \001(\t:\001-\022\'\n\007foreman\030\005 \001(\0132\026.exec.Dril" +
+      "lbitEndpoint\022\024\n\014options_json\030\006 \001(\t\022\022\n\nto" +
+      "tal_cost\030\007 \001(\001\022\025\n\nqueue_name\030\010 \001(\t:\001-\"\242\004" +
+      "\n\014QueryProfile\022 \n\002id\030\001 \001(\0132\024.exec.shared" +
+      ".QueryId\022$\n\004type\030\002 \001(\0162\026.exec.shared.Que" +
+      "ryType\022\r\n\005start\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005qu" +
+      "ery\030\005 \001(\t\022\014\n\004plan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\013",
+      "2\026.exec.DrillbitEndpoint\0222\n\005state\030\010 \001(\0162" +
+      "#.exec.shared.QueryResult.QueryState\022\027\n\017" +
+      "total_fragments\030\t \001(\005\022\032\n\022finished_fragme" +
+      "nts\030\n \001(\005\022;\n\020fragment_profile\030\013 \003(\0132!.ex" +
+      "ec.shared.MajorFragmentProfile\022\017\n\004user\030\014" +
+      " \001(\t:\001-\022\r\n\005error\030\r \001(\t\022\024\n\014verboseError\030\016" +
+      " \001(\t\022\020\n\010error_id\030\017 \001(\t\022\022\n\nerror_node\030\020 \001" +
+      "(\t\022\024\n\014options_json\030\021 \001(\t\022\017\n\007planEnd\030\022 \001(" +
+      "\003\022\024\n\014queueWaitEnd\030\023 \001(\003\022\022\n\ntotal_cost\030\024 " +
+      "\001(\001\022\025\n\nqueue_name\030\025 \001(\t:\001-\"t\n\024MajorFragm",
+      "entProfile\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n" +
+      "\026minor_fragment_profile\030\002 \003(\0132!.exec.sha" +
+      "red.MinorFragmentProfile\"\350\002\n\024MinorFragme" +
+      "ntProfile\022)\n\005state\030\001 \001(\0162\032.exec.shared.F" +
+      "ragmentState\022(\n\005error\030\002 \001(\0132\031.exec.share" +
+      "d.DrillPBError\022\031\n\021minor_fragment_id\030\003 \001(" +
+      "\005\0226\n\020operator_profile\030\004 \003(\0132\034.exec.share" +
+      "d.OperatorProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n" +
+      "\010end_time\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017" +
+      "max_memory_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132",
+      "\026.exec.DrillbitEndpoint\022\023\n\013last_update\030\n" +
+      " \001(\003\022\025\n\rlast_progress\030\013 \001(\003\"\377\001\n\017Operator" +
+      "Profile\0221\n\rinput_profile\030\001 \003(\0132\032.exec.sh" +
+      "ared.StreamProfile\022\023\n\013operator_id\030\003 \001(\005\022" +
+      "\025\n\roperator_type\030\004 \001(\005\022\023\n\013setup_nanos\030\005 " +
+      "\001(\003\022\025\n\rprocess_nanos\030\006 \001(\003\022#\n\033peak_local" +
+      "_memory_allocated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132" +
+      "\030.exec.shared.MetricValue\022\022\n\nwait_nanos\030" +
+      "\t \001(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022" +
+      "\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Met",
+      "ricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_valu" +
+      "e\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registr" +
+      "y\022\035\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022" +
+      "\014\n\004name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003(" +
+      "\t\"W\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004d" +
+      "ata\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared." +
+      "SaslStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020" +
+      "\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007" +
+      "\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEX" +
+      "ECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFr",
+      "agmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALL" +
+      "OCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\t" +
+      "CANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_" +
+      "REQUESTED\020\006*\204\006\n\020CoreOperatorType\022\021\n\rSING" +
+      "LE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FIL" +
+      "TER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004" +
+      "\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDE" +
+      "R\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030" +
+      "ORDERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022" +
+      "\026\n\022UNORDERED_RECEIVER\020\013\022\020\n\014RANGE_SENDER\020",
+      "\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTOR_REMOVE" +
+      "R\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\nTOP_N_SO" +
+      "RT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UN" +
+      "ION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_ROW_GROUP" +
+      "_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TAB" +
+      "LE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_" +
+      "WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WR" +
+      "ITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SC" +
+      "AN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLE" +
+      "X_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUMER\020 \022\022\n\016HB",
+      "ASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOO" +
+      "P_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_" +
+      "SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&*g\n\nSaslStatus" +
+      "\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SA" +
+      "SL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SA" +
+      "SL_FAILED\020\004B.\n\033org.apache.drill.exec.pro" +
+      "toB\rUserBitSharedH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/protocol/src/main/java/org/apache/drill/exec/proto/beans/DrillPBError.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/DrillPBError.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/DrillPBError.java
index ee237d9..1a105f2 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/DrillPBError.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/DrillPBError.java
@@ -47,7 +47,10 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
         RESOURCE(7),
         SYSTEM(8),
         UNSUPPORTED_OPERATION(9),
-        VALIDATION(10);
+        VALIDATION(10),
+        EXECUTION_ERROR(11),
+        INTERNAL_ERROR(12),
+        UNSPECIFIED_ERROR(13);
         
         public final int number;
         
@@ -76,6 +79,9 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
                 case 8: return SYSTEM;
                 case 9: return UNSUPPORTED_OPERATION;
                 case 10: return VALIDATION;
+                case 11: return EXECUTION_ERROR;
+                case 12: return INTERNAL_ERROR;
+                case 13: return UNSPECIFIED_ERROR;
                 default: return null;
             }
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/protocol/src/main/protobuf/GeneralRPC.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/GeneralRPC.proto b/protocol/src/main/protobuf/GeneralRPC.proto
index 26ab821..bed2ad3 100644
--- a/protocol/src/main/protobuf/GeneralRPC.proto
+++ b/protocol/src/main/protobuf/GeneralRPC.proto
@@ -19,7 +19,7 @@ enum RpcMode {
 }
 
 message RpcHeader{
-	optional RpcMode mode = 1; 
+	optional RpcMode mode = 1;
 	optional int32 coordination_id = 2; // reusable coordination identifier.  Sender defines.  Server returns on return.  Irrelevant for purely single direction rpc.
 	optional int32 rpc_type = 3; // a rpc mode specific rpc type.
 }
@@ -29,7 +29,3 @@ message CompleteRpcMessage {
     optional bytes protobuf_body = 2; // required
     optional bytes raw_body = 3; // optional
 }
-
-
-
-

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 205611b..dc8bdb6 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -90,6 +90,21 @@ message DrillPBError{
      * - invalid entries in SQL tree
      */
     VALIDATION = 10;
+    /* Execution exception
+     *  - Internal errors not related to bad code
+     */
+    EXECUTION_ERROR = 11;
+    /* Internal exception
+     *  - Failed assertions
+     *  - Other "this should not happen" cases
+     */
+    INTERNAL_ERROR = 12;
+    /* Unspecified exception
+     *  - Exception caught but cause is unknown
+     * Indicates code that needs revisiting to move error reporting
+     * closer to the cause.
+     */
+    UNSPECIFIED_ERROR = 13;
   }
   optional string error_id = 1; // for debug tracing purposes
   optional DrillbitEndpoint endpoint = 2;
@@ -114,7 +129,6 @@ message StackTraceElementWrapper {
     optional bool is_native_method = 5;
 }
 
-
 message ParsingError{
   optional int32 start_column = 2;
   optional int32 start_row = 3;


[08/11] drill git commit: DRILL-6049: Misc. hygiene and code cleanup changes

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/FixedValueVectors.java b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
index 1f6a008..79beb52 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -106,7 +106,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   @Override
-  public int getValueCapacity(){
+  public int getValueCapacity() {
     return data.capacity() / VALUE_WIDTH;
   }
 
@@ -129,7 +129,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
   @Override
   public void allocateNew() {
-    if (!allocateNewSafe()){
+    if (!allocateNewSafe()) {
       throw new OutOfMemoryException("Failure while allocating buffer.");
     }
   }
@@ -264,12 +264,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
   }
 
   @Override
-  public TransferPair getTransferPair(BufferAllocator allocator){
+  public TransferPair getTransferPair(BufferAllocator allocator) {
     return new TransferImpl(getField(), allocator);
   }
 
   @Override
-  public TransferPair getTransferPair(String ref, BufferAllocator allocator){
+  public TransferPair getTransferPair(String ref, BufferAllocator allocator) {
     return new TransferImpl(getField().withPath(ref), allocator);
   }
 
@@ -278,7 +278,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     return new TransferImpl((${minor.class}Vector) to);
   }
 
-  public void transferTo(${minor.class}Vector target){
+  public void transferTo(${minor.class}Vector target) {
     target.clear();
     target.data = data.transferOwnership(target.allocator).buffer;
     target.data.writerIndex(data.writerIndex());
@@ -298,10 +298,10 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     return valueCount * ${type.width};
   }
 
-  private class TransferImpl implements TransferPair{
+  private class TransferImpl implements TransferPair {
     private ${minor.class}Vector to;
 
-    public TransferImpl(MaterializedField field, BufferAllocator allocator){
+    public TransferImpl(MaterializedField field, BufferAllocator allocator) {
       to = new ${minor.class}Vector(field, allocator);
     }
 
@@ -310,12 +310,12 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
     @Override
-    public ${minor.class}Vector getTo(){
+    public ${minor.class}Vector getTo() {
       return to;
     }
 
     @Override
-    public void transfer(){
+    public void transfer() {
       transferTo(to);
     }
 
@@ -330,7 +330,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
   }
 
-  public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from){
+  public void copyFrom(int fromIndex, int thisIndex, ${minor.class}Vector from) {
     <#if (type.width > 8)>
     from.data.getBytes(fromIndex * VALUE_WIDTH, data, thisIndex * VALUE_WIDTH, VALUE_WIDTH);
     <#else> <#-- type.width <= 8 -->
@@ -340,7 +340,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     </#if> <#-- type.width -->
   }
 
-  public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from){
+  public void copyFromSafe(int fromIndex, int thisIndex, ${minor.class}Vector from) {
     while(thisIndex >= getValueCapacity()) {
         reAlloc();
     }
@@ -376,24 +376,24 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     }
 
     @Override
-    public boolean isNull(int index){
+    public boolean isNull(int index) {
       return false;
     }
-
     <#if (type.width > 8)>
+    
     public ${minor.javaType!type.javaType} get(int index) {
       return data.slice(index * VALUE_WIDTH, VALUE_WIDTH);
     }
-
     <#if (minor.class == "Interval")>
-    public void get(int index, ${minor.class}Holder holder){
+    
+    public void get(int index, ${minor.class}Holder holder) {
       final int offsetIndex = index * VALUE_WIDTH;
       holder.months = data.getInt(offsetIndex);
       holder.days = data.getInt(offsetIndex + ${minor.daysOffset});
       holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
     }
 
-    public void get(int index, Nullable${minor.class}Holder holder){
+    public void get(int index, Nullable${minor.class}Holder holder) {
       final int offsetIndex = index * VALUE_WIDTH;
       holder.isSet = 1;
       holder.months = data.getInt(offsetIndex);
@@ -407,52 +407,25 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       final int months  = data.getInt(offsetIndex);
       final int days    = data.getInt(offsetIndex + ${minor.daysOffset});
       final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-      final Period p = new Period();
-      return p.plusMonths(months).plusDays(days).plusMillis(millis);
+      return DateUtilities.fromInterval(months, days, millis);
     }
 
     public StringBuilder getAsStringBuilder(int index) {
-
       final int offsetIndex = index * VALUE_WIDTH;
-
-      int months  = data.getInt(offsetIndex);
-      final int days    = data.getInt(offsetIndex + ${minor.daysOffset});
-      int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-
-      final int years  = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      months = (months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-
-      final int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-
-      final int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-
-      final long seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-
-      final String yearString = (Math.abs(years) == 1) ? " year " : " years ";
-      final String monthString = (Math.abs(months) == 1) ? " month " : " months ";
-      final String dayString = (Math.abs(days) == 1) ? " day " : " days ";
-
-      return(new StringBuilder().
-             append(years).append(yearString).
-             append(months).append(monthString).
-             append(days).append(dayString).
-             append(hours).append(":").
-             append(minutes).append(":").
-             append(seconds).append(".").
-             append(millis));
+      final int months = data.getInt(offsetIndex);
+      final int days   = data.getInt(offsetIndex + ${minor.daysOffset});
+      final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});      
+      return DateUtilities.intervalStringBuilder(months, days, millis);
     }
-
     <#elseif (minor.class == "IntervalDay")>
-    public void get(int index, ${minor.class}Holder holder){
+    
+    public void get(int index, ${minor.class}Holder holder) {
       final int offsetIndex = index * VALUE_WIDTH;
       holder.days = data.getInt(offsetIndex);
       holder.milliseconds = data.getInt(offsetIndex + ${minor.millisecondsOffset});
     }
 
-    public void get(int index, Nullable${minor.class}Holder holder){
+    public void get(int index, Nullable${minor.class}Holder holder) {
       final int offsetIndex = index * VALUE_WIDTH;
       holder.isSet = 1;
       holder.days = data.getInt(offsetIndex);
@@ -462,38 +435,19 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     @Override
     public ${friendlyType} getObject(int index) {
       final int offsetIndex = index * VALUE_WIDTH;
+      final int days   = data.getInt(offsetIndex);
       final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-      final int  days   = data.getInt(offsetIndex);
-      final Period p = new Period();
-      return p.plusDays(days).plusMillis(millis);
+      return DateUtilities.fromIntervalDay(days, millis);
     }
 
     public StringBuilder getAsStringBuilder(int index) {
       final int offsetIndex = index * VALUE_WIDTH;
-
-      int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});
-      final int  days   = data.getInt(offsetIndex);
-
-      final int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-
-      final int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-
-      final int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-
-      final String dayString = (Math.abs(days) == 1) ? " day " : " days ";
-
-      return(new StringBuilder().
-              append(days).append(dayString).
-              append(hours).append(":").
-              append(minutes).append(":").
-              append(seconds).append(".").
-              append(millis));
+      final int days   = data.getInt(offsetIndex);
+      final int millis = data.getInt(offsetIndex + ${minor.millisecondsOffset});     
+      return DateUtilities.intervalDayStringBuilder(days, millis);
     }
-
     <#elseif minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense">
+
     public void get(int index, ${minor.class}Holder holder) {
       holder.start = index * VALUE_WIDTH;
       holder.buffer = data;
@@ -515,17 +469,19 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       // Get the BigDecimal object
       return DecimalUtility.getBigDecimalFromSparse(data, index * VALUE_WIDTH, ${minor.nDecimalDigits}, getField().getScale());
       <#else>
-      return DecimalUtility.getBigDecimalFromDense(data, index * VALUE_WIDTH, ${minor.nDecimalDigits}, getField().getScale(), ${minor.maxPrecisionDigits}, VALUE_WIDTH);
+      return DecimalUtility.getBigDecimalFromDense(data, index * VALUE_WIDTH,
+          ${minor.nDecimalDigits}, getField().getScale(),
+          ${minor.maxPrecisionDigits}, VALUE_WIDTH);
       </#if>
     }
-
     <#else>
-    public void get(int index, ${minor.class}Holder holder){
+
+    public void get(int index, ${minor.class}Holder holder) {
       holder.buffer = data;
       holder.start = index * VALUE_WIDTH;
     }
 
-    public void get(int index, Nullable${minor.class}Holder holder){
+    public void get(int index, Nullable${minor.class}Holder holder) {
       holder.isSet = 1;
       holder.buffer = data;
       holder.start = index * VALUE_WIDTH;
@@ -535,76 +491,61 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public ${friendlyType} getObject(int index) {
       return data.slice(index * VALUE_WIDTH, VALUE_WIDTH)
     }
-
     </#if>
     <#else> <#-- type.width <= 8 -->
+    
     public ${minor.javaType!type.javaType} get(int index) {
       return data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
-
     <#if type.width == 4>
+    
     public long getTwoAsLong(int index) {
       return data.getLong(index * VALUE_WIDTH);
     }
-
     </#if>
     <#if minor.class == "Date">
+
     @Override
     public ${friendlyType} getObject(int index) {
       org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
       date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
       return date;
     }
-
     <#elseif minor.class == "TimeStamp">
+
     @Override
     public ${friendlyType} getObject(int index) {
       org.joda.time.DateTime date = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
       date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
       return date;
     }
-
     <#elseif minor.class == "IntervalYear">
+
     @Override
     public ${friendlyType} getObject(int index) {
-      final int value = get(index);
-      final int years  = (value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      final int months = (value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      final Period p = new Period();
-      return p.plusYears(years).plusMonths(months);
+      return DateUtilities.fromIntervalYear(get(index));
     }
 
     public StringBuilder getAsStringBuilder(int index) {
-
-      int months  = data.getInt(index);
-
-      final int years  = (months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      months = (months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-
-      final String yearString = (Math.abs(years) == 1) ? " year " : " years ";
-      final String monthString = (Math.abs(months) == 1) ? " month " : " months ";
-
-      return(new StringBuilder().
-             append(years).append(yearString).
-             append(months).append(monthString));
+      return DateUtilities.intervalYearStringBuilder(data.getInt(index));
     }
-
     <#elseif minor.class == "Time">
+
     @Override
     public DateTime getObject(int index) {
       org.joda.time.DateTime time = new org.joda.time.DateTime(get(index), org.joda.time.DateTimeZone.UTC);
       time = time.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
       return time;
     }
-
     <#elseif minor.class == "Decimal9" || minor.class == "Decimal18">
+
     @Override
     public ${friendlyType} getObject(int index) {
       final BigInteger value = BigInteger.valueOf(((${type.boxedType})get(index)).${type.javaType}Value());
       return new BigDecimal(value, getField().getScale());
     }
-
     <#else>
+
     @Override
     public ${friendlyType} getObject(int index) {
       return get(index);
@@ -613,9 +554,9 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public ${minor.javaType!type.javaType} getPrimitiveObject(int index) {
       return get(index);
     }
-
     </#if>
-    public void get(int index, ${minor.class}Holder holder){
+
+    public void get(int index, ${minor.class}Holder holder) {
       <#if minor.class.startsWith("Decimal")>
       holder.scale = getField().getScale();
       holder.precision = getField().getPrecision();
@@ -624,7 +565,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
 
-    public void get(int index, Nullable${minor.class}Holder holder){
+    public void get(int index, Nullable${minor.class}Holder holder) {
       holder.isSet = 1;
       holder.value = data.get${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH);
     }
@@ -658,8 +599,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
      * @param value
      *          value to set
      */
-
   <#if (type.width > 8)>
+  
     public void set(int index, <#if (type.width > 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
       data.setBytes(index * VALUE_WIDTH, value, 0, VALUE_WIDTH);
     }
@@ -670,8 +611,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       }
       data.setBytes(index * VALUE_WIDTH, value, 0, VALUE_WIDTH);
     }
-
     <#if minor.class == "Interval">
+    
     public void set(int index, int months, int days, int milliseconds) {
       final int offsetIndex = index * VALUE_WIDTH;
       data.setInt(offsetIndex, months);
@@ -701,8 +642,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public void setSafe(int index, Nullable${minor.class}Holder holder) {
       setSafe(index, holder.months, holder.days, holder.milliseconds);
     }
-
     <#elseif minor.class == "IntervalDay">
+    
     public void set(int index, int days, int milliseconds) {
       final int offsetIndex = index * VALUE_WIDTH;
       data.setInt(offsetIndex, days);
@@ -728,11 +669,11 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       set(index, holder.days, holder.milliseconds);
     }
 
-    public void setSafe(int index, Nullable${minor.class}Holder holder){
+    public void setSafe(int index, Nullable${minor.class}Holder holder) {
       setSafe(index, holder.days, holder.milliseconds);
     }
-
     <#elseif minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense">
+    
     public void setSafe(int index, int start, DrillBuf buffer) {
       while(index >= getValueCapacity()) {
         reAlloc();
@@ -755,8 +696,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
     public void setSafe(int index, Nullable${minor.class}Holder holder) {
       setSafe(index, holder.start, holder.buffer);
     }
-
       <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
+      
     public void set(int index, BigDecimal value) {
       DecimalUtility.getSparseFromBigDecimal(value, data, index * VALUE_WIDTH,
            field.getScale(), field.getPrecision(), ${minor.nDecimalDigits});
@@ -768,13 +709,13 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       }
       set(index, value);
     }
-
       </#if>
-    public void set(int index, int start, DrillBuf buffer){
+      
+    public void set(int index, int start, DrillBuf buffer) {
       data.setBytes(index * VALUE_WIDTH, buffer, start, VALUE_WIDTH);
     }
-
     </#if>
+    
     @Override
     public void generateTestData(int count) {
       setValueCount(count);
@@ -782,13 +723,13 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
       final int valueCount = getAccessor().getValueCount();
       for(int i = 0; i < valueCount; i++, even = !even) {
         final byte b = even ? Byte.MIN_VALUE : Byte.MAX_VALUE;
-        for(int w = 0; w < VALUE_WIDTH; w++){
+        for(int w = 0; w < VALUE_WIDTH; w++) {
           data.setByte(i + w, b);
         }
       }
     }
-
   <#else> <#-- type.width <= 8 -->
+  
     public void set(int index, <#if (type.width >= 4)>${minor.javaType!type.javaType}<#else>int</#if> value) {
       data.set${(minor.javaType!type.javaType)?cap_first}(index * VALUE_WIDTH, value);
     }
@@ -855,8 +796,8 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
         }
       }
     }
-
   </#if> <#-- type.width -->
+  
     @Override
     public void setValueCount(int valueCount) {
       final int currentValueCapacity = getValueCapacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/ListWriters.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/ListWriters.java b/exec/vector/src/main/codegen/templates/ListWriters.java
index 16d41ec..f10cfc4 100644
--- a/exec/vector/src/main/codegen/templates/ListWriters.java
+++ b/exec/vector/src/main/codegen/templates/ListWriters.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -43,7 +43,10 @@ package org.apache.drill.exec.vector.complex.impl;
 public class ${mode}ListWriter extends AbstractFieldWriter {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${mode}ListWriter.class);
 
-  static enum Mode { INIT, IN_MAP, IN_LIST <#list vv.types as type><#list type.minor as minor>, IN_${minor.class?upper_case}</#list></#list> }
+  enum Mode {
+    INIT, IN_MAP, IN_LIST
+    <#list vv.types as type><#list type.minor as minor>,
+    IN_${minor.class?upper_case}</#list></#list> }
 
   private final String name;
   protected final ${containerClass} container;
@@ -69,7 +72,6 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
     if(writer != null) {
       writer.allocate();
     }
-
     <#if mode == "Repeated">
     container.allocateNew();
     </#if>
@@ -97,12 +99,14 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
   }
 
   public void setValueCount(int count){
-    if(innerVector != null) innerVector.getMutator().setValueCount(count);
+    if (innerVector != null) {
+      innerVector.getMutator().setValueCount(count);
+    }
   }
 
   @Override
   public MapWriter map() {
-    switch(mode) {
+    switch (mode) {
     case INIT:
       int vectorCount = container.size();
       final RepeatedMapVector vector = container.addOrGet(name, RepeatedMapVector.TYPE, RepeatedMapVector.class);
@@ -116,21 +120,23 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
       return writer;
     case IN_MAP:
       return writer;
+    default:
+      throw UserException
+        .unsupportedError()
+        .message(getUnsupportedErrorMsg("MAP", mode.name()))
+        .build(logger);
     }
-
-  throw UserException.unsupportedError().message(getUnsupportedErrorMsg("MAP", mode.name())).build(logger);
-
   }
 
   @Override
   public ListWriter list() {
-    switch(mode) {
+    switch (mode) {
     case INIT:
       final int vectorCount = container.size();
       final RepeatedListVector vector = container.addOrGet(name, RepeatedListVector.TYPE, RepeatedListVector.class);
       innerVector = vector;
       writer = new RepeatedListWriter(null, vector, this);
-      if(vectorCount != container.size()) {
+      if (vectorCount != container.size()) {
         writer.allocate();
       }
       writer.setPosition(${index});
@@ -138,10 +144,12 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
       return writer;
     case IN_LIST:
       return writer;
+    default:
+      throw UserException
+        .unsupportedError()
+        .message(getUnsupportedErrorMsg("LIST", mode.name()))
+        .build(logger);
     }
-
-  throw UserException.unsupportedError().message(getUnsupportedErrorMsg("LIST", mode.name())).build(logger);
-
   }
 
   <#list vv.types as type><#list type.minor as minor>
@@ -149,12 +157,11 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
   <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
-
   private static final MajorType ${upperName}_TYPE = Types.repeated(MinorType.${upperName});
 
   @Override
   public ${capName}Writer ${lowerName}() {
-    switch(mode) {
+    switch (mode) {
     case INIT:
       final int vectorCount = container.size();
       final Repeated${capName}Vector vector = container.addOrGet(name, ${upperName}_TYPE, Repeated${capName}Vector.class);
@@ -168,19 +175,22 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
       return writer;
     case IN_${upperName}:
       return writer;
+    default:
+      throw UserException
+         .unsupportedError()
+         .message(getUnsupportedErrorMsg("${upperName}", mode.name()))
+         .build(logger);
     }
-
-  throw UserException.unsupportedError().message(getUnsupportedErrorMsg("${upperName}", mode.name())).build(logger);
-
   }
+  
   </#list></#list>
-
+  @Override
   public MaterializedField getField() {
     return container.getField();
   }
-
   <#if mode == "Repeated">
-
+  
+  @Override
   public void startList() {
     final RepeatedListVector list = (RepeatedListVector) container;
     final RepeatedListVector.RepeatedMutator mutator = list.getMutator();
@@ -202,11 +212,13 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
     }
   }
 
+  @Override
   public void endList() {
     // noop, we initialize state at start rather than end.
   }
   <#else>
 
+  @Override
   public void setPosition(int index) {
     super.setPosition(index);
     if(writer != null) {
@@ -214,10 +226,12 @@ public class ${mode}ListWriter extends AbstractFieldWriter {
     }
   }
 
+  @Override
   public void startList() {
     // noop
   }
 
+  @Override
   public void endList() {
     // noop
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index fdb0200..93f8e7b 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -15,7 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
+import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.NullableVectorDefinitionSetter;
@@ -49,12 +51,9 @@ package org.apache.drill.exec.vector;
  */
 
 public final class ${className} extends BaseDataValueVector implements <#if type.major == "VarLen">VariableWidth<#else>FixedWidth</#if>Vector, NullableVector {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${className}.class);
 
   private final FieldReader reader = new Nullable${minor.class}ReaderImpl(Nullable${minor.class}Vector.this);
 
-  private final MaterializedField bitsField = MaterializedField.create("$bits$", Types.required(MinorType.UINT1));
-
   /**
    * Set value flag. Meaning:
    * <ul>
@@ -67,13 +66,26 @@ public final class ${className} extends BaseDataValueVector implements <#if type
    */
 
   private final UInt1Vector bits = new UInt1Vector(bitsField, allocator);
-  private final ${valuesName} values = new ${minor.class}Vector(field, allocator);
+
+  private final ${valuesName} values;
 
   private final Mutator mutator = new Mutator();
-  private final Accessor accessor = new Accessor();
+  private final Accessor accessor;
 
   public ${className}(MaterializedField field, BufferAllocator allocator) {
     super(field, allocator);
+    
+    // The values vector has its own name, and has the same type and attributes
+    // as the nullable vector. This ensures that
+    // things like scale and precision are preserved in the values vector.
+    
+    values = new ${minor.class}Vector(
+        MaterializedField.create(VALUES_VECTOR_NAME, field.getType()),
+        allocator);
+    
+    field.addChild(bits.getField());
+    field.addChild(values.getField());
+    accessor = new Accessor();
   }
 
   @Override
@@ -128,6 +140,11 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   }
 
   @Override
+  public int getAllocatedSize(){
+    return bits.getAllocatedSize() + values.getAllocatedSize();
+  }
+
+  @Override
   public DrillBuf getBuffer() {
     return values.getBuffer();
   }
@@ -138,6 +155,13 @@ public final class ${className} extends BaseDataValueVector implements <#if type
   @Override
   public UInt1Vector getBitsVector() { return bits; }
 
+  <#if type.major == "VarLen">
+  @Override
+  public UInt4Vector getOffsetVector() {
+    return ((VariableWidthVector) values).getOffsetVector();
+  }
+
+  </#if>
   @Override
   public void setInitialCapacity(int numRecords) {
     bits.setInitialCapacity(numRecords);

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/UnionListWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionListWriter.java b/exec/vector/src/main/codegen/templates/UnionListWriter.java
index c676769..81d5f9c 100644
--- a/exec/vector/src/main/codegen/templates/UnionListWriter.java
+++ b/exec/vector/src/main/codegen/templates/UnionListWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -32,15 +32,12 @@ package org.apache.drill.exec.vector.complex.impl;
  * This class is generated using freemarker and the ${.template_name} template.
  */
 
-@SuppressWarnings("unused")
 public class UnionListWriter extends AbstractFieldWriter {
 
   private ListVector vector;
   private UInt4Vector offsets;
   private PromotableWriter writer;
   private boolean inMap = false;
-  private String mapName;
-  private int lastIndex = 0;
 
   public UnionListWriter(ListVector vector) {
     super(null);
@@ -74,14 +71,10 @@ public class UnionListWriter extends AbstractFieldWriter {
   }
 
   @Override
-  public void close() throws Exception {
-
-  }
-
+  public void close() throws Exception { }
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign fields = minor.fields!type.fields />
   <#assign uncappedName = name?uncap_first/>
-
   <#if !minor.class?starts_with("Decimal")>
 
   @Override
@@ -92,16 +85,13 @@ public class UnionListWriter extends AbstractFieldWriter {
   @Override
   public ${name}Writer <#if uncappedName == "int">integer<#else>${uncappedName}</#if>(String name) {
     assert inMap;
-    mapName = name;
     final int nextOffset = offsets.getAccessor().get(idx() + 1);
     vector.getMutator().setNotNull(idx());
     writer.setPosition(nextOffset);
     ${name}Writer ${uncappedName}Writer = writer.<#if uncappedName == "int">integer<#else>${uncappedName}</#if>(name);
     return ${uncappedName}Writer;
   }
-
   </#if>
-
   </#list></#list>
 
   @Override
@@ -140,9 +130,7 @@ public class UnionListWriter extends AbstractFieldWriter {
   }
 
   @Override
-  public void endList() {
-
-  }
+  public void endList() { }
 
   @Override
   public void start() {
@@ -161,11 +149,9 @@ public class UnionListWriter extends AbstractFieldWriter {
       offsets.getMutator().setSafe(idx() + 1, nextOffset + 1);
     }
   }
-
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign fields = minor.fields!type.fields />
   <#assign uncappedName = name?uncap_first/>
-
   <#if !minor.class?starts_with("Decimal")>
 
   @Override
@@ -177,9 +163,6 @@ public class UnionListWriter extends AbstractFieldWriter {
     writer.write${name}(<#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
     offsets.getMutator().setSafe(idx() + 1, nextOffset + 1);
   }
-
   </#if>
-
   </#list></#list>
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/UnionVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionVector.java b/exec/vector/src/main/codegen/templates/UnionVector.java
index a46779d..248b010 100644
--- a/exec/vector/src/main/codegen/templates/UnionVector.java
+++ b/exec/vector/src/main/codegen/templates/UnionVector.java
@@ -32,9 +32,15 @@ import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.drill.exec.vector.complex.impl.ComplexCopier;
+
+import com.google.common.base.Preconditions;
+
 import org.apache.drill.exec.util.CallBack;
 import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
+import org.apache.drill.exec.record.MaterializedField;
+
+import com.google.common.annotations.VisibleForTesting;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -43,43 +49,91 @@ import org.apache.drill.exec.memory.AllocationManager.BufferLedger;
 
 
 /**
- * A vector which can hold values of different types. It does so by using a MapVector which contains a vector for each
- * primitive type that is stored. MapVector is used in order to take advantage of its serialization/deserialization methods,
- * as well as the addOrGet method.
+ * A vector which can hold values of different types. It does so by using a
+ * MapVector which contains a vector for each primitive type that is stored.
+ * MapVector is used in order to take advantage of its
+ * serialization/deserialization methods, as well as the addOrGet method.
  *
- * For performance reasons, UnionVector stores a cached reference to each subtype vector, to avoid having to do the map lookup
- * each time the vector is accessed.
+ * For performance reasons, UnionVector stores a cached reference to each
+ * subtype vector, to avoid having to do the map lookup each time the vector is
+ * accessed.
  */
 public class UnionVector implements ValueVector {
 
+  public static final int NULL_MARKER = 0;
+  public static final String TYPE_VECTOR_NAME = "types";
+  public static final String INTERNAL_MAP_NAME = "internal";
+  
+  private static final MajorType MAJOR_TYPES[] = new MajorType[MinorType.values().length];
+  
+  static {
+    MAJOR_TYPES[MinorType.MAP.ordinal()] = Types.optional(MinorType.MAP);
+    MAJOR_TYPES[MinorType.LIST.ordinal()] = Types.optional(MinorType.LIST);
+    <#list vv.types as type>
+      <#list type.minor as minor>
+        <#assign name = minor.class?cap_first />
+        <#assign fields = minor.fields!type.fields />
+        <#assign uncappedName = name?uncap_first/>
+        <#if !minor.class?starts_with("Decimal")>
+    MAJOR_TYPES[MinorType.${name?upper_case}.ordinal()] = Types.optional(MinorType.${name?upper_case});
+        </#if>
+      </#list>
+    </#list>
+  }
+
   private MaterializedField field;
   private BufferAllocator allocator;
   private Accessor accessor = new Accessor();
   private Mutator mutator = new Mutator();
   private int valueCount;
 
+  /**
+   * Map which holds one vector for each subtype, along with a vector that indicates
+   * types and the null state. There appears to be no reason other than convenience
+   * for using a map. Future implementations may wish to store vectors directly in
+   * the union vector, but must then implement the required vector serialization/
+   * deserialization and other functionality.
+   */
+  
   private MapVector internalMap;
+  
+  /**
+   * Cached type vector. The vector's permament location is in the
+   * internal map, it is cached for performance. Call
+   * {@link #getTypeVector()} to get the cached copy, or to refresh
+   * the cache from the internal map if not set.
+   */
+  
   private UInt1Vector typeVector;
 
-  private MapVector mapVector;
-  private ListVector listVector;
+  /**
+   * Set of cached vectors that duplicate vectors store in the
+   * internal map. Used to avoid a name lookup on every access.
+   * The cache is populated as vectors are added. But, after the
+   * union is sent over the wire, the map is populated, but the
+   * array is not. It will be repopulated upon first access to
+   * the deserialized vectors.
+   */
+  
+  private ValueVector cachedSubtypes[] = new ValueVector[MinorType.values().length];
 
   private FieldReader reader;
-  private NullableBitVector bit;
-
-  private int singleType = 0;
-  private ValueVector singleVector;
-  private MajorType majorType;
 
   private final CallBack callBack;
 
   public UnionVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
+    
+    // The metadata may start off listing subtypes for which vectors
+    // do not actually exist. It appears that the semantics are to list
+    // the subtypes that *could* appear. For example, in a sort we may
+    // have two types: one batch has type A, the other type B, but the
+    // batches must list both A and B as subtypes.
+    
     this.field = field.clone();
     this.allocator = allocator;
-    this.internalMap = new MapVector("internal", allocator, callBack);
-    this.typeVector = internalMap.addOrGet("types", Types.required(MinorType.UINT1), UInt1Vector.class);
+    this.internalMap = new MapVector(INTERNAL_MAP_NAME, allocator, callBack);
+    this.typeVector = internalMap.addOrGet(TYPE_VECTOR_NAME, Types.required(MinorType.UINT1), UInt1Vector.class);
     this.field.addChild(internalMap.getField().clone());
-    this.majorType = field.getType();
     this.callBack = callBack;
   }
 
@@ -87,94 +141,180 @@ public class UnionVector implements ValueVector {
   public BufferAllocator getAllocator() {
     return allocator;
   }
-
+  
   public List<MinorType> getSubTypes() {
-    return majorType.getSubTypeList();
-  }
-
+    return field.getType().getSubTypeList();
+  }
+  
+  @SuppressWarnings("unchecked")
+  public <T extends ValueVector> T subtype(MinorType type) {
+    return (T) cachedSubtypes[type.ordinal()];
+  }
+
+  
+  /**
+   * Add an externally-created subtype vector. The vector must represent a type that
+   * does not yet exist in the union, and must be of OPTIONAL mode. Does not call
+   * the callback since the client (presumably) knows that it is adding the type.
+   * The caller must also allocate the buffer for the vector.
+   * 
+   * @param vector subtype vector to add
+   */
+  
+  public void addType(ValueVector vector) {
+    MinorType type = vector.getField().getType().getMinorType();
+    assert subtype(type) == null;
+    assert vector.getField().getType().getMode() == DataMode.OPTIONAL;
+    assert vector.getField().getName().equals(type.name().toLowerCase());
+    cachedSubtypes[type.ordinal()] = vector;
+    internalMap.putChild(type.name(), vector);
+    addSubType(type);
+  }
+  
   public void addSubType(MinorType type) {
-    if (majorType.getSubTypeList().contains(type)) {
+    if (field.getType().getSubTypeList().contains(type)) {
       return;
     }
-    majorType =  MajorType.newBuilder(this.majorType).addSubType(type).build();
-    field = MaterializedField.create(field.getName(), majorType);
+    field.replaceType(
+        MajorType.newBuilder(field.getType()).addSubType(type).build());
     if (callBack != null) {
       callBack.doWork();
     }
   }
 
-  private static final MajorType MAP_TYPE = Types.optional(MinorType.MAP);
+  /**
+   * "Classic" way to add a subtype when working directly with a union vector.
+   * Creates the vector, adds it to the internal structures and creates a
+   * new buffer of the default size.
+   * 
+   * @param type the type to add
+   * @param vectorClass class of the vector to create
+   * @return typed form of the new value vector
+   */
+  
+  private <T extends ValueVector> T classicAddType(MinorType type, Class<? extends ValueVector> vectorClass) {
+    int vectorCount = internalMap.size();
+    @SuppressWarnings("unchecked")
+    T vector = (T) internalMap.addOrGet(type.name().toLowerCase(), MAJOR_TYPES[type.ordinal()], vectorClass);
+    cachedSubtypes[type.ordinal()] = vector;
+    if (internalMap.size() > vectorCount) {
+      vector.allocateNew();
+      addSubType(type);
+      if (callBack != null) {
+        callBack.doWork();
+      }
+    }
+    return vector;
+  }
 
   public MapVector getMap() {
+    MapVector mapVector = subtype(MinorType.MAP);
     if (mapVector == null) {
-      int vectorCount = internalMap.size();
-      mapVector = internalMap.addOrGet("map", MAP_TYPE, MapVector.class);
-      addSubType(MinorType.MAP);
-      if (internalMap.size() > vectorCount) {
-        mapVector.allocateNew();
-      }
+      mapVector = classicAddType(MinorType.MAP, MapVector.class);
     }
     return mapVector;
   }
-  <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
-  <#assign fields = minor.fields!type.fields />
-  <#assign uncappedName = name?uncap_first/>
-  <#if !minor.class?starts_with("Decimal")>
 
-  private Nullable${name}Vector ${uncappedName}Vector;
-  private static final MajorType ${name?upper_case}_TYPE = Types.optional(MinorType.${name?upper_case});
+  public ListVector getList() {
+    ListVector listVector = subtype(MinorType.LIST);
+    if (listVector == null) {
+      listVector = classicAddType(MinorType.LIST, ListVector.class);
+    }
+    return listVector;
+  }
+  <#-- Generating a method per type is probably overkill. However, existing code
+       depends on these methods, so didn't want to remove them. Over time, a
+       generic, parameterized addOrGet(MinorType type) would be more compact.
+       Would need a function to map from minor type to vector class, which
+       can be generated here or in TypeHelper. -->
+  <#list vv.types as type>
+    <#list type.minor as minor>
+      <#assign name = minor.class?cap_first />
+      <#assign fields = minor.fields!type.fields />
+      <#assign uncappedName = name?uncap_first/>
+      <#if !minor.class?starts_with("Decimal")>
 
   public Nullable${name}Vector get${name}Vector() {
-    if (${uncappedName}Vector == null) {
-      int vectorCount = internalMap.size();
-      ${uncappedName}Vector = internalMap.addOrGet("${uncappedName}", ${name?upper_case}_TYPE, Nullable${name}Vector.class);
-      addSubType(MinorType.${name?upper_case});
-      if (internalMap.size() > vectorCount) {
-        ${uncappedName}Vector.allocateNew();
-      }
+    Nullable${name}Vector vector = subtype(MinorType.${name?upper_case});
+    if (vector == null) {
+      vector = classicAddType(MinorType.${name?upper_case}, Nullable${name}Vector.class);
     }
-    return ${uncappedName}Vector;
+    return vector;
   }
-  </#if>
-  </#list></#list>
-
-  private static final MajorType LIST_TYPE = Types.optional(MinorType.LIST);
-
-  public ListVector getList() {
-    if (listVector == null) {
-      int vectorCount = internalMap.size();
-      listVector = internalMap.addOrGet("list", LIST_TYPE, ListVector.class);
-      addSubType(MinorType.LIST);
-      if (internalMap.size() > vectorCount) {
-        listVector.allocateNew();
-      }
+      </#if>
+    </#list>
+  </#list>
+  
+  /**
+   * Add or get a type member given the type.
+   * 
+   * @param type the type of the vector to retrieve
+   * @return the (potentially newly created) vector that backs the given type
+   */
+  
+  public ValueVector getMember(MinorType type) {
+    switch (type) {
+    case MAP:
+      return getMap();
+    case LIST:
+      return getList();
+  <#-- This awkard switch statement and call to type-specific method logic
+       can be generalized as described above. -->
+  <#list vv.types as type>
+    <#list type.minor as minor>
+      <#assign name = minor.class?cap_first />
+      <#assign fields = minor.fields!type.fields />
+      <#assign uncappedName = name?uncap_first/>
+      <#if !minor.class?starts_with("Decimal")>
+    case ${name?upper_case}:
+      return get${name}Vector();
+      </#if>
+    </#list>
+  </#list>
+    default:
+      throw new UnsupportedOperationException(type.toString());
     }
-    return listVector;
+  }
+  
+  @SuppressWarnings("unchecked")
+  public <T extends ValueVector> T member(MinorType type) {
+    return (T) getMember(type);
   }
 
   public int getTypeValue(int index) {
-    return typeVector.getAccessor().get(index);
+    return getTypeVector().getAccessor().get(index);
   }
 
   public UInt1Vector getTypeVector() {
+    if (typeVector == null) {
+      typeVector = (UInt1Vector) internalMap.getChild(TYPE_VECTOR_NAME);
+    }
     return typeVector;
   }
+  
+  @VisibleForTesting
+  public MapVector getTypeMap() {
+    return internalMap;
+  }
 
   @Override
   public void allocateNew() throws OutOfMemoryException {
     internalMap.allocateNew();
-    if (typeVector != null) {
-      typeVector.zeroVector();
-    }
+    getTypeVector().zeroVector();
+  }
+
+  public void allocateNew(int rowCount) throws OutOfMemoryException {
+    // The map vector does not have a form that takes a row count,
+    // but it should.
+    internalMap.allocateNew();
+    getTypeVector().zeroVector();
   }
 
   @Override
   public boolean allocateNewSafe() {
     boolean safe = internalMap.allocateNewSafe();
     if (safe) {
-      if (typeVector != null) {
-        typeVector.zeroVector();
-      }
+      getTypeVector().zeroVector();
     }
     return safe;
   }
@@ -184,7 +324,7 @@ public class UnionVector implements ValueVector {
 
   @Override
   public int getValueCapacity() {
-    return Math.min(typeVector.getValueCapacity(), internalMap.getValueCapacity());
+    return Math.min(getTypeVector().getValueCapacity(), internalMap.getValueCapacity());
   }
 
   @Override
@@ -200,12 +340,7 @@ public class UnionVector implements ValueVector {
 
   @Override
   public void collectLedgers(Set<BufferLedger> ledgers) {
-    // Most vectors are held inside the internal map.
-
     internalMap.collectLedgers(ledgers);
-    if (bit != null) {
-      bit.collectLedgers(ledgers);
-    }
   }
 
   @Override
@@ -231,7 +366,6 @@ public class UnionVector implements ValueVector {
   public void transferTo(UnionVector target) {
     internalMap.makeTransferPair(target.internalMap).transfer();
     target.valueCount = valueCount;
-    target.majorType = majorType;
   }
 
   public void copyFrom(int inIndex, int outIndex, UnionVector from) {
@@ -249,16 +383,48 @@ public class UnionVector implements ValueVector {
     copyFromSafe(fromIndex, toIndex, (UnionVector) from);
   }
 
+  /**
+   * Add a vector that matches the argument. Transfer the buffer from the argument
+   * to the new vector.
+   * 
+   * @param v the vector to clone and add
+   * @return the cloned vector that now holds the data from the argument
+   */
+  
   public ValueVector addVector(ValueVector v) {
     String name = v.getField().getType().getMinorType().name().toLowerCase();
     MajorType type = v.getField().getType();
+    MinorType minorType = type.getMinorType();
     Preconditions.checkState(internalMap.getChild(name) == null, String.format("%s vector already exists", name));
-    final ValueVector newVector = internalMap.addOrGet(name, type, BasicTypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
+    final ValueVector newVector = internalMap.addOrGet(name, type, BasicTypeHelper.getValueVectorClass(minorType, type.getMode()));
     v.makeTransferPair(newVector).transfer();
     internalMap.putChild(name, newVector);
-    addSubType(v.getField().getType().getMinorType());
+    cachedSubtypes[minorType.ordinal()] = newVector;
+    addSubType(minorType);
     return newVector;
   }
+  
+  // Called from SchemaUtil
+  
+  public ValueVector setFirstType(ValueVector v, int newValueCount) {
+    
+    // We can't check that this really is the first subtype since
+    // the subtypes can be declared before vectors are added.
+    
+    Preconditions.checkState(accessor.getValueCount() == 0);
+    final ValueVector vv = addVector(v);
+    MinorType type = v.getField().getType().getMinorType();
+    ValueVector.Accessor vAccessor = vv.getAccessor();
+    for (int i = 0; i < newValueCount; i++) {
+      if (! vAccessor.isNull(i)) {
+        mutator.setType(i, type);
+      } else {
+        mutator.setNull(i);
+      }
+    }
+    mutator.setValueCount(newValueCount);
+    return vv;
+  }
 
   @Override
   public void toNullable(ValueVector nullableVector) {
@@ -267,7 +433,7 @@ public class UnionVector implements ValueVector {
 
   private class TransferImpl implements TransferPair {
 
-    UnionVector to;
+    private final UnionVector to;
 
     public TransferImpl(MaterializedField field, BufferAllocator allocator) {
       to = new UnionVector(field, allocator, null);
@@ -319,13 +485,12 @@ public class UnionVector implements ValueVector {
 
   @Override
   public UserBitShared.SerializedField getMetadata() {
-    SerializedField.Builder b = getField() //
-            .getAsBuilder() //
-            .setBufferLength(getBufferSize()) //
-            .setValueCount(valueCount);
-
-    b.addChild(internalMap.getMetadata());
-    return b.build();
+    return getField()
+            .getAsBuilder()
+            .setBufferLength(getBufferSize())
+            .setValueCount(valueCount)
+            .addChild(internalMap.getMetadata())
+            .build();
   }
 
   @Override
@@ -366,18 +531,16 @@ public class UnionVector implements ValueVector {
 
   @Override
   public Iterator<ValueVector> iterator() {
-    List<ValueVector> vectors = Lists.newArrayList(internalMap.iterator());
-    vectors.add(typeVector);
-    return vectors.iterator();
+    return internalMap.iterator();
   }
 
   public class Accessor extends BaseValueVector.BaseAccessor {
 
     @Override
     public Object getObject(int index) {
-      int type = typeVector.getAccessor().get(index);
+      int type = getTypeVector().getAccessor().get(index);
       switch (type) {
-      case 0:
+      case NULL_MARKER:
         return null;
       <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
       <#assign fields = minor.fields!type.fields />
@@ -386,7 +549,6 @@ public class UnionVector implements ValueVector {
       case MinorType.${name?upper_case}_VALUE:
         return get${name}Vector().getAccessor().getObject(index);
       </#if>
-
       </#list></#list>
       case MinorType.MAP_VALUE:
         return getMap().getAccessor().getObject(index);
@@ -412,7 +574,12 @@ public class UnionVector implements ValueVector {
 
     @Override
     public boolean isNull(int index) {
-      return typeVector.getAccessor().get(index) == 0;
+      
+      // Note that type code == 0 is used to indicate a null.
+      // This corresponds to the LATE type, not the NULL type.
+      // This is presumably an artifact of an earlier implementation...
+      
+      return getTypeVector().getAccessor().get(index) == NULL_MARKER;
     }
 
     public int isSet(int index) {
@@ -449,14 +616,12 @@ public class UnionVector implements ValueVector {
         break;
       </#if>
       </#list></#list>
-      case MAP: {
+      case MAP:
         ComplexCopier.copy(reader, writer);
         break;
-      }
-      case LIST: {
+      case LIST:
         ComplexCopier.copy(reader, writer);
         break;
-      }
       default:
         throw new UnsupportedOperationException();
       }
@@ -475,7 +640,11 @@ public class UnionVector implements ValueVector {
     </#list></#list>
 
     public void setType(int index, MinorType type) {
-      typeVector.getMutator().setSafe(index, type.getNumber());
+      getTypeVector().getMutator().setSafe(index, type.getNumber());
+    }
+    
+    public void setNull(int index) {
+      getTypeVector().getMutator().setSafe(index, NULL_MARKER);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/UnionWriter.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionWriter.java b/exec/vector/src/main/codegen/templates/UnionWriter.java
index 7a123b4..58cc455 100644
--- a/exec/vector/src/main/codegen/templates/UnionWriter.java
+++ b/exec/vector/src/main/codegen/templates/UnionWriter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -29,10 +29,11 @@ package org.apache.drill.exec.vector.complex.impl;
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
-@SuppressWarnings("unused")
+
 public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
 
-  UnionVector data;
+  // Accessed by UnionReader
+  protected UnionVector data;
   private MapWriter mapWriter;
   private UnionListWriter listWriter;
   private List<BaseWriter> writers = Lists.newArrayList();
@@ -59,7 +60,6 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
     }
   }
 
-
   @Override
   public void start() {
     data.getMutator().setType(idx(), MinorType.MAP);
@@ -145,11 +145,9 @@ public class UnionWriter extends AbstractFieldWriter implements FieldWriter {
     get${name}Writer().write${name}(<#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
   }
   </#if>
-
   </#list></#list>
 
-  public void writeNull() {
-  }
+  public void writeNull() { }
 
   @Override
   public MapWriter map() {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index a29194a..87dbe95 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -50,17 +50,14 @@ package org.apache.drill.exec.vector;
  * variable, this width is used as a guess for certain calculations.</li>
  * <li>The equivalent Java primitive is '${minor.javaType!type.javaType}'<li>
  * </ul>
- * NB: this class is automatically generated from ${.template_name} and ValueVectorTypes.tdd using FreeMarker.
+ * NB: this class is automatically generated from <tt>${.template_name}</tt>
+ * and <tt>ValueVectorTypes.tdd</tt> using FreeMarker.
  */
 
 public final class ${minor.class}Vector extends BaseDataValueVector implements VariableWidthVector {
 
-  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
   private static final int INITIAL_BYTE_COUNT = Math.min(INITIAL_VALUE_ALLOCATION * DEFAULT_RECORD_BYTE_COUNT, MAX_BUFFER_SIZE);
-  private static final int MIN_BYTE_COUNT = 4096;
-  public final static String OFFSETS_VECTOR_NAME = "$offsets$";
-
-  private final MaterializedField offsetsField = MaterializedField.create(OFFSETS_VECTOR_NAME, Types.required(MinorType.UINT4));
+  
   private final UInt${type.width}Vector offsetVector = new UInt${type.width}Vector(offsetsField, allocator);
   private final FieldReader reader = new ${minor.class}ReaderImpl(${minor.class}Vector.this);
 
@@ -175,6 +172,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements V
     return offsetVector.getBuffer().memoryAddress();
   }
 
+  @Override
   public UInt${type.width}Vector getOffsetVector() {
     return offsetVector;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
index 992ae03..67594fe 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.expr.fn.impl;
 
-import org.joda.time.Period;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
 import org.joda.time.format.DateTimeFormatterBuilder;
@@ -26,18 +25,23 @@ import org.joda.time.format.DateTimeParser;
 
 import com.carrotsearch.hppc.ObjectIntHashMap;
 
-// Utility class for Date, DateTime, TimeStamp, Interval data types
-public class DateUtility {
+/**
+ * Utility class for Date, DateTime, TimeStamp, Interval data types.
+ * <p>
+ * WARNING: This class is excluded from the JDBC driver. If vectors refer
+ * to this code, they will fail when called from JDBC.
+ */
 
+public class DateUtility {
 
-    /* We have a hashmap that stores the timezone as the key and an index as the value
-     * While storing the timezone in value vectors, holders we only use this index. As we
-     * reconstruct the timestamp, we use this index to index through the array timezoneList
-     * and get the corresponding timezone and pass it to joda-time
-     */
+  /* We have a hashmap that stores the timezone as the key and an index as the value
+   * While storing the timezone in value vectors, holders we only use this index. As we
+   * reconstruct the timestamp, we use this index to index through the array timezoneList
+   * and get the corresponding timezone and pass it to joda-time
+   */
   public static ObjectIntHashMap<String> timezoneMap = new ObjectIntHashMap<String>();
 
-    public static String[] timezoneList =  {"Africa/Abidjan",
+  public static String[] timezoneList =  {  "Africa/Abidjan",
                                             "Africa/Accra",
                                             "Africa/Addis_Ababa",
                                             "Africa/Algiers",
@@ -612,71 +616,53 @@ public class DateUtility {
                                             "WET",
                                             "Zulu"};
 
-    static {
-      for (int i = 0; i < timezoneList.length; i++) {
-        timezoneMap.put(timezoneList[i], i);
-      }
+  static {
+    for (int i = 0; i < timezoneList.length; i++) {
+      timezoneMap.put(timezoneList[i], i);
     }
+  }
 
-    public static final DateTimeFormatter formatDate        = DateTimeFormat.forPattern("yyyy-MM-dd");
-    public static final DateTimeFormatter formatTimeStamp    = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS");
-    public static final DateTimeFormatter formatTimeStampTZ = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ");
-    public static final DateTimeFormatter formatTime        = DateTimeFormat.forPattern("HH:mm:ss.SSS");
+  public static final DateTimeFormatter formatDate        = DateTimeFormat.forPattern("yyyy-MM-dd");
+  public static final DateTimeFormatter formatTimeStamp    = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS");
+  public static final DateTimeFormatter formatTimeStampTZ = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS ZZZ");
+  public static final DateTimeFormatter formatTime        = DateTimeFormat.forPattern("HH:mm:ss.SSS");
 
-    public static DateTimeFormatter dateTimeTZFormat = null;
-    public static DateTimeFormatter timeFormat = null;
+  public static DateTimeFormatter dateTimeTZFormat = null;
+  public static DateTimeFormatter timeFormat = null;
 
-    public static final int yearsToMonths = 12;
-    public static final int hoursToMillis = 60 * 60 * 1000;
-    public static final int minutesToMillis = 60 * 1000;
-    public static final int secondsToMillis = 1000;
-    public static final int monthToStandardDays = 30;
-    public static final long monthsToMillis = 2592000000L; // 30 * 24 * 60 * 60 * 1000
-    public static final int daysToStandardMillis = 24 * 60 * 60 * 1000;
 
 
   public static int getIndex(String timezone) {
-        return timezoneMap.get(timezone);
-    }
+    return timezoneMap.get(timezone);
+  }
 
-    public static String getTimeZone(int index) {
-        return timezoneList[index];
-    }
-
-    // Function returns the date time formatter used to parse date strings
-    public static DateTimeFormatter getDateTimeFormatter() {
+  public static String getTimeZone(int index) {
+    return timezoneList[index];
+  }
 
-        if (dateTimeTZFormat == null) {
-            DateTimeFormatter dateFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
-            DateTimeParser optionalTime = DateTimeFormat.forPattern(" HH:mm:ss").getParser();
-            DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
-            DateTimeParser optionalZone = DateTimeFormat.forPattern(" ZZZ").getParser();
+  // Returns the date time formatter used to parse date strings
+  public static DateTimeFormatter getDateTimeFormatter() {
 
-            dateTimeTZFormat = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).appendOptional(optionalZone).toFormatter();
-        }
+    if (dateTimeTZFormat == null) {
+      DateTimeFormatter dateFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
+      DateTimeParser optionalTime = DateTimeFormat.forPattern(" HH:mm:ss").getParser();
+      DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
+      DateTimeParser optionalZone = DateTimeFormat.forPattern(" ZZZ").getParser();
 
-        return dateTimeTZFormat;
+      dateTimeTZFormat = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).appendOptional(optionalZone).toFormatter();
     }
 
-    // Function returns time formatter used to parse time strings
-    public static DateTimeFormatter getTimeFormatter() {
-        if (timeFormat == null) {
-            DateTimeFormatter timeFormatter = DateTimeFormat.forPattern("HH:mm:ss");
-            DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
-            timeFormat = new DateTimeFormatterBuilder().append(timeFormatter).appendOptional(optionalSec).toFormatter();
-        }
-        return timeFormat;
-    }
-
-    public static int monthsFromPeriod(Period period){
-      return (period.getYears() * yearsToMonths) + period.getMonths();
-    }
+    return dateTimeTZFormat;
+  }
 
-    public static int millisFromPeriod(final Period period){
-      return (period.getHours() * hoursToMillis) +
-      (period.getMinutes() * minutesToMillis) +
-      (period.getSeconds() * secondsToMillis) +
-      (period.getMillis());
+  // Returns time formatter used to parse time strings
+  public static DateTimeFormatter getTimeFormatter() {
+    if (timeFormat == null) {
+      DateTimeFormatter timeFormatter = DateTimeFormat.forPattern("HH:mm:ss");
+      DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
+      timeFormat = new DateTimeFormatterBuilder().append(timeFormatter).appendOptional(optionalSec).toFormatter();
     }
+    return timeFormat;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index b4b23c7..fa4d276 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -26,6 +26,7 @@ import java.util.Objects;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.apache.drill.exec.proto.UserBitShared.NamePart;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
@@ -38,7 +39,7 @@ import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 
 public class MaterializedField {
   private final String name;
-  private final MajorType type;
+  private MajorType type;
   // use an ordered set as existing code relies on order (e,g. parquet writer)
   private final LinkedHashSet<MaterializedField> children;
 
@@ -87,13 +88,47 @@ public class MaterializedField {
     children.add(field);
   }
 
+  public void removeChild(MaterializedField field) {
+    children.remove(field);
+  }
+
+  /**
+   * Replace the type with a new one that has the same minor type
+   * and mode, but with perhaps different details.
+   * <p>
+   * The type is immutable. But, it contains subtypes, used or lists
+   * and unions. To add a subtype, we must create a whole new major type.
+   * <p>
+   * It appears that the <tt>MaterializedField</tt> class was also meant
+   * to be immutable. But, it holds the children for a map, and contains
+   * methods to add children. So, it is not immutable.
+   * <p>
+   * This method allows evolving a list or union without the need to create
+   * a new <tt>MaterializedField</tt>. Doing so is problematic for nested
+   * maps because the map (or list, or union) holds onto the
+   * <tt>MaterializedField</tt>'s of its children. There is no way for
+   * an inner map to reach out and change the child of its parent.
+   * <p>
+   * By allowing the non-critical metadata to change, we preserve the
+   * child relationships as a list or union evolves.
+   * @param type
+   */
+
+  public void replaceType(MajorType newType) {
+    assert type.getMinorType() == newType.getMinorType();
+    assert type.getMode() == newType.getMode();
+    type = newType;
+  }
+
   @Override
   public MaterializedField clone() {
     return withPathAndType(name, getType());
   }
 
   public MaterializedField cloneEmpty() {
-    return create(name, type);
+    return create(name, type.toBuilder()
+        .clearSubType()
+        .build());
   }
 
   public MaterializedField withType(MajorType type) {
@@ -213,16 +248,77 @@ public class MaterializedField {
     // But, unset fields are equivalent to 0. Can't use the protobuf-provided
     // isEquals(), that treats set and unset fields as different.
 
+    if (! Types.isEquivalent(type, other.type)) {
+      return false;
+    }
+
+    // Compare children -- but only for maps, not the internal children
+    // for Varchar, repeated or nullable types.
+
+    if (type.getMinorType() != MinorType.MAP) {
+      return true;
+    }
+
+    if (children == null || other.children == null) {
+      return children == other.children;
+    }
+    if (children.size() != other.children.size()) {
+      return false;
+    }
+
+    // Maps are name-based, not position. But, for our
+    // purposes, we insist on identical ordering.
+
+    Iterator<MaterializedField> thisIter = children.iterator();
+    Iterator<MaterializedField> otherIter = other.children.iterator();
+    while (thisIter.hasNext()) {
+      MaterializedField thisChild = thisIter.next();
+      MaterializedField otherChild = otherIter.next();
+      if (! thisChild.isEquivalent(otherChild)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Determine if the present column schema can be promoted to the
+   * given schema. Promotion is possible if the schemas are
+   * equivalent, or if required mode is promoted to nullable, or
+   * if scale or precision can be increased.
+   *
+   * @param other the field to which this one is to be promoted
+   * @return true if promotion is possible, false otherwise
+   */
+
+  public boolean isPromotableTo(MaterializedField other, boolean allowModeChange) {
+    if (! name.equalsIgnoreCase(other.name)) {
+      return false;
+    }
+
+    // Requires full type equality, including fields such as precision and scale.
+    // But, unset fields are equivalent to 0. Can't use the protobuf-provided
+    // isEquals(), that treats set and unset fields as different.
+
     if (type.getMinorType() != other.type.getMinorType()) {
       return false;
     }
     if (type.getMode() != other.type.getMode()) {
-      return false;
+
+      // Modes differ, but type can be promoted from required to
+      // nullable
+
+      if (! allowModeChange) {
+        return false;
+      }
+      if (! (type.getMode() == DataMode.REQUIRED && other.type.getMode() == DataMode.OPTIONAL)) {
+        return false;
+      }
     }
-    if (type.getScale() != other.type.getScale()) {
+    if (type.getScale() > other.type.getScale()) {
       return false;
     }
-    if (type.getPrecision() != other.type.getPrecision()) {
+    if (type.getPrecision() > other.type.getPrecision()) {
       return false;
     }
 
@@ -233,7 +329,7 @@ public class MaterializedField {
       return true;
     }
 
-    if (children == null || other.children == null) {
+    if (children == null  ||  other.children == null) {
       return children == other.children;
     }
     if (children.size() != other.children.size()) {
@@ -248,7 +344,7 @@ public class MaterializedField {
     while (thisIter.hasNext()) {
       MaterializedField thisChild = thisIter.next();
       MaterializedField otherChild = otherIter.next();
-      if (! thisChild.isEquivalent(otherChild)) {
+      if (! thisChild.isPromotableTo(otherChild, allowModeChange)) {
         return false;
       }
     }
@@ -269,30 +365,45 @@ public class MaterializedField {
   @Override
   public String toString() {
     final int maxLen = 10;
-    String childString = children != null && !children.isEmpty() ? toString(children, maxLen) : "";
     StringBuilder builder = new StringBuilder();
     builder
-        .append(name)
-        .append("(")
-        .append(type.getMinorType().name());
+      .append("[`")
+      .append(name)
+      .append("` (")
+      .append(type.getMinorType().name());
 
     if (type.hasPrecision()) {
       builder.append("(");
       builder.append(type.getPrecision());
       if (type.hasScale()) {
-        builder.append(",");
+        builder.append(", ");
         builder.append(type.getScale());
       }
       builder.append(")");
     }
 
     builder
-        .append(":")
-        .append(type.getMode().name())
-        .append(")")
-        .append(childString);
+      .append(":")
+      .append(type.getMode().name())
+      .append(")");
+
+    if (type.getSubTypeCount() > 0) {
+      builder
+        .append(", subtypes=(")
+        .append(type.getSubTypeList().toString())
+        .append(")");
+    }
 
-    return builder.toString();
+    if (children != null && ! children.isEmpty()) {
+      builder
+        .append(", children=(")
+        .append(toString(children, maxLen))
+        .append(")");
+    }
+
+    return builder
+        .append("]")
+        .toString();
   }
 
   /**
@@ -307,7 +418,6 @@ public class MaterializedField {
 
   private String toString(Collection<?> collection, int maxLen) {
     StringBuilder builder = new StringBuilder();
-    builder.append(" [");
     int i = 0;
     for (Iterator<?> iterator = collection.iterator(); iterator.hasNext() && i < maxLen; i++) {
       if (i > 0){
@@ -315,7 +425,6 @@ public class MaterializedField {
       }
       builder.append(iterator.next());
     }
-    builder.append("]");
     return builder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index 4391e8c..b27d7ba 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -107,6 +107,9 @@ public abstract class BaseDataValueVector extends BaseValueVector {
 
   @Override
   public void exchange(ValueVector other) {
+
+    // Exchange the data buffers
+
     BaseDataValueVector target = (BaseDataValueVector) other;
     DrillBuf temp = data;
     data = target.data;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index 63f4528..864d8e4 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -17,20 +17,20 @@
  */
 package org.apache.drill.exec.vector;
 
-import io.netty.buffer.DrillBuf;
-
 import java.util.Collections;
 import java.util.Iterator;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
-
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.TransferPair;
 
+import com.google.common.base.Preconditions;
+
+import io.netty.buffer.DrillBuf;
+
 public abstract class BaseValueVector implements ValueVector {
+
   /**
    * Physical maximum allocation. This is the value prior to Drill 1.11.
    * This size causes memory fragmentation. Please use

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/DateUtilities.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/DateUtilities.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/DateUtilities.java
new file mode 100644
index 0000000..4ea460b
--- /dev/null
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/DateUtilities.java
@@ -0,0 +1,191 @@
+/*
+ * 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.vector;
+
+import org.joda.time.Period;
+
+/**
+ * Utility class for Date, DateTime, TimeStamp, Interval data types.
+ * <p>
+ * WARNING: This class is included from the JDBC driver. There is another, similar
+ * class called <tt>org.apache.drill.exec.expr.fn.impl.DateUtility</tt>. If vectors refer
+ * to that class, they will fail when called from JDBC. So, place code here if
+ * it is needed by JDBC, in the other class if only needed by the Drill engine.
+ * (This is a very poor design, but it is what it is.)
+ */
+
+public class DateUtilities {
+
+  public static final int yearsToMonths = 12;
+  public static final int hoursToMillis = 60 * 60 * 1000;
+  public static final int minutesToMillis = 60 * 1000;
+  public static final int secondsToMillis = 1000;
+  public static final int monthToStandardDays = 30;
+  public static final long monthsToMillis = 2592000000L; // 30 * 24 * 60 * 60 * 1000
+  public static final int daysToStandardMillis = 24 * 60 * 60 * 1000;
+
+  public static int monthsFromPeriod(Period period){
+    return (period.getYears() * yearsToMonths) + period.getMonths();
+  }
+
+  public static int periodToMillis(final Period period){
+    return (period.getHours() * hoursToMillis) +
+           (period.getMinutes() * minutesToMillis) +
+           (period.getSeconds() * secondsToMillis) +
+           (period.getMillis());
+  }
+
+  public static int toMonths(int years, int months) {
+    return years * yearsToMonths + months;
+  }
+
+  public static int periodToMonths(Period value) {
+    return value.getYears() * yearsToMonths + value.getMonths();
+  }
+
+  public static Period fromIntervalYear(int value) {
+    final int years  = (value / yearsToMonths);
+    final int months = (value % yearsToMonths);
+    return new Period()
+        .plusYears(years)
+        .plusMonths(months);
+  }
+
+  public static StringBuilder intervalYearStringBuilder(int months) {
+    final int years = months / yearsToMonths;
+    months %= yearsToMonths;
+
+    return new StringBuilder()
+           .append(years)
+           .append(pluralify("year", years))
+           .append(" ")
+           .append(months)
+           .append(pluralify("month", months));
+  }
+
+  public static StringBuilder intervalYearStringBuilder(Period value) {
+    return intervalYearStringBuilder(
+        value.getYears() * 12 + value.getMonths());
+  }
+
+  public static String pluralify(String term, int value) {
+    term = (Math.abs(value) == 1) ? term : term + "s";
+    return " " + term;
+  }
+
+  public static Period fromIntervalDay(int days, int millis) {
+    return new Period()
+        .plusDays(days)
+        .plusMillis(millis);
+  }
+
+  public static StringBuilder intervalDayStringBuilder(int days, int millis) {
+
+    final int hours  = millis / (hoursToMillis);
+    millis %= (hoursToMillis);
+
+    final int minutes = millis / (minutesToMillis);
+    millis %= (minutesToMillis);
+
+    final int seconds = millis / (secondsToMillis);
+    millis %= (secondsToMillis);
+
+    StringBuilder buf = new StringBuilder()
+            .append(days)
+            .append(pluralify("day", days))
+            .append(" ")
+            .append(hours)
+            .append(":")
+            .append(asTwoDigits(minutes))
+            .append(":")
+            .append(asTwoDigits(seconds));
+    if (millis != 0) {
+      buf.append(".")
+         .append(millis);
+    }
+    return buf;
+  }
+
+  public static StringBuilder intervalDayStringBuilder(Period value) {
+    return intervalDayStringBuilder(
+        value.getDays(),
+        periodToMillis(value));
+  }
+
+  public static Period fromInterval(int months, int days, int millis) {
+    return new Period()
+        .plusMonths(months)
+        .plusDays(days)
+        .plusMillis(millis);
+  }
+
+  public static String asTwoDigits(int value) {
+    return String.format("%02d", value);
+  }
+
+  public static StringBuilder intervalStringBuilder(int months, int days, int millis) {
+
+    final int years = months / yearsToMonths;
+    months %= yearsToMonths;
+
+    final int hours  = millis / hoursToMillis;
+    millis %= hoursToMillis;
+
+    final int minutes = millis / minutesToMillis;
+    millis %= minutesToMillis;
+
+    final int seconds = millis / secondsToMillis;
+    millis %= secondsToMillis;
+
+    StringBuilder buf = new StringBuilder()
+           .append(years)
+           .append(pluralify("year", years))
+           .append(" ")
+           .append(months)
+           .append(pluralify("month", months))
+           .append(" ")
+           .append(days)
+           .append(pluralify("day", days))
+           .append(" ")
+           .append(hours)
+           .append(":")
+           .append(asTwoDigits(minutes))
+           .append(":")
+           .append(asTwoDigits(seconds));
+    if (millis != 0) {
+      buf.append(".")
+         .append(millis);
+    }
+    return buf;
+  }
+
+  public static StringBuilder intervalStringBuilder(Period value) {
+    return intervalStringBuilder(
+        value.getYears() * 12 + value.getMonths(),
+        value.getDays(),
+        periodToMillis(value));
+  }
+
+  public static int timeToMillis(int hours, int minutes, int seconds, int millis) {
+    return ((hours * 60 +
+             minutes) * 60 +
+            seconds) * 1000 +
+           millis;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/NullableVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/NullableVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/NullableVector.java
index 51b5e0c..80b732a 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/NullableVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/NullableVector.java
@@ -17,7 +17,13 @@
  */
 package org.apache.drill.exec.vector;
 
-public interface NullableVector extends ValueVector{
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.record.MaterializedField;
+
+public interface NullableVector extends ValueVector {
+
+  MaterializedField bitsField = MaterializedField.create(BITS_VECTOR_NAME, Types.required(MinorType.UINT1));
 
   ValueVector getBitsVector();
   ValueVector getValuesVector();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
index bc06803..44a467e 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueVector.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.vector;
 import java.io.Closeable;
 import java.util.Set;
 
-import com.google.common.annotations.VisibleForTesting;
 import io.netty.buffer.DrillBuf;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
@@ -86,6 +85,12 @@ public interface ValueVector extends Closeable, Iterable<ValueVector> {
 
   int MAX_ROW_COUNT = Character.MAX_VALUE + 1;
 
+  // Commonly-used internal vector names
+
+  String BITS_VECTOR_NAME = "$bits$";
+  String OFFSETS_VECTOR_NAME = "$offsets$";
+  String VALUES_VECTOR_NAME = "$values$";
+
   /**
    * Allocate new buffers. ValueVector implements logic to determine how much to allocate.
    * @throws OutOfMemoryException Thrown if no memory can be allocated.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
index f5373d0..1e77417 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/VariableWidthVector.java
@@ -17,8 +17,24 @@
  */
 package org.apache.drill.exec.vector;
 
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.record.MaterializedField;
+
 public interface VariableWidthVector extends ValueVector {
 
+  int DEFAULT_RECORD_BYTE_COUNT = 8;
+  int MIN_BYTE_COUNT = 4096;
+  MaterializedField offsetsField = MaterializedField.create(OFFSETS_VECTOR_NAME, Types.required(MinorType.UINT4));
+
+  interface VariableWidthAccessor extends Accessor {
+    int getValueLength(int index);
+  }
+
+  interface VariableWidthMutator extends Mutator {
+    void setValueLengthSafe(int index, int length);
+  }
+
   /**
    * Allocate a new memory space for this vector.  Must be called prior to using the ValueVector.
    *
@@ -39,13 +55,7 @@ public interface VariableWidthVector extends ValueVector {
   @Override
   VariableWidthAccessor getAccessor();
 
-  interface VariableWidthAccessor extends Accessor {
-    int getValueLength(int index);
-  }
-
   int getCurrentSizeInBytes();
 
-  interface VariableWidthMutator extends Mutator {
-    void setValueLengthSafe(int index, int length);
-  }
+  UInt4Vector getOffsetVector();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
index 5ac28c5..5515b7a 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/AbstractMapVector.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.vector.ValueVector;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
-/*
+/**
  * Base class for MapVectors. Currently used by RepeatedMapVector and MapVector
  */
 public abstract class AbstractMapVector extends AbstractContainerVector {
@@ -47,14 +47,14 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
 
   protected AbstractMapVector(MaterializedField field, BufferAllocator allocator, CallBack callBack) {
     super(field.clone(), allocator, callBack);
-    MaterializedField clonedField = field.clone();
     // create the hierarchy of the child vectors based on the materialized field
-    for (MaterializedField child : clonedField.getChildren()) {
-      if (!child.equals(BaseRepeatedValueVector.OFFSETS_FIELD)) {
-        final String fieldName = child.getName();
-        final ValueVector v = BasicTypeHelper.getNewVector(child, allocator, callBack);
-        putVector(fieldName, v);
+    for (MaterializedField child : field.getChildren()) {
+      if (child.getName().equals(BaseRepeatedValueVector.OFFSETS_FIELD.getName())) {
+        continue;
       }
+      final String fieldName = child.getName();
+      final ValueVector v = BasicTypeHelper.getNewVector(child, allocator, callBack);
+      putVector(fieldName, v);
     }
   }
 
@@ -77,13 +77,13 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
     boolean success = false;
     try {
       for (final ValueVector v : vectors.values()) {
-        if (!v.allocateNewSafe()) {
+        if (! v.allocateNewSafe()) {
           return false;
         }
       }
       success = true;
     } finally {
-      if (!success) {
+      if (! success) {
         clear();
       }
     }
@@ -145,7 +145,7 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
 
   private boolean nullFilled(ValueVector vector) {
     for (int r = 0; r < vector.getAccessor().getValueCount(); r++) {
-      if (!vector.getAccessor().isNull(r)) {
+      if (! vector.getAccessor().isNull(r)) {
         return false;
       }
     }
@@ -304,4 +304,17 @@ public abstract class AbstractMapVector extends AbstractContainerVector {
     }
     return count;
   }
+
+  @Override
+  public void exchange(ValueVector other) {
+    AbstractMapVector otherMap = (AbstractMapVector) other;
+    if (vectors.size() != otherMap.vectors.size()) {
+      throw new IllegalStateException("Maps have different column counts");
+    }
+    for (int i = 0; i < vectors.size(); i++) {
+      assert vectors.getByOrdinal(i).getField().isEquivalent(
+          otherMap.vectors.getByOrdinal(i).getField());
+      vectors.getByOrdinal(i).exchange(otherMap.vectors.getByOrdinal(i));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
index 8472f80..4b0c1b5 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/BaseRepeatedValueVector.java
@@ -85,7 +85,6 @@ public abstract class BaseRepeatedValueVector extends BaseValueVector implements
     return success;
   }
 
-
   @Override
   public UInt4Vector getOffsetVector() { return offsets; }
 


[02/11] drill git commit: DRILL-6093 : Account for simple columns in project cpu costing

Posted by am...@apache.org.
DRILL-6093 : Account for simple columns in project cpu costing

close apache/drill#1093


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

Branch: refs/heads/master
Commit: a2058fd16b555cebd0a539d5c9b993e0d628eced
Parents: 7d1e81a
Author: Gautam Parai <gp...@maprtech.com>
Authored: Tue Jan 16 15:16:16 2018 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 14:40:33 2018 -0800

----------------------------------------------------------------------
 .../drill/exec/planner/common/DrillProjectRelBase.java   | 10 ++++++----
 .../test/java/org/apache/drill/TestProjectPushDown.java  | 11 +++++++++++
 2 files changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a2058fd1/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
index 44d708e..8141a8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
@@ -74,10 +74,12 @@ public abstract class DrillProjectRelBase extends Project implements DrillRelNod
     if (PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
       return super.computeSelfCost(planner, mq).multiplyBy(.1);
     }
-
-    // cost is proportional to the number of rows and number of columns being projected
-    double rowCount = nonSimpleFieldCount > 0 ? mq.getRowCount(this) : 0;
-    double cpuCost = DrillCostBase.PROJECT_CPU_COST * rowCount * nonSimpleFieldCount;
+    double rowCount = mq.getRowCount(this);
+    // Attribute small cost for projecting simple fields. In reality projecting simple columns in not free and
+    // this allows projection pushdown/project-merge rules to kick-in thereby eliminating unneeded columns from
+    // the projection.
+    double cpuCost = DrillCostBase.PROJECT_CPU_COST * rowCount * nonSimpleFieldCount
+        + (this.getRowType().getFieldCount() - nonSimpleFieldCount) * rowCount * DrillCostBase.BASE_CPU_COST;
 
     DrillCostFactory costFactory = (DrillCostFactory) planner.getCostFactory();
     return costFactory.makeCost(rowCount, cpuCost, 0, 0);

http://git-wip-us.apache.org/repos/asf/drill/blob/a2058fd1/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
index ad55a0d..013c954 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
@@ -287,6 +287,17 @@ public class TestProjectPushDown extends PlanTestBase {
     }
   }
 
+  @Test
+  public void testProjectPushdownPastJoinWithJoinPushExpressions() throws Exception {
+    final String query = "SELECT L.L_QUANTITY FROM cp.`tpch/lineitem.parquet` L, cp.`tpch/orders.parquet` O" +
+        " WHERE cast(L.L_ORDERKEY as int) = cast(O.O_ORDERKEY as int)";
+    final String[] expectedPatterns = {
+        ".*HashJoin.*", "Project.*\\(L_QUANTITY.*CAST\\(\\$0\\)\\:INTEGER.*", "Project.*CAST\\(\\$0\\)\\:INTEGER.*"};
+    // L_ORDERKEY, O_ORDERKEY should not be present in the projects below the join
+    final String[] excludedPatterns = {".*Project\\(L_ORDERKEY=.*", ".*Project\\(O_ORDERKEY=.*"};
+    PlanTestBase.testPlanMatchingPatterns(query, expectedPatterns, excludedPatterns);
+  }
+
   protected void testPushDown(PushDownTestInstance test) throws Exception {
     testPhysicalPlan(test.getSql(), test.getExpected());
   }


[09/11] drill git commit: DRILL-6049: Misc. hygiene and code cleanup changes

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSeparatePlanningTest.java
----------------------------------------------------------------------
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 c99f0a7..8b84995 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,66 +17,56 @@
  */
 package org.apache.drill.exec;
 
-import static org.junit.Assert.*;
-import io.netty.buffer.DrillBuf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import java.nio.file.Paths;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.concurrent.ExecutionException;
 
-import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder.QuerySummary;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetReader;
 import org.apache.drill.categories.PlannerTest;
 import org.apache.drill.categories.SlowTest;
-import org.apache.drill.common.DrillAutoCloseables;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.client.PrintingResultsListener;
-import org.apache.drill.exec.client.QuerySubmitter.Format;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.UserBitShared.QueryData;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos.QueryPlanFragments;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.ConnectionThrottle;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.util.VectorUtil;
-import org.apache.drill.exec.vector.ValueVector;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.junit.experimental.categories.Category;
 
 /**
- * Class to test different planning use cases (separate form query execution)
+ * Class to test different planning use cases (separate from query execution)
  *
  */
 @Category({SlowTest.class, PlannerTest.class})
-public class DrillSeparatePlanningTest extends BaseTestQuery {
+public class DrillSeparatePlanningTest extends ClusterTest {
   @BeforeClass
   public static void setupFiles() {
     dirTestWatcher.copyResourceToRoot(Paths.get("multilevel", "json"));
     dirTestWatcher.copyResourceToRoot(Paths.get("multilevel", "csv"));
   }
 
-  @Test(timeout=60000)
+  @Before
+  public void testSetup() throws Exception {
+    ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher)
+        .clusterSize(2);
+    startCluster(builder);
+  }
+
+  @Test(timeout=60_000)
   public void testSingleFragmentQuery() throws Exception {
-    final String query = "SELECT * FROM cp.`employee.json` where  employee_id > 1 and  employee_id < 1000";
+    final String query = "SELECT * FROM cp.`employee.json` where employee_id > 1 and employee_id < 1000";
 
     QueryPlanFragments planFragments = getFragmentsHelper(query);
 
@@ -85,251 +75,134 @@ public class DrillSeparatePlanningTest extends BaseTestQuery {
     assertEquals(1, planFragments.getFragmentsCount());
     assertTrue(planFragments.getFragments(0).getLeafFragment());
 
-    getResultsHelper(planFragments);
+    QuerySummary summary = client.queryBuilder().plan(planFragments.getFragmentsList()).run();
+    assertEquals(997, summary.recordCount());
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=60_000)
   public void testMultiMinorFragmentSimpleQuery() throws Exception {
     final String query = "SELECT o_orderkey FROM dfs.`multilevel/json`";
 
     QueryPlanFragments planFragments = getFragmentsHelper(query);
 
     assertNotNull(planFragments);
-
     assertTrue((planFragments.getFragmentsCount() > 1));
 
-    for ( PlanFragment planFragment : planFragments.getFragmentsList()) {
+    for (PlanFragment planFragment : planFragments.getFragmentsList()) {
       assertTrue(planFragment.getLeafFragment());
     }
 
-    getResultsHelper(planFragments);
+    int rowCount = getResultsHelper(planFragments);
+    assertEquals(120, rowCount);
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=60_000)
   public void testMultiMinorFragmentComplexQuery() throws Exception {
     final String query = "SELECT dir0, sum(o_totalprice) FROM dfs.`multilevel/json` group by dir0 order by dir0";
 
     QueryPlanFragments planFragments = getFragmentsHelper(query);
 
     assertNotNull(planFragments);
-
     assertTrue((planFragments.getFragmentsCount() > 1));
 
     for ( PlanFragment planFragment : planFragments.getFragmentsList()) {
       assertTrue(planFragment.getLeafFragment());
     }
 
-    getResultsHelper(planFragments);
+    int rowCount = getResultsHelper(planFragments);
+    assertEquals(8, rowCount);
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=60_000)
   public void testPlanningNoSplit() throws Exception {
     final String query = "SELECT dir0, sum(o_totalprice) FROM dfs.`multilevel/json` group by dir0 order by dir0";
 
-    updateTestCluster(2, config);
-
-    List<QueryDataBatch> results = client.runQuery(QueryType.SQL, "alter session set `planner.slice_target`=1");
-    for(QueryDataBatch batch : results) {
-      batch.release();
-    }
-
-    DrillRpcFuture<QueryPlanFragments> queryFragmentsFutures = client.planQuery(QueryType.SQL, query, false);
-
-    final QueryPlanFragments planFragments = queryFragmentsFutures.get();
-
-    assertNotNull(planFragments);
+    client.alterSession("planner.slice_target", 1);
+    try {
+      final QueryPlanFragments planFragments = client.planQuery(query);
 
-    assertTrue((planFragments.getFragmentsCount() > 1));
+      assertNotNull(planFragments);
+      assertTrue((planFragments.getFragmentsCount() > 1));
 
-    PlanFragment rootFragment = planFragments.getFragments(0);
-    assertFalse(rootFragment.getLeafFragment());
+      PlanFragment rootFragment = planFragments.getFragments(0);
+      assertFalse(rootFragment.getLeafFragment());
 
-    getCombinedResultsHelper(planFragments);
+      QuerySummary summary = client.queryBuilder().plan(planFragments.getFragmentsList()).run();
+      assertEquals(3, summary.recordCount());
+    }
+    finally {
+      client.resetSession("planner.slice_target");
+    }
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=60_000)
   public void testPlanningNegative() throws Exception {
     final String query = "SELECT dir0, sum(o_totalprice) FROM dfs.`multilevel/json` group by dir0 order by dir0";
 
-    updateTestCluster(2, config);
     // LOGICAL is not supported
-    DrillRpcFuture<QueryPlanFragments> queryFragmentsFutures = client.planQuery(QueryType.LOGICAL, query, false);
-
-    final QueryPlanFragments planFragments = queryFragmentsFutures.get();
+    final QueryPlanFragments planFragments = client.planQuery(QueryType.LOGICAL, query, false);
 
     assertNotNull(planFragments);
-
     assertNotNull(planFragments.getError());
-
     assertTrue(planFragments.getFragmentsCount()==0);
-
   }
 
-  @Test(timeout=60000)
+  @Test(timeout=60_000)
   public void testPlanning() throws Exception {
     final String query = "SELECT dir0, columns[3] FROM dfs.`multilevel/csv` order by dir0";
 
-    updateTestCluster(2, config);
-
-    List<QueryDataBatch> results = client.runQuery(QueryType.SQL, "alter session set `planner.slice_target`=1");
-    for(QueryDataBatch batch : results) {
-      batch.release();
+    client.alterSession("planner.slice_target", 1);
+    try {
+      // Original version, but no reason to dump output to test results.
+//      long rows = client.queryBuilder().sql(query).print(Format.TSV, VectorUtil.DEFAULT_COLUMN_WIDTH);
+      QuerySummary summary = client.queryBuilder().sql(query).run();
+      assertEquals(120, summary.recordCount());
     }
-    AwaitableUserResultsListener listener =
-        new AwaitableUserResultsListener(new PrintingResultsListener(client.getConfig(), Format.TSV, VectorUtil.DEFAULT_COLUMN_WIDTH));
-    client.runQuery(QueryType.SQL, query, listener);
-    @SuppressWarnings("unused")
-    int rows = listener.await();
-  }
-
-  private QueryPlanFragments getFragmentsHelper(final String query) throws InterruptedException, ExecutionException, RpcException {
-    updateTestCluster(2, config);
-
-    List<QueryDataBatch> results = client.runQuery(QueryType.SQL, "alter session set `planner.slice_target`=1");
-    for(QueryDataBatch batch : results) {
-      batch.release();
+    finally {
+      client.resetSession("planner.slice_target");
     }
+  }
 
-    DrillRpcFuture<QueryPlanFragments> queryFragmentsFutures = client.planQuery(QueryType.SQL, query, true);
+  private QueryPlanFragments getFragmentsHelper(final String query) {
+    client.alterSession("planner.slice_target", 1);
+    try {
+      QueryPlanFragments planFragments = client.planQuery(QueryType.SQL, query, true);
 
-    final QueryPlanFragments planFragments = queryFragmentsFutures.get();
+      // Uncomment for debugging.
 
-    for (PlanFragment fragment : planFragments.getFragmentsList()) {
-      System.out.println(fragment.getFragmentJson());
+//      for (PlanFragment fragment : planFragments.getFragmentsList()) {
+//        System.out.println(fragment.getFragmentJson());
+//      }
+      return planFragments;
+    }
+    finally {
+      client.resetSession("planner.slice_target");
     }
-
-    return planFragments;
   }
 
-  private void getResultsHelper(final QueryPlanFragments planFragments) throws Exception {
+  private int getResultsHelper(final QueryPlanFragments planFragments) throws Exception {
+    int totalRows = 0;
     for (PlanFragment fragment : planFragments.getFragmentsList()) {
       DrillbitEndpoint assignedNode = fragment.getAssignment();
-      @SuppressWarnings("resource")
-      DrillClient fragmentClient = new DrillClient(true);
-      Properties props = new Properties();
-      props.setProperty("drillbit", assignedNode.getAddress() + ":" + assignedNode.getUserPort());
-      fragmentClient.connect(props);
-
-      ShowResultsUserResultsListener myListener = new ShowResultsUserResultsListener(getAllocator());
-      AwaitableUserResultsListener listenerBits =
-          new AwaitableUserResultsListener(myListener);
-      fragmentClient.runQuery(QueryType.SQL, "select hostname, user_port from sys.drillbits where `current`=true",
-          listenerBits);
-      int row = listenerBits.await();
-      assertEquals(1, row);
-      List<Map<String,String>> records = myListener.getRecords();
-      assertEquals(1, records.size());
-      Map<String,String> record = records.get(0);
-      assertEquals(2, record.size());
-      Iterator<Entry<String, String>> iter = record.entrySet().iterator();
-      Entry<String, String> entry;
-      String host = null;
-      String port = null;
-      for (int i = 0; i < 2; i++) {
-       entry = iter.next();
-       if (entry.getKey().equalsIgnoreCase("hostname")) {
-          host = entry.getValue();
-        } else if (entry.getKey().equalsIgnoreCase("user_port")) {
-          port = entry.getValue();
-        } else {
-          fail("Unknown field: " + entry.getKey());
-        }
-       }
-      assertTrue(props.getProperty("drillbit").equalsIgnoreCase(host+":" + port));
+      ClientFixture fragmentClient = cluster.client(assignedNode.getAddress(), assignedNode.getUserPort());
+
+      RowSet rowSet = fragmentClient.queryBuilder().sql("select hostname, user_port from sys.drillbits where `current`=true").rowSet();
+      assertEquals(1, rowSet.rowCount());
+      RowSetReader reader = rowSet.reader();
+      assertTrue(reader.next());
+      String host = reader.scalar("hostname").getString();
+      int port = reader.scalar("user_port").getInt();
+      rowSet.clear();
+
+      assertEquals(assignedNode.getAddress(), host);
+      assertEquals(assignedNode.getUserPort(), port);
 
       List<PlanFragment> fragmentList = Lists.newArrayList();
       fragmentList.add(fragment);
-      AwaitableUserResultsListener listener =
-          new AwaitableUserResultsListener(new SilentListener());
-      fragmentClient.runQuery(QueryType.EXECUTION, fragmentList, listener);
-      @SuppressWarnings("unused")
-      int rows = listener.await();
+      QuerySummary summary = fragmentClient.queryBuilder().plan(fragmentList).run();
+      totalRows += summary.recordCount();
       fragmentClient.close();
     }
-  }
-
-  private void getCombinedResultsHelper(final QueryPlanFragments planFragments) throws Exception {
-      ShowResultsUserResultsListener myListener = new ShowResultsUserResultsListener(getAllocator());
-      @SuppressWarnings("unused")
-      AwaitableUserResultsListener listenerBits =
-          new AwaitableUserResultsListener(myListener);
-      AwaitableUserResultsListener listener =
-          new AwaitableUserResultsListener(new SilentListener());
-      client.runQuery(QueryType.EXECUTION, planFragments.getFragmentsList(), listener);
-      @SuppressWarnings("unused")
-      int rows = listener.await();
-  }
-
-  /**
-   * Helper class to get results
-   *
-   */
-  static class ShowResultsUserResultsListener implements UserResultsListener {
-
-    private QueryId queryId;
-    private final RecordBatchLoader loader;
-    private final BufferAllocator allocator;
-    private UserException ex;
-    private List<Map<String,String>> records = Lists.newArrayList();
-
-    public ShowResultsUserResultsListener(BufferAllocator allocator) {
-      this.loader = new RecordBatchLoader(allocator);
-      this.allocator = allocator;
-    }
-
-    public QueryId getQueryId() {
-      return queryId;
-    }
-
-    public List<Map<String, String>> getRecords() {
-      return records;
-    }
-
-    public UserException getEx() {
-      return ex;
-    }
-
-    @Override
-    public void queryIdArrived(QueryId queryId) {
-     this.queryId = queryId;
-    }
-
-    @Override
-    public void submissionFailed(UserException ex) {
-      DrillAutoCloseables.closeNoChecked(allocator);
-      this.ex = ex;
-    }
-
-    @Override
-    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
-      QueryData queryHeader = result.getHeader();
-      int rows = queryHeader.getRowCount();
-      try {
-        if ( result.hasData() ) {
-          DrillBuf data = result.getData();
-          loader.load(queryHeader.getDef(), data);
-          for (int i = 0; i < rows; i++) {
-             Map<String,String> record = Maps.newHashMap();
-            for (VectorWrapper<?> vw : loader) {
-              final String field = vw.getValueVector().getMetadata().getNamePart().getName();
-              final ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
-              final Object value = i < accessor.getValueCount() ? accessor.getObject(i) : null;
-              final String display = value == null ? null : value.toString();
-              record.put(field, display);
-            }
-            records.add(record);
-          }
-          loader.clear();
-        }
-        result.release();
-      } catch (SchemaChangeException e) {
-        fail(e.getMessage());
-      }
-
-    }
-
-    @Override
-    public void queryCompleted(QueryState state) {
-    }
+    return totalRows;
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 9ade940..e60533b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -38,6 +38,7 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
 import org.apache.drill.exec.util.GuavaPatcher;
+import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.DrillTest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,6 +46,8 @@ import org.joda.time.DateTimeUtils;
 import org.joda.time.DateTimeZone;
 import org.junit.After;
 import org.junit.BeforeClass;
+import org.junit.ClassRule;
+
 import java.io.IOException;
 import java.text.DateFormatSymbols;
 import java.util.Locale;
@@ -52,6 +55,9 @@ import java.util.Locale;
 
 public class ExecTest extends DrillTest {
 
+  @ClassRule
+  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
+
   protected static SystemOptionManager optionManager;
   static {
     GuavaPatcher.patch();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
index 3c41c81..91ce653 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index 94a9f12..eaf5e02 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -156,7 +156,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
   protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorType[] colTypes, String[] expectFirstTwoValues, BitControl.PlanFragment planFragment) throws Exception {
     @SuppressWarnings("resource")
     final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    @SuppressWarnings("resource")
     final Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
 
     bit1.run();
@@ -173,7 +172,6 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     final MockTableDef.MockScanEntry entry = new MockTableDef.MockScanEntry(10, false, 0, 1, columns);
     final MockSubScanPOP scanPOP = new MockSubScanPOP("testTable", false, java.util.Collections.singletonList(entry));
 
-    @SuppressWarnings("resource")
     final ScanBatch batch = createMockScanBatch(bit1, scanPOP, planFragment);
 
     batch.next();
@@ -202,7 +200,7 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     try {
       final FragmentContext context =
           new FragmentContext(bit.getContext(), planFragment, null, bit.getContext().getFunctionImplementationRegistry());
-      return creator.getBatch(context,scanPOP, children);
+      return (ScanBatch) creator.getBatch(context, scanPOP, children);
     } catch (Exception ex) {
       throw new DrillRuntimeException("Error when setup fragment context" + ex);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
index e7d0a97..4860869 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TopNBatchTest.java
@@ -135,7 +135,7 @@ public class TopNBatchTest extends PopUnitTestBase {
         VectorContainer resultContainer = queue.getHyperBatch();
         resultContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE);
 
-        RowSet.HyperRowSet actualHyperSet = new HyperRowSetImpl(resultContainer, queue.getFinalSv4());
+        RowSet.HyperRowSet actualHyperSet = HyperRowSetImpl.fromContainer(resultContainer, queue.getFinalSv4());
         new RowSetComparison(expectedRowSet).verify(actualHyperSet);
       } finally {
         if (expectedRowSet != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
index 7be6195..f517b1d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggrSpill.java
@@ -29,6 +29,7 @@ import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.LogFixture;
 import org.apache.drill.test.ProfileParser;
 import org.apache.drill.test.QueryBuilder;
@@ -47,7 +48,7 @@ import static org.junit.Assert.assertTrue;
  * Test spilling for the Hash Aggr operator (using the mock reader)
  */
 @Category({SlowTest.class, OperatorTest.class})
-public class TestHashAggrSpill {
+public class TestHashAggrSpill extends DrillTest {
 
   @Rule
   public final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
index 90183d9..7a66f43 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/window/TestWindowFrame.java
@@ -271,7 +271,6 @@ public class TestWindowFrame extends BaseTestQuery {
       .run();
   }
 
-
   @Test
   public void testLag() throws Exception {
     testBuilder()

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
index 563d97e..a79ecf5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestExternalSort.java
@@ -41,14 +41,26 @@ public class TestExternalSort extends BaseTestQuery {
 
   @Test
   public void testNumericTypesManaged() throws Exception {
-    testNumericTypes( false );
+    testNumericTypes(false);
   }
 
   @Test
   public void testNumericTypesLegacy() throws Exception {
-    testNumericTypes( true );
+    testNumericTypes(true);
   }
 
+  /**
+   * Test union type support in sort using numeric types: BIGINT and FLOAT8
+   * Drill does not support union types fully. Sort was adapted to handle them.
+   * This test simply verifies that the sort handles these types, even though
+   * Drill does not.
+   *
+   * @param testLegacy
+   *          true to test the old (pre-1.11) sort, false to test the new (1.11
+   *          and later) sort
+   * @throws Exception
+   */
+
   private void testNumericTypes(boolean testLegacy) throws Exception {
     final int record_count = 10000;
     final String tableDirName = "numericTypes";
@@ -103,8 +115,9 @@ public class TestExternalSort extends BaseTestQuery {
 
   private String getOptions(boolean testLegacy) {
     String options = "alter session set `exec.enable_union_type` = true";
-    options += ";alter session set `" + ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED_OPTION.getOptionName() + "` = " +
-        Boolean.toString(testLegacy);
+    options += ";alter session set `"
+        + ExecConstants.EXTERNAL_SORT_DISABLE_MANAGED_OPTION.getOptionName()
+        + "` = " + Boolean.toString(testLegacy);
     return options;
   }
 
@@ -159,10 +172,10 @@ public class TestExternalSort extends BaseTestQuery {
     }
 
     TestBuilder builder = testBuilder()
-            .sqlQuery("select * from dfs.`%s` order by a desc", tableDirName)
-            .ordered()
-            .optionSettingQueriesForTestQuery(getOptions(testLegacy))
-            .baselineColumns("a");
+        .sqlQuery("select * from dfs.`%s` order by a desc", tableDirName)
+        .ordered()
+        .optionSettingQueriesForTestQuery(getOptions(testLegacy))
+        .baselineColumns("a");
     // Strings come first because order by is desc
     for (int i = record_count; i >= 0;) {
       i--;
@@ -225,12 +238,13 @@ public class TestExternalSort extends BaseTestQuery {
       rowSet.clear();
     }
 
-    // Test framework currently doesn't handle changing schema (i.e. new columns) on the client side
+    // Test framework currently doesn't handle changing schema (i.e. new
+    // columns) on the client side
     TestBuilder builder = testBuilder()
-            .sqlQuery("select a, b, c from dfs.`%s` order by a desc", tableDirName)
-            .ordered()
-            .optionSettingQueriesForTestQuery(getOptions(testLegacy))
-            .baselineColumns("a", "b", "c");
+        .sqlQuery("select a, b, c from dfs.`%s` order by a desc", tableDirName)
+        .ordered()
+        .optionSettingQueriesForTestQuery(getOptions(testLegacy))
+        .baselineColumns("a", "b", "c");
     for (int i = record_count; i >= 0;) {
       builder.baselineValues((long) i, (long) i--, null);
       if (i >= 0) {
@@ -238,6 +252,9 @@ public class TestExternalSort extends BaseTestQuery {
       }
     }
     builder.go();
+
+    // TODO: Useless test: just dumps to console
+
     test("select * from dfs.`%s` order by a desc", tableDirName);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index 04a1df8..2cd1793 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
 import org.apache.drill.test.DrillTest;
 import org.apache.drill.test.ClusterFixtureBuilder;
 import org.apache.drill.categories.SlowTest;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
index cd408cb..e106171 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/SortTestUtilities.java
@@ -32,8 +32,8 @@ import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrapper.BatchMerger;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.TupleMetadata;
+import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
@@ -63,12 +63,10 @@ public class SortTestUtilities {
   }
 
   @SuppressWarnings("resource")
-  public static PriorityQueueCopierWrapper makeCopier(OperatorFixture fixture, String sortOrder, String nullOrder) {
+  public static Sort makeCopierConfig(String sortOrder, String nullOrder) {
     FieldReference expr = FieldReference.getWithQuotedRef("key");
     Ordering ordering = new Ordering(sortOrder, expr, nullOrder);
-    Sort popConfig = new Sort(null, Lists.newArrayList(ordering), false);
-    OperatorContext opContext = fixture.operatorContext(popConfig);
-    return new PriorityQueueCopierWrapper(opContext);
+    return new Sort(null, Lists.newArrayList(ordering), false);
   }
 
   public static class CopierTester {
@@ -91,24 +89,30 @@ public class SortTestUtilities {
     }
 
     public void run() throws Exception {
-      PriorityQueueCopierWrapper copier = makeCopier(fixture, sortOrder, nullOrder);
-      List<BatchGroup> batches = new ArrayList<>();
-      TupleMetadata schema = null;
-      for (SingleRowSet rowSet : rowSets) {
-        batches.add(new BatchGroup.InputBatch(rowSet.container(), rowSet.getSv2(),
-                    fixture.allocator(), rowSet.size()));
-        if (schema == null) {
-          schema = rowSet.schema();
+      Sort popConfig = SortTestUtilities.makeCopierConfig(sortOrder, nullOrder);
+      OperatorContext opContext = fixture.newOperatorContext(popConfig);
+      PriorityQueueCopierWrapper copier = new PriorityQueueCopierWrapper(opContext);
+      try {
+        List<BatchGroup> batches = new ArrayList<>();
+        TupleMetadata schema = null;
+        for (SingleRowSet rowSet : rowSets) {
+          batches.add(new BatchGroup.InputBatch(rowSet.container(), rowSet.getSv2(),
+                      fixture.allocator(), rowSet.size()));
+          if (schema == null) {
+            schema = rowSet.schema();
+          }
         }
+        int rowCount = outputRowCount();
+        VectorContainer dest = new VectorContainer();
+        BatchMerger merger = copier.startMerge(new BatchSchema(SelectionVectorMode.NONE, schema.toFieldList()),
+                                               batches, dest, rowCount, null);
+
+        verifyResults(merger, dest);
+        dest.clear();
+        merger.close();
+      } finally {
+        opContext.close();
       }
-      int rowCount = outputRowCount();
-      VectorContainer dest = new VectorContainer();
-      BatchMerger merger = copier.startMerge(new BatchSchema(SelectionVectorMode.NONE, schema.toFieldList()),
-                                             batches, dest, rowCount, null);
-
-      verifyResults(merger, dest);
-      dest.clear();
-      merger.close();
     }
 
     public int outputRowCount() {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
index 5d438ee..66481a7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
@@ -25,6 +25,8 @@ import java.util.List;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.impl.xsort.managed.PriorityQueueCopierWrapper.BatchMerger;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortTestUtilities.CopierTester;
 import org.apache.drill.exec.record.BatchSchema;
@@ -55,7 +57,9 @@ public class TestCopier extends SubOperatorTest {
   public void testEmptyInput() throws Exception {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     List<BatchGroup> batches = new ArrayList<>();
-    PriorityQueueCopierWrapper copier = SortTestUtilities.makeCopier(fixture, Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED);
+    Sort popConfig = SortTestUtilities.makeCopierConfig(Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED);
+    OperatorContext opContext = fixture.newOperatorContext(popConfig);
+    PriorityQueueCopierWrapper copier = new PriorityQueueCopierWrapper(opContext);
     VectorContainer dest = new VectorContainer();
     try {
       // TODO: Create a vector allocator to pass as last parameter so
@@ -63,11 +67,13 @@ public class TestCopier extends SubOperatorTest {
       // code. Only nuisance is that we don't have the required metadata
       // readily at hand here...
 
-      @SuppressWarnings({ "resource", "unused" })
+      @SuppressWarnings({"resource", "unused"})
       BatchMerger merger = copier.startMerge(schema, batches, dest, 10, null);
       fail();
     } catch (AssertionError e) {
       // Expected
+    } finally {
+      opContext.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
index e913c39..1315a86 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestExternalSortInternals.java
@@ -24,16 +24,16 @@ import static org.junit.Assert.assertTrue;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeAction;
 import org.apache.drill.exec.physical.impl.xsort.managed.SortMemoryManager.MergeTask;
 import org.apache.drill.test.ConfigBuilder;
-import org.apache.drill.test.DrillTest;
-import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.SubOperatorTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category(OperatorTest.class)
-public class TestExternalSortInternals extends DrillTest {
+public class TestExternalSortInternals extends SubOperatorTest {
 
   private static final int ONE_MEG = 1024 * 1024;
 
@@ -650,7 +650,7 @@ public class TestExternalSortInternals extends DrillTest {
 
   @Test
   public void testMetrics() {
-    OperatorFixture.MockStats stats = new OperatorFixture.MockStats();
+    OperatorStats stats = new OperatorStats(100, 101, 0, fixture.allocator());
     SortMetrics metrics = new SortMetrics(stats);
 
     // Input stats
@@ -667,55 +667,55 @@ public class TestExternalSortInternals extends DrillTest {
 
     // Buffer memory
 
-    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+    assertEquals(0L, stats.getLongStat(ExternalSortBatch.Metric.MIN_BUFFER));
 
     metrics.updateMemory(1_000_000);
-    assertEquals(1_000_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+    assertEquals(1_000_000L, stats.getLongStat(ExternalSortBatch.Metric.MIN_BUFFER));
 
     metrics.updateMemory(2_000_000);
-    assertEquals(1_000_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+    assertEquals(1_000_000L, stats.getLongStat(ExternalSortBatch.Metric.MIN_BUFFER));
 
     metrics.updateMemory(100_000);
-    assertEquals(100_000D, stats.getStat(ExternalSortBatch.Metric.MIN_BUFFER), 0.01);
+    assertEquals(100_000L, stats.getLongStat(ExternalSortBatch.Metric.MIN_BUFFER));
 
     // Peak batches
 
-    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+    assertEquals(0L, stats.getLongStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY));
 
     metrics.updatePeakBatches(10);
-    assertEquals(10D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+    assertEquals(10L, stats.getLongStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY));
 
     metrics.updatePeakBatches(1);
-    assertEquals(10D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+    assertEquals(10L, stats.getLongStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY));
 
     metrics.updatePeakBatches(20);
-    assertEquals(20D, stats.getStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY), 0.01);
+    assertEquals(20L, stats.getLongStat(ExternalSortBatch.Metric.PEAK_BATCHES_IN_MEMORY));
 
     // Merge count
 
-    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+    assertEquals(0L, stats.getLongStat(ExternalSortBatch.Metric.MERGE_COUNT));
 
     metrics.incrMergeCount();
-    assertEquals(1D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+    assertEquals(1L, stats.getLongStat(ExternalSortBatch.Metric.MERGE_COUNT));
 
     metrics.incrMergeCount();
-    assertEquals(2D, stats.getStat(ExternalSortBatch.Metric.MERGE_COUNT), 0.01);
+    assertEquals(2L, stats.getLongStat(ExternalSortBatch.Metric.MERGE_COUNT));
 
     // Spill count
 
-    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+    assertEquals(0L, stats.getLongStat(ExternalSortBatch.Metric.SPILL_COUNT));
 
     metrics.incrSpillCount();
-    assertEquals(1D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+    assertEquals(1L, stats.getLongStat(ExternalSortBatch.Metric.SPILL_COUNT));
 
     metrics.incrSpillCount();
-    assertEquals(2D, stats.getStat(ExternalSortBatch.Metric.SPILL_COUNT), 0.01);
+    assertEquals(2L, stats.getLongStat(ExternalSortBatch.Metric.SPILL_COUNT));
 
     // Write bytes
 
-    assertEquals(0D, stats.getStat(ExternalSortBatch.Metric.SPILL_MB), 0.01);
+    assertEquals(0L, stats.getLongStat(ExternalSortBatch.Metric.SPILL_MB));
 
     metrics.updateWriteBytes(17 * ONE_MEG + ONE_MEG * 3 / 4);
-    assertEquals(17.75D, stats.getStat(ExternalSortBatch.Metric.SPILL_MB), 0.001);
+    assertEquals(17.75D, stats.getDoubleStat(ExternalSortBatch.Metric.SPILL_MB), 0.01);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
index 93411d7..7c3c4cf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSortImpl.java
@@ -82,7 +82,7 @@ public class TestSortImpl extends DrillTest {
     FieldReference expr = FieldReference.getWithQuotedRef("key");
     Ordering ordering = new Ordering(sortOrder, expr, nullOrder);
     Sort popConfig = new Sort(null, Lists.newArrayList(ordering), false);
-    OperatorContext opContext = fixture.operatorContext(popConfig);
+    OperatorContext opContext = fixture.newOperatorContext(popConfig);
     QueryId queryId = QueryId.newBuilder()
         .setPart1(1234)
         .setPart2(5678)
@@ -157,7 +157,7 @@ public class TestSortImpl extends DrillTest {
       }
       for (RowSet expectedSet : expected) {
         assertTrue(results.next());
-        RowSet rowSet = toRowSet(fixture, results, dest);
+        RowSet rowSet = toRowSet(results, dest);
         // Uncomment these for debugging. Leave them commented otherwise
         // to avoid polluting the Maven build output unnecessarily.
 //        System.out.println("Expected:");
@@ -173,6 +173,11 @@ public class TestSortImpl extends DrillTest {
       results.close();
       dest.clear();
       sort.close();
+
+      // Note: context closed separately because this is normally done by
+      // the external sort itself after closing the output container.
+
+      sort.opContext().close();
       validateFinalStats(sort);
     }
 
@@ -191,9 +196,9 @@ public class TestSortImpl extends DrillTest {
    * @return
    */
 
-  private static RowSet toRowSet(OperatorFixture fixture, SortResults results, VectorContainer dest) {
+  private static RowSet toRowSet(SortResults results, VectorContainer dest) {
     if (results.getSv4() != null) {
-      return new HyperRowSetImpl(dest, results.getSv4());
+      return HyperRowSetImpl.fromContainer(dest, results.getSv4());
     } else if (results.getSv2() != null) {
       return IndirectRowSet.fromSv2(dest, results.getSv2());
     } else {
@@ -447,7 +452,7 @@ public class TestSortImpl extends DrillTest {
     }
     while (results.next()) {
       timer.stop();
-      RowSet output = toRowSet(fixture, results, dest);
+      RowSet output = toRowSet(results, dest);
       validator.validate(output);
       timer.start();
     }
@@ -456,6 +461,7 @@ public class TestSortImpl extends DrillTest {
     results.close();
     dest.clear();
     sort.close();
+    sort.opContext().close();
   }
 
   /**
@@ -544,6 +550,7 @@ public class TestSortImpl extends DrillTest {
     results.close();
     dest.clear();
     sort.close();
+    sort.opContext().close();
     System.out.println(timer.elapsed(TimeUnit.MILLISECONDS));
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
index c24f1a6..d4cce28 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
@@ -82,14 +82,18 @@ public class TestSorter extends DrillTest {
   }
 
   public void runSorterTest(Sort popConfig, SingleRowSet rowSet, SingleRowSet expected) throws Exception {
-    OperatorContext opContext = fixture.operatorContext(popConfig);
+    OperatorContext opContext = fixture.newOperatorContext(popConfig);
     SorterWrapper sorter = new SorterWrapper(opContext);
 
-    sorter.sortBatch(rowSet.container(), rowSet.getSv2());
+    try {
+      sorter.sortBatch(rowSet.container(), rowSet.getSv2());
 
-    new RowSetComparison(expected)
-        .verifyAndClearAll(rowSet);
-    sorter.close();
+      new RowSetComparison(expected)
+          .verifyAndClearAll(rowSet);
+      sorter.close();
+    } finally {
+      opContext.close();
+    }
   }
 
   // Test degenerate case: no rows
@@ -143,15 +147,20 @@ public class TestSorter extends DrillTest {
     protected final OperatorFixture fixture;
     protected final SorterWrapper sorter;
     protected final boolean nullable;
+    protected final OperatorContext opContext;
 
     public BaseSortTester(OperatorFixture fixture, String sortOrder, String nullOrder, boolean nullable) {
       this.fixture = fixture;
       Sort popConfig = makeSortConfig("key", sortOrder, nullOrder);
       this.nullable = nullable;
 
-      OperatorContext opContext = fixture.operatorContext(popConfig);
+      opContext = fixture.newOperatorContext(popConfig);
       sorter = new SorterWrapper(opContext);
     }
+
+    public void close() {
+      opContext.close();
+    }
   }
 
   private abstract static class SortTester extends BaseSortTester {
@@ -474,33 +483,41 @@ public class TestSorter extends DrillTest {
 
   @Test
   public void testNumericTypes() throws Exception {
-    TestSorterNumeric tester1 = new TestSorterNumeric(fixture, true);
+    TestSorterNumeric tester = new TestSorterNumeric(fixture, true);
+    try {
 //      tester1.test(MinorType.TINYINT); // DRILL-5329
 //      tester1.test(MinorType.UINT1); DRILL-5329
 //      tester1.test(MinorType.SMALLINT); DRILL-5329
 //      tester1.test(MinorType.UINT2); DRILL-5329
-    tester1.test(MinorType.INT);
+      tester.test(MinorType.INT);
 //      tester1.test(MinorType.UINT4); DRILL-5329
-    tester1.test(MinorType.BIGINT);
+      tester.test(MinorType.BIGINT);
 //      tester1.test(MinorType.UINT8); DRILL-5329
-    tester1.test(MinorType.FLOAT4);
-    tester1.test(MinorType.FLOAT8);
-    tester1.test(MinorType.DECIMAL9);
-    tester1.test(MinorType.DECIMAL18);
+      tester.test(MinorType.FLOAT4);
+      tester.test(MinorType.FLOAT8);
+      tester.test(MinorType.DECIMAL9);
+      tester.test(MinorType.DECIMAL18);
 //      tester1.test(MinorType.DECIMAL28SPARSE); DRILL-5329
 //      tester1.test(MinorType.DECIMAL38SPARSE); DRILL-5329
 //    tester1.test(MinorType.DECIMAL28DENSE); No writer
 //    tester1.test(MinorType.DECIMAL38DENSE); No writer
-    tester1.test(MinorType.DATE);
-    tester1.test(MinorType.TIME);
-    tester1.test(MinorType.TIMESTAMP);
+      tester.test(MinorType.DATE);
+      tester.test(MinorType.TIME);
+      tester.test(MinorType.TIMESTAMP);
+    } finally {
+      tester.close();
+    }
   }
 
   @Test
   public void testVarCharTypes() throws Exception {
     TestSorterStringAsc tester = new TestSorterStringAsc(fixture);
-    tester.test(MinorType.VARCHAR);
+    try {
+      tester.test(MinorType.VARCHAR);
 //      tester.test(MinorType.VAR16CHAR); DRILL-5329
+    } finally {
+      tester.close();
+    }
   }
 
   /**
@@ -512,7 +529,11 @@ public class TestSorter extends DrillTest {
   @Test
   public void testVarBinary() throws Exception {
     TestSorterBinaryAsc tester = new TestSorterBinaryAsc(fixture);
-    tester.test(MinorType.VARBINARY);
+    try {
+      tester.test(MinorType.VARBINARY);
+    } finally {
+      tester.close();
+    }
   }
 
   /**
@@ -524,7 +545,11 @@ public class TestSorter extends DrillTest {
   @Test
   public void testInterval() throws Exception {
     TestSorterIntervalAsc tester = new TestSorterIntervalAsc(fixture);
-    tester.test();
+    try {
+      tester.test();
+    } finally {
+      tester.close();
+    }
   }
 
   /**
@@ -536,7 +561,11 @@ public class TestSorter extends DrillTest {
   @Test
   public void testIntervalYear() throws Exception {
     TestSorterIntervalYearAsc tester = new TestSorterIntervalYearAsc(fixture);
-    tester.test();
+    try {
+      tester.test();
+    } finally {
+      tester.close();
+    }
   }
 
   /**
@@ -548,13 +577,21 @@ public class TestSorter extends DrillTest {
   @Test
   public void testIntervalDay() throws Exception {
     TestSorterIntervalDayAsc tester = new TestSorterIntervalDayAsc(fixture);
-    tester.test();
+    try {
+      tester.test();
+    } finally {
+      tester.close();
+    }
   }
 
   @Test
   public void testDesc() throws Exception {
     TestSorterNumeric tester = new TestSorterNumeric(fixture, false);
-    tester.test(MinorType.INT);
+    try {
+      tester.test(MinorType.INT);
+    } finally {
+      tester.close();
+    }
   }
 
   /**
@@ -566,13 +603,29 @@ public class TestSorter extends DrillTest {
   @Test
   public void testNullable() throws Exception {
     TestSorterNullableNumeric tester = new TestSorterNullableNumeric(fixture, true, true);
-    tester.test(MinorType.INT);
+    try {
+      tester.test(MinorType.INT);
+    } finally {
+      tester.close();
+    }
     tester = new TestSorterNullableNumeric(fixture, true, false);
-    tester.test(MinorType.INT);
+    try {
+      tester.test(MinorType.INT);
+    } finally {
+      tester.close();
+    }
     tester = new TestSorterNullableNumeric(fixture, false, true);
-    tester.test(MinorType.INT);
+    try {
+      tester.test(MinorType.INT);
+    } finally {
+      tester.close();
+    }
     tester = new TestSorterNullableNumeric(fixture, false, false);
-    tester.test(MinorType.INT);
+    try {
+      tester.test(MinorType.INT);
+    } finally {
+      tester.close();
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
index fda4442..088994f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
@@ -299,7 +299,7 @@ public class PhysicalOpUnitTestBase extends ExecTest {
 //        optManager.getOption(withAny(new TypeValidators.PositiveLongValidator("", 1l, 1l))); result = 10;
         drillbitContext.getCompiler(); result = new CodeCompiler(drillConf, optionManager);
         fragContext.getOptions(); result = optionManager;
-        fragContext.getOptionSet(); result = optionManager;
+        fragContext.getOptions(); result = optionManager;
         fragContext.getManagedBuffer(); result = bufManager.getManagedBuffer();
         fragContext.shouldContinue(); result = true;
         fragContext.getExecutionControls(); result = executionControls;
@@ -342,7 +342,6 @@ public class PhysicalOpUnitTestBase extends ExecTest {
     new NonStrictExpectations() {
       {
         opContext.getStats();result = opStats;
-        opContext.getStatsWriter(); result = opStats;
         opContext.getAllocator(); result = allocator;
         opContext.getFragmentContext(); result = fragContext;
         opContext.getOperatorDefn(); result = popConfig;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
index 5ce8e3f..f0cc172 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestCsv.java
@@ -29,16 +29,13 @@ import java.io.PrintWriter;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.store.easy.text.TextFormatPlugin.TextFormatConfig;
-import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.apache.drill.test.rowSet.SchemaBuilder;
-import org.apache.drill.test.DirTestWatcher;
 import org.junit.BeforeClass;
-import org.junit.ClassRule;
 import org.junit.Test;
 
 /**
@@ -52,9 +49,6 @@ public class TestCsv extends ClusterTest {
 
   private static File testDir;
 
-  @ClassRule
-  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
-
   @BeforeClass
   public static void setup() throws Exception {
     startCluster(ClusterFixture.builder(dirTestWatcher).maxParallelization(1));

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestHeaderBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestHeaderBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestHeaderBuilder.java
index 47bb903..f7648d9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestHeaderBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/easy/text/compliant/TestHeaderBuilder.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.store.easy.text.compliant;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.drill.exec.store.easy.text.compliant.HeaderBuilder.HeaderError;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.test.DrillTest;
 import org.junit.Test;
 
@@ -34,7 +34,7 @@ public class TestHeaderBuilder extends DrillTest {
     hb.startBatch();
     try {
       hb.finishRecord();
-    } catch (HeaderError e) {
+    } catch (UserException e) {
       assertTrue(e.getMessage().contains("must define at least one header"));
     }
 
@@ -43,7 +43,7 @@ public class TestHeaderBuilder extends DrillTest {
     parse(hb,"");
     try {
       hb.finishRecord();
-    } catch (HeaderError e) {
+    } catch (UserException e) {
       assertTrue(e.getMessage().contains("must define at least one header"));
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
index 802ce1b..387caa7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
@@ -65,8 +65,6 @@ import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.exec.util.VectorUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.ClassRule;
-
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.io.Resources;
@@ -108,9 +106,6 @@ public class BaseTestQuery extends ExecTest {
 
   private static ScanResult classpathScan;
 
-  @ClassRule
-  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
-
   @BeforeClass
   public static void setupDefaultTestCluster() throws Exception {
     config = DrillConfig.create(cloneDefaultTestConfigProperties());

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
index 12be961..3873740 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
@@ -26,11 +26,16 @@ import java.io.Reader;
 import java.io.StringReader;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.ExecutionException;
 
+import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.apache.drill.exec.proto.UserProtos.QueryPlanFragments;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.testing.Controls;
@@ -96,6 +101,9 @@ public class ClientFixture implements AutoCloseable {
 
     if (cluster.usesZK()) {
       client = new DrillClient(cluster.config());
+    } else if (builder.clientProps != null  &&
+        builder.clientProps.containsKey(DrillProperties.DRILLBIT_CONNECTION)) {
+      client = new DrillClient(cluster.config(), cluster.serviceSet().getCoordinator(), true);
     } else {
       client = new DrillClient(cluster.config(), cluster.serviceSet().getCoordinator());
     }
@@ -191,6 +199,25 @@ public class ClientFixture implements AutoCloseable {
     }
   }
 
+  /**
+   * Plan a query without execution.
+   * @throws ExecutionException
+   * @throws InterruptedException
+   */
+
+  public QueryPlanFragments planQuery(QueryType type, String query, boolean isSplitPlan) {
+    DrillRpcFuture<QueryPlanFragments> queryFragmentsFutures = client.planQuery(type, query, isSplitPlan);
+    try {
+      return queryFragmentsFutures.get();
+    } catch (InterruptedException | ExecutionException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  public QueryPlanFragments planQuery(String sql) {
+    return planQuery(QueryType.SQL, sql, false);
+  }
+
   @Override
   public void close() {
     if (client == null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
----------------------------------------------------------------------
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 6514ac8..8ee87c0 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
@@ -320,6 +320,23 @@ public class ClusterFixture extends BaseFixture implements AutoCloseable {
     return clients.get(0);
   }
 
+  /**
+   * Create a test client for a specific host and port.
+   *
+   * @param host host, must be one of those created by this
+   * fixture
+   * @param port post, must be one of those created by this
+   * fixture
+   * @return a test client. Client will be closed when this cluster
+   * fixture closes, or can be closed early
+   */
+
+  public ClientFixture client(String host, int port) {
+    return clientBuilder()
+      .property(DrillProperties.DRILLBIT_CONNECTION, String.format("%s:%d", host, port))
+      .build();
+  }
+
   public RestClientFixture restClientFixture() {
     if (restClientFixture == null) {
       restClientFixture = restClientBuilder().build();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
index c85c591..1ae2a87 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterTest.java
@@ -20,7 +20,9 @@ package org.apache.drill.test;
 import java.io.IOException;
 
 import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.test.rowSet.RowSet;
 import org.junit.AfterClass;
+import org.junit.ClassRule;
 
 /**
  * Base class for tests that use a single cluster fixture for a set of
@@ -72,6 +74,9 @@ import org.junit.AfterClass;
 
 public class ClusterTest extends DrillTest {
 
+  @ClassRule
+  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
+
   protected static ClusterFixture cluster;
   protected static ClientFixture client;
 
@@ -116,4 +121,34 @@ public class ClusterTest extends DrillTest {
   public QueryBuilder queryBuilder( ) {
     return client.queryBuilder();
   }
+
+  /**
+   * Handy development-time tool to run a query and print the results. Use this
+   * when first developing tests. Then, encode the expected results using
+   * the appropriate tool and verify them rather than just printing them to
+   * create the final test.
+   *
+   * @param sql the query to run
+   */
+
+  protected void runAndPrint(String sql) {
+    QueryResultSet results = client.queryBuilder().sql(sql).resultSet();
+    try {
+      for (;;) {
+        RowSet rowSet = results.next();
+        if (rowSet == null) {
+          break;
+        }
+        if (rowSet.rowCount() > 0) {
+          rowSet.print();
+        }
+        rowSet.clear();
+      }
+      System.out.println(results.recordCount());
+    } catch (Exception e) {
+      throw new IllegalStateException(e);
+    } finally {
+      results.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
index 99bbacc..cd68bf3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/DrillTestWrapper.java
@@ -767,7 +767,7 @@ public class DrillTestWrapper {
           if (!expectedRecord.containsKey(s)) {
             throw new Exception("Unexpected column '" + s + "' returned by query.");
           }
-          if (!compareValues(expectedRecord.get(s), actualRecord.get(s), counter, s)) {
+          if (! compareValues(expectedRecord.get(s), actualRecord.get(s), counter, s)) {
             i++;
             continue findMatch;
           }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/OperatorFixture.java
----------------------------------------------------------------------
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 a1b8af5..6135b1c 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
@@ -18,9 +18,7 @@
 package org.apache.drill.test;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.Callable;
 
 import org.apache.drill.common.config.DrillConfig;
@@ -28,6 +26,7 @@ import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.CodeCompiler;
+import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
@@ -36,9 +35,7 @@ import org.apache.drill.exec.ops.BaseOperatorContext;
 import org.apache.drill.exec.ops.BufferManager;
 import org.apache.drill.exec.ops.BufferManagerImpl;
 import org.apache.drill.exec.ops.FragmentContextInterface;
-import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.ops.OperatorStatReceiver;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.record.BatchSchema;
@@ -46,9 +43,9 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.TupleMetadata;
 import org.apache.drill.exec.record.TupleSchema;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.test.ClusterFixtureBuilder.RuntimeOption;
@@ -131,21 +128,23 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   public static class TestFragmentContext extends BaseFragmentContext {
 
     private final DrillConfig config;
-    private final OptionSet options;
+    private final OptionManager options;
     private final CodeCompiler compiler;
     private ExecutionControls controls;
     private final BufferManagerImpl bufferManager;
+    private final BufferAllocator allocator;
 
-    public TestFragmentContext(DrillConfig config, OptionSet options, BufferAllocator allocator) {
+    public TestFragmentContext(DrillConfig config, OptionManager options, BufferAllocator allocator) {
       super(newFunctionRegistry(config, options));
       this.config = config;
       this.options = options;
+      this.allocator = allocator;
       compiler = new CodeCompiler(config, options);
       bufferManager = new BufferManagerImpl(allocator);
     }
 
     private static FunctionImplementationRegistry newFunctionRegistry(
-        DrillConfig config, OptionSet options) {
+        DrillConfig config, OptionManager options) {
       ScanResult classpathScan = ClassPathScanner.fromPrescan(config);
       return new FunctionImplementationRegistry(config, classpathScan, options);
     }
@@ -155,7 +154,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
     }
 
     @Override
-    public OptionSet getOptionSet() {
+    public OptionManager getOptions() {
       return options;
     }
 
@@ -188,66 +187,33 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
     protected BufferManager getBufferManager() {
       return bufferManager;
     }
-  }
-
-  /**
-   * Implements a write-only version of the stats collector for use by operators,
-   * then provides simplified test-time accessors to get the stats values when
-   * validating code in tests.
-   */
-
-  public static class MockStats implements OperatorStatReceiver {
-
-    public Map<Integer, Double> stats = new HashMap<>();
-
-    @Override
-    public void addLongStat(MetricDef metric, long value) {
-      setStat(metric, getStat(metric) + value);
-    }
 
+    @SuppressWarnings("resource")
     @Override
-    public void addDoubleStat(MetricDef metric, double value) {
-      setStat(metric, getStat(metric) + value);
+    public OperatorContext newOperatorContext(PhysicalOperator popConfig,
+        OperatorStats stats) throws OutOfMemoryException {
+      BufferAllocator childAllocator = allocator.newChildAllocator(
+          "test:" + popConfig.getClass().getSimpleName(),
+          popConfig.getInitialAllocation(),
+          popConfig.getMaxAllocation()
+          );
+      return new TestOperatorContext(this, childAllocator, popConfig);
     }
 
     @Override
-    public void setLongStat(MetricDef metric, long value) {
-      setStat(metric, value);
+    public OperatorContext newOperatorContext(PhysicalOperator popConfig)
+        throws OutOfMemoryException {
+      return newOperatorContext(popConfig, null);
     }
 
     @Override
-    public void setDoubleStat(MetricDef metric, double value) {
-      setStat(metric, value);
+    public String getQueryUserName() {
+      return "fred";
     }
-
-    public double getStat(MetricDef metric) {
-      return getStat(metric.metricId());
-    }
-
-    private double getStat(int metricId) {
-      Double value = stats.get(metricId);
-      return value == null ? 0 : value;
-    }
-
-    private void setStat(MetricDef metric, double value) {
-      setStat(metric.metricId(), value);
-    }
-
-    private void setStat(int metricId, double value) {
-      stats.put(metricId, value);
-    }
-
-    // Timing stats not supported for test.
-    @Override
-    public void startWait() { }
-
-    @Override
-    public void stopWait() {  }
   }
 
   private final SystemOptionManager options;
   private final TestFragmentContext context;
-  private final OperatorStatReceiver stats;
 
   protected OperatorFixture(OperatorFixtureBuilder builder) {
     config = builder.configBuilder().build();
@@ -262,7 +228,6 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
       applySystemOptions(builder.systemOptions);
     }
     context = new TestFragmentContext(config, options, allocator);
-    stats = new MockStats();
    }
 
   private void applySystemOptions(List<RuntimeOption> systemOptions) {
@@ -272,7 +237,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   }
 
   public SystemOptionManager options() { return options; }
-  public FragmentContextInterface fragmentExecContext() { return context; }
+  public FragmentContextInterface fragmentContext() { return context; }
 
   @Override
   public void close() throws Exception {
@@ -312,7 +277,7 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
   public RowSet wrap(VectorContainer container) {
     switch (container.getSchema().getSelectionVectorMode()) {
     case FOUR_BYTE:
-      return new HyperRowSetImpl(container, container.getSelectionVector4());
+      return HyperRowSetImpl.fromContainer(container, container.getSelectionVector4());
     case NONE:
       return DirectRowSet.fromContainer(container);
     case TWO_BYTE:
@@ -324,25 +289,17 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
 
   public static class TestOperatorContext extends BaseOperatorContext {
 
-    private final OperatorStatReceiver stats;
+    private final OperatorStats stats;
 
     public TestOperatorContext(FragmentContextInterface fragContext,
         BufferAllocator allocator,
-        PhysicalOperator config,
-        OperatorStatReceiver stats) {
+        PhysicalOperator config) {
       super(fragContext, allocator, config);
-      this.stats = stats;
-    }
-
-    @Override
-    public OperatorStatReceiver getStatsWriter() {
-      return stats;
+      stats = new OperatorStats(100, 101, 0, allocator);
     }
 
     @Override
-    public OperatorStats getStats() {
-      throw new UnsupportedOperationException("getStats() not supported for tests");
-    }
+    public OperatorStats getStats() { return stats; }
 
     @Override
     public <RESULT> ListenableFuture<RESULT> runCallableAs(
@@ -351,8 +308,14 @@ public class OperatorFixture extends BaseFixture implements AutoCloseable {
     }
   }
 
-  public OperatorContext operatorContext(PhysicalOperator config) {
-    return new TestOperatorContext(context, allocator(), config, stats);
+  @SuppressWarnings("resource")
+  public OperatorContext newOperatorContext(PhysicalOperator popConfig) {
+    BufferAllocator childAllocator = allocator.newChildAllocator(
+        "test:" + popConfig.getClass().getSimpleName(),
+        popConfig.getInitialAllocation(),
+        popConfig.getMaxAllocation()
+        );
+    return new TestOperatorContext(context, childAllocator, popConfig);
   }
 
   public RowSet wrap(VectorContainer container, SelectionVector2 sv2) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 2d1aa9b..2f735d9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -36,6 +36,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.client.PrintingResultsListener;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
@@ -216,6 +217,7 @@ public class QueryBuilder {
   private final ClientFixture client;
   private QueryType queryType;
   private String queryText;
+  private List<PlanFragment> planFragments;
 
   QueryBuilder(ClientFixture client) {
     this.client = client;
@@ -236,6 +238,19 @@ public class QueryBuilder {
   }
 
   /**
+   * Run a physical plan presented as a list of fragments.
+   *
+   * @param planFragments fragments that make up the plan
+   * @return this builder
+   */
+
+  public QueryBuilder plan(List<PlanFragment> planFragments) {
+    queryType = QueryType.EXECUTION;
+    this.planFragments = planFragments;
+    return this;
+  }
+
+  /**
    * Parse a single SQL statement (with optional ending semi-colon) from
    * the file provided.
    * @param file the file containing exactly one SQL statement, with
@@ -258,6 +273,13 @@ public class QueryBuilder {
     return query(QueryType.PHYSICAL, plan);
   }
 
+  /**
+   * Run a query contained in a resource file.
+   *
+   * @param resource Name of the resource
+   * @return this builder
+   */
+
   public QueryBuilder sqlResource(String resource) {
     sql(ClusterFixture.loadResource(resource));
     return this;
@@ -300,13 +322,14 @@ public class QueryBuilder {
   }
 
   /**
-   * Run the query and return the first result set as a
+   * Run the query and return the first non-empty batch as a
    * {@link DirectRowSet} object that can be inspected directly
    * by the code using a {@link RowSetReader}.
    * <p>
-   * An enhancement is to provide a way to read a series of result
+   *
+   * @see {@link #rowSetIterator()} for a version that reads a series of
    * batches as row sets.
-   * @return a row set that represents the first batch returned from
+   * @return a row set that represents the first non-empty batch returned from
    * the query
    * @throws RpcException if anything goes wrong
    */
@@ -425,8 +448,16 @@ public class QueryBuilder {
 
   public void withListener(UserResultsListener listener) {
     Preconditions.checkNotNull(queryType, "Query not provided.");
-    Preconditions.checkNotNull(queryText, "Query not provided.");
-    client.client().runQuery(queryType, queryText, listener);
+    if (planFragments != null) {
+      try {
+        client.client().runQuery(QueryType.EXECUTION, planFragments, listener);
+      } catch(RpcException e) {
+        throw new IllegalStateException(e);
+      }
+    } else {
+      Preconditions.checkNotNull(queryText, "Query not provided.");
+      client.client().runQuery(queryType, queryText, listener);
+    }
   }
 
   /**
@@ -481,7 +512,6 @@ public class QueryBuilder {
   public long print() throws Exception {
     DrillConfig config = client.cluster().config( );
 
-
     boolean verbose = ! config.getBoolean(QueryTestUtil.TEST_QUERY_PRINTING_SILENT) ||
                       DrillTest.verbose();
     if (verbose) {
@@ -560,6 +590,11 @@ public class QueryBuilder {
     return new QuerySummary(queryId, recordCount, batchCount, elapsed, state);
   }
 
+  public QueryResultSet resultSet() {
+    BufferingQueryEventListener listener = withEventListener();
+    return new QueryResultSet(listener, client.allocator());
+  }
+
   /**
    * Submit an "EXPLAIN" statement, and return the column value which
    * contains the plan's string.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/QueryResultSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryResultSet.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryResultSet.java
new file mode 100644
index 0000000..cf13e2b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryResultSet.java
@@ -0,0 +1,110 @@
+/*
+ * 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.test;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.test.BufferingQueryEventListener.QueryEvent;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
+
+/**
+ * Returns query results as an iterator over row sets. Provides
+ * a very easy way for tests to work with query data using the
+ * row set tools.
+ */
+
+public class QueryResultSet {
+  private BufferingQueryEventListener listener;
+  private boolean eof;
+  private int recordCount = 0;
+  private int batchCount = 0;
+  private QueryId queryId = null;
+  @SuppressWarnings("unused")
+  private QueryState state = null;
+  final RecordBatchLoader loader;
+
+  public QueryResultSet(BufferingQueryEventListener listener, BufferAllocator allocator) {
+    this.listener = listener;
+    loader = new RecordBatchLoader(allocator);
+  }
+
+  /**
+   * Return the next batch of data as a row set. The first batch is usually
+   * empty as it carries only schema.
+   *
+   * @return the next batch as a row set, or null if EOF
+   * @throws Exception on a server error
+   */
+
+  public DirectRowSet next() throws Exception {
+    if (eof) {
+      return null;
+    }
+    for (;;) {
+      QueryEvent event = listener.get();
+      switch (event.type)
+      {
+      case BATCH:
+        batchCount++;
+        recordCount += event.batch.getHeader().getRowCount();
+        loader.load(event.batch.getHeader().getDef(), event.batch.getData());
+        event.batch.release();
+        return DirectRowSet.fromVectorAccessible(loader.allocator(), loader);
+
+      case EOF:
+        state = event.state;
+        eof = true;
+        return null;
+
+      case ERROR:
+        state = event.state;
+        eof = true;
+        throw event.error;
+
+      case QUERY_ID:
+        queryId = event.queryId;
+        continue;
+
+      default:
+        throw new IllegalStateException("Unexpected event: " + event.type);
+      }
+    }
+  }
+
+  public QueryId queryId() { return queryId; }
+  public int recordCount() { return recordCount; }
+  public int batchCount() { return batchCount; }
+
+  public void close() {
+    try {
+      while (! eof) {
+        RowSet rowSet = next();
+        if (rowSet != null) {
+          rowSet.clear();
+        }
+      }
+    } catch (Exception e) {
+      throw new IllegalStateException(e);
+    } finally {
+      loader.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
index 8a3db9f..d0ca662 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/HyperRowSetImpl.java
@@ -61,6 +61,10 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
     this.sv4 = sv4;
   }
 
+  public static HyperRowSet fromContainer(VectorContainer container, SelectionVector4 sv4) {
+    return new HyperRowSetImpl(container, sv4);
+  }
+
   @Override
   public boolean isExtendable() { return false; }
 
@@ -80,4 +84,10 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
 
   @Override
   public int rowCount() { return sv4.getCount(); }
+
+  @Override
+  public void clear() {
+    super.clear();
+    sv4.clear();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
index 10e9032..e84f2d3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/PerformanceTool.java
@@ -30,13 +30,10 @@ import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.RepeatedIntVector;
 import org.apache.drill.exec.vector.accessor.ColumnAccessors.IntColumnWriter;
 import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
-import org.apache.drill.exec.vector.accessor.ScalarWriter;
 import org.apache.drill.exec.vector.accessor.writer.AbstractArrayWriter.ArrayObjectWriter;
 import org.apache.drill.exec.vector.accessor.writer.NullableScalarWriter;
 import org.apache.drill.exec.vector.accessor.writer.ScalarArrayWriter;
 import org.apache.drill.test.OperatorFixture;
-import org.apache.drill.test.rowSet.RowSet.ExtendableRowSet;
-import org.apache.drill.test.rowSet.RowSetWriter;
 import org.apache.drill.test.rowSet.SchemaBuilder;
 
 import com.google.common.base.Stopwatch;
@@ -278,19 +275,4 @@ public class PerformanceTool {
       e.printStackTrace();
     }
   }
-
-  @SuppressWarnings("unused")
-  private static void testWriter2(TupleMetadata rowSchema,
-      OperatorFixture fixture, Stopwatch timer) {
-    ExtendableRowSet rs = fixture.rowSet(rowSchema);
-    RowSetWriter writer = rs.writer(4096);
-    ScalarWriter colWriter = writer.scalar(0);
-    timer.start();
-    for (int i = 0; i < ROW_COUNT; i++) {
-      colWriter.setInt(i);
-      writer.save();
-    }
-    timer.stop();
-    writer.done().clear();
-  }
 }


[04/11] drill git commit: DRILL-6079 : Fixed memory leak in Parquet Reader

Posted by am...@apache.org.
DRILL-6079 : Fixed memory leak in Parquet Reader

close apache/drill#1087


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/48623eac
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/48623eac
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/48623eac

Branch: refs/heads/master
Commit: 48623eac53ecf92754bb150094086dac46625b6a
Parents: 4c8a55b
Author: Salim Achouche <sa...@gmail.com>
Authored: Fri Dec 22 11:50:56 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Tue Jan 23 17:32:51 2018 -0800

----------------------------------------------------------------------
 .../parquet/columnreaders/AsyncPageReader.java  | 52 +++++++++++---------
 1 file changed, 30 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/48623eac/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
index 4f1ac12..036b546 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java
@@ -17,39 +17,40 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
-import com.google.common.base.Stopwatch;
-import io.netty.buffer.DrillBuf;
+import static org.apache.parquet.column.Encoding.valueOf;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.DirectDecompressor;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.parquet.hadoop.CodecFactory;
-import org.apache.parquet.hadoop.codec.SnappyCodec;
-import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.drill.exec.util.filereader.DirectBufInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DirectDecompressor;
+import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.parquet.column.page.DictionaryPage;
 import org.apache.parquet.format.PageHeader;
 import org.apache.parquet.format.PageType;
 import org.apache.parquet.format.Util;
+import org.apache.parquet.hadoop.CodecFactory;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.xerial.snappy.Snappy;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
+import com.google.common.base.Stopwatch;
 
-import static org.apache.parquet.column.Encoding.valueOf;
+import io.netty.buffer.DrillBuf;
 /**
  * The AyncPageReader reads one page of data at a time asynchronously from the provided InputStream. The
  * first request to the page reader creates a Future Task (AsyncPageReaderTask) and submits it to the
@@ -219,7 +220,7 @@ class AsyncPageReader extends PageReader {
       Stopwatch timer = Stopwatch.createStarted();
       parentColumnReader.parentReader.getOperatorContext().getStats().startWait();
       try {
-        asyncPageRead.poll().get(); // get the result of execution
+        waitForExecutionResult(); // get the result of execution
         synchronized (pageQueueSyncronize) {
           boolean pageQueueFull = pageQueue.remainingCapacity() == 0;
           readStatus = pageQueue.take(); // get the data if no exception has been thrown
@@ -253,7 +254,7 @@ class AsyncPageReader extends PageReader {
       do {
         if (pageHeader.getType() == PageType.DICTIONARY_PAGE) {
           readDictionaryPageData(readStatus, parentColumnReader);
-          asyncPageRead.poll().get(); // get the result of execution
+          waitForExecutionResult(); // get the result of execution
           synchronized (pageQueueSyncronize) {
             boolean pageQueueFull = pageQueue.remainingCapacity() == 0;
             readStatus = pageQueue.take(); // get the data if no exception has been thrown
@@ -283,6 +284,14 @@ class AsyncPageReader extends PageReader {
 
   }
 
+  private void waitForExecutionResult() throws InterruptedException, ExecutionException {
+    // Get the execution result but don't remove the Future object from the "asyncPageRead" queue yet;
+    // this will ensure that cleanup will happen properly in case of an exception being thrown
+    asyncPageRead.peek().get(); // get the result of execution
+    // Alright now remove the Future object
+    asyncPageRead.poll();
+  }
+
   @Override public void clear() {
     //Cancelling all existing AsyncPageReaderTasks
     while (asyncPageRead != null && !asyncPageRead.isEmpty()) {
@@ -395,7 +404,6 @@ class AsyncPageReader extends PageReader {
     @Override
     public Void call() throws IOException {
       ReadStatus readStatus = new ReadStatus();
-
       long bytesRead = 0;
       long valuesRead = 0;
       final long totalValuesRead = parent.totalPageValuesRead;


[10/11] drill git commit: DRILL-6049: Misc. hygiene and code cleanup changes

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
index 1b96f28..a38c3c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -32,8 +32,9 @@ import com.carrotsearch.hppc.cursors.IntDoubleCursor;
 import com.carrotsearch.hppc.cursors.IntLongCursor;
 import com.carrotsearch.hppc.procedures.IntDoubleProcedure;
 import com.carrotsearch.hppc.procedures.IntLongProcedure;
+import com.google.common.annotations.VisibleForTesting;
 
-public class OperatorStats implements OperatorStatReceiver {
+public class OperatorStats {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorStats.class);
 
   protected final int operatorId;
@@ -89,7 +90,8 @@ public class OperatorStats implements OperatorStatReceiver {
     }
   }
 
-  private OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator) {
+  @VisibleForTesting
+  public OperatorStats(int operatorId, int operatorType, int inputCount, BufferAllocator allocator) {
     super();
     this.allocator = allocator;
     this.operatorId = operatorId;
@@ -169,7 +171,6 @@ public class OperatorStats implements OperatorStatReceiver {
     inProcessing = false;
   }
 
-  @Override
   public synchronized void startWait() {
     assert !inWait : assertionError("starting waiting");
     stopProcessing();
@@ -177,7 +178,6 @@ public class OperatorStats implements OperatorStatReceiver {
     waitMark = System.nanoTime();
   }
 
-  @Override
   public synchronized void stopWait() {
     assert inWait : assertionError("stopping waiting");
     startProcessing();
@@ -203,7 +203,6 @@ public class OperatorStats implements OperatorStatReceiver {
         .toString();
   }
 
-
   public OperatorProfile getProfile() {
     final OperatorProfile.Builder b = OperatorProfile //
         .newBuilder() //
@@ -213,14 +212,11 @@ public class OperatorStats implements OperatorStatReceiver {
         .setProcessNanos(processingNanos)
         .setWaitNanos(waitNanos);
 
-    if(allocator != null){
+    if (allocator != null) {
       b.setPeakLocalMemoryAllocated(allocator.getPeakMemoryAllocation());
     }
 
-
-
     addAllMetrics(b);
-
     return b.build();
   }
 
@@ -249,7 +245,6 @@ public class OperatorStats implements OperatorStatReceiver {
     public void apply(int key, long value) {
       builder.addMetric(MetricValue.newBuilder().setMetricId(key).setLongValue(value));
     }
-
   }
 
   public void addLongMetrics(OperatorProfile.Builder builder) {
@@ -278,22 +273,62 @@ public class OperatorStats implements OperatorStatReceiver {
     }
   }
 
-  @Override
+  /**
+   * Set a stat to the specified long value. Creates the stat
+   * if the stat does not yet exist.
+   *
+   * @param metric the metric to update
+   * @param value the value to set
+   */
+
   public void addLongStat(MetricDef metric, long value){
     longMetrics.putOrAdd(metric.metricId(), value, value);
   }
 
-  @Override
+  @VisibleForTesting
+  public long getLongStat(MetricDef metric) {
+    return longMetrics.get(metric.metricId());
+  }
+
+  /**
+   * Add a double value to the existing value. Creates the stat
+   * (with an initial value of zero) if the stat does not yet
+   * exist.
+   *
+   * @param metric the metric to update
+   * @param value the value to add to the existing value
+   */
+
   public void addDoubleStat(MetricDef metric, double value){
     doubleMetrics.putOrAdd(metric.metricId(), value, value);
   }
 
-  @Override
+  @VisibleForTesting
+  public double getDoubleStat(MetricDef metric) {
+    return doubleMetrics.get(metric.metricId());
+  }
+
+  /**
+   * Add a long value to the existing value. Creates the stat
+   * (with an initial value of zero) if the stat does not yet
+   * exist.
+   *
+   * @param metric the metric to update
+   * @param value the value to add to the existing value
+   */
+
   public void setLongStat(MetricDef metric, long value){
     longMetrics.put(metric.metricId(), value);
   }
 
-  @Override
+  /**
+   * Set a stat to the specified double value. Creates the stat
+   * if the stat does not yet exist.
+   *
+   * @param metric the metric to update
+   * @param value the value to set
+   */
+
   public void setDoubleStat(MetricDef metric, double value){
     doubleMetrics.put(metric.metricId(), value);
   }
@@ -313,5 +348,4 @@ public class OperatorStats implements OperatorStatReceiver {
   public long getProcessingNanos() {
     return processingNanos;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index d17c337..d42680a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -40,7 +40,7 @@ public interface GroupScan extends Scan, HasAffinity{
    *                             2) NULL is interpreted as ALL_COLUMNS.
    *  How to handle skipAll query is up to each storage plugin, with different policy in corresponding RecordReader.
    */
-  public static final List<SchemaPath> ALL_COLUMNS = ImmutableList.of(SchemaPath.getSimplePath("*"));
+  public static final List<SchemaPath> ALL_COLUMNS = ImmutableList.of(SchemaPath.STAR_COLUMN);
 
   public static final long NO_COLUMN_STATS = -1;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 0871621..b418fd4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -75,7 +75,7 @@ public class ImplCreator {
     // to true.
 
     if (AssertionUtil.isAssertionsEnabled() ||
-        context.getOptionSet().getOption(ExecConstants.ENABLE_ITERATOR_VALIDATOR) ||
+        context.getOptions().getOption(ExecConstants.ENABLE_ITERATOR_VALIDATOR) ||
         context.getConfig().getBoolean(ExecConstants.ENABLE_ITERATOR_VALIDATION)) {
       root = IteratorValidatorInjector.rewritePlanWithIteratorValidator(context, root);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
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 77e9ea4..e0d1545 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
@@ -96,7 +96,7 @@ public class ScanBatch implements CloseableRecordBatch {
     this.readers = readerList.iterator();
     this.implicitColumns = implicitColumnList.iterator();
     if (!readers.hasNext()) {
-      throw UserException.systemError(
+      throw UserException.internalError(
           new ExecutionSetupException("A scan batch must contain at least one reader."))
         .build(logger);
     }
@@ -110,7 +110,7 @@ public class ScanBatch implements CloseableRecordBatch {
       if (!verifyImplcitColumns(readerList.size(), implicitColumnList)) {
         Exception ex = new ExecutionSetupException("Either implicit column list does not have same cardinality as reader list, "
             + "or implicit columns are not same across all the record readers!");
-        throw UserException.systemError(ex)
+        throw UserException.internalError(ex)
             .addContext("Setup failed for", readerList.get(0).getClass().getSimpleName())
             .build(logger);
       }
@@ -210,11 +210,13 @@ public class ScanBatch implements CloseableRecordBatch {
           logger.error("Close failed for reader " + currentReaderClassName, e2);
         }
       }
-      throw UserException.systemError(e)
+      throw UserException.internalError(e)
           .addContext("Setup failed for", currentReaderClassName)
           .build(logger);
+    } catch (UserException ex) {
+      throw ex;
     } catch (Exception ex) {
-      throw UserException.systemError(ex).build(logger);
+      throw UserException.internalError(ex).build(logger);
     } finally {
       oContext.getStats().stopProcessing();
     }
@@ -254,7 +256,7 @@ public class ScanBatch implements CloseableRecordBatch {
       }
     } catch(SchemaChangeException e) {
       // No exception should be thrown here.
-      throw UserException.systemError(e)
+      throw UserException.internalError(e)
         .addContext("Failure while allocating implicit vectors")
         .build(logger);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
----------------------------------------------------------------------
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 34c0f94..442a753 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
@@ -336,7 +336,7 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
   private PriorityQueue createNewPriorityQueue(VectorAccessible batch, int limit)
     throws SchemaChangeException, ClassTransformationException, IOException {
     return createNewPriorityQueue(
-      mainMapping, leftMapping, rightMapping, context.getOptionSet(), context.getFunctionRegistry(), context.getDrillbitContext().getCompiler(),
+      mainMapping, leftMapping, rightMapping, context.getOptions(), context.getFunctionRegistry(), context.getDrillbitContext().getCompiler(),
       config.getOrderings(), batch, unionTypeEnabled, codegenDump, limit, oContext.getAllocator(), schema.getSelectionVectorMode());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
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 3abf0fc..be0f61f 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
@@ -300,7 +300,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
       return false;
     }
     final NameSegment expr = ((SchemaPath)ex.getExpr()).getRootSegment();
-    return expr.getPath().contains(StarColumnHelper.STAR_COLUMN);
+    return expr.getPath().contains(SchemaPath.WILDCARD);
   }
 
   private void setupNewSchemaFromInput(RecordBatch incomingBatch) throws SchemaChangeException {
@@ -542,7 +542,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
       final NameSegment expr = ((SchemaPath) ex.getExpr()).getRootSegment();
       final NameSegment ref = ex.getRef().getRootSegment();
       final boolean refHasPrefix = ref.getPath().contains(StarColumnHelper.PREFIX_DELIMITER);
-      final boolean exprContainsStar = expr.getPath().contains(StarColumnHelper.STAR_COLUMN);
+      final boolean exprContainsStar = expr.getPath().contains(SchemaPath.WILDCARD);
 
       if (refHasPrefix || exprContainsStar) {
         needed = true;
@@ -596,10 +596,10 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
     final NameSegment ref = ex.getRef().getRootSegment();
     final boolean exprHasPrefix = expr.getPath().contains(StarColumnHelper.PREFIX_DELIMITER);
     final boolean refHasPrefix = ref.getPath().contains(StarColumnHelper.PREFIX_DELIMITER);
-    final boolean exprIsStar = expr.getPath().equals(StarColumnHelper.STAR_COLUMN);
-    final boolean refContainsStar = ref.getPath().contains(StarColumnHelper.STAR_COLUMN);
-    final boolean exprContainsStar = expr.getPath().contains(StarColumnHelper.STAR_COLUMN);
-    final boolean refEndsWithStar = ref.getPath().endsWith(StarColumnHelper.STAR_COLUMN);
+    final boolean exprIsStar = expr.getPath().equals(SchemaPath.WILDCARD);
+    final boolean refContainsStar = ref.getPath().contains(SchemaPath.WILDCARD);
+    final boolean exprContainsStar = expr.getPath().contains(SchemaPath.WILDCARD);
+    final boolean refEndsWithStar = ref.getPath().endsWith(SchemaPath.WILDCARD);
 
     String exprPrefix = EMPTY_STRING;
     String exprSuffix = expr.getPath();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
----------------------------------------------------------------------
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 ac6a462..e75619e 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
@@ -17,6 +17,11 @@
  */
 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;
 
 import org.apache.drill.common.expression.SchemaPath;
@@ -30,11 +35,8 @@ import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
-import org.apache.drill.exec.util.BatchPrinter;
 import org.apache.drill.exec.vector.VectorValidator;
 
-import static org.apache.drill.exec.record.RecordBatch.IterOutcome.*;
-
 
 public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
   private static final org.slf4j.Logger logger =

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
index 2288419..4199191 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
@@ -38,7 +38,7 @@ public class IteratorValidatorCreator implements BatchCreator<IteratorValidator>
     Preconditions.checkArgument(children.size() == 1);
     RecordBatch child = children.iterator().next();
     IteratorValidatorBatchIterator iter = new IteratorValidatorBatchIterator(child);
-    boolean validateBatches = context.getOptionSet().getOption(ExecConstants.ENABLE_VECTOR_VALIDATOR) ||
+    boolean validateBatches = context.getOptions().getOption(ExecConstants.ENABLE_VECTOR_VALIDATOR) ||
                               context.getConfig().getBoolean(ExecConstants.ENABLE_VECTOR_VALIDATION);
     iter.enableBatchValidation(validateBatches);
     logger.trace("Iterator validation enabled for " + child.getClass().getSimpleName() +

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
index 2054c9b..9150fe3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/ExternalSortBatch.java
@@ -486,7 +486,19 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
   @Override
   public void close() {
+
+    // Sanity check: if close is called twice, just ignore
+    // the second call.
+
+    if (sortImpl == null) {
+      return;
+    }
+
     RuntimeException ex = null;
+
+    // If we got far enough to have a results iterator, close
+    // that first.
+
     try {
       if (resultsIterator != null) {
         resultsIterator.close();
@@ -495,6 +507,9 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     } catch (RuntimeException e) {
       ex = (ex == null) ? e : ex;
     }
+
+    // Then close the "guts" of the sort operation.
+
     try {
       if (sortImpl != null) {
         sortImpl.close();
@@ -506,14 +521,22 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
 
     // The call to super.close() clears out the output container.
     // Doing so requires the allocator here, so it must be closed
-    // after the super call.
+    // (when closing the operator context) after the super call.
 
     try {
       super.close();
     } catch (RuntimeException e) {
       ex = (ex == null) ? e : ex;
     }
-    // Note: allocator is closed by the FragmentManager
+
+    // Finally close the operator context (which closes the
+    // child allocator.)
+
+    try {
+      oContext.close();
+    } catch (RuntimeException e) {
+      ex = ex == null ? e : ex;
+    }
     if (ex != null) {
       throw ex;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
index dee24dc..bca28f1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/MergeSortWrapper.java
@@ -142,7 +142,7 @@ public class MergeSortWrapper extends BaseSortWrapper implements SortResults {
   }
 
   private MSorter createNewMSorter(List<Ordering> orderings, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping) {
-    CodeGenerator<MSorter> cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFragmentContext().getOptionSet());
+    CodeGenerator<MSorter> cg = CodeGenerator.get(MSorter.TEMPLATE_DEFINITION, context.getFragmentContext().getOptions());
     cg.plainJavaCapable(true);
 
     // Uncomment out this line to debug the generated code.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
index 4d21b11..dda42a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/PriorityQueueCopierWrapper.java
@@ -80,7 +80,7 @@ public class PriorityQueueCopierWrapper extends BaseSortWrapper {
   private PriorityQueueCopier newCopier(VectorAccessible batch) {
     // Generate the copier code and obtain the resulting class
 
-    CodeGenerator<PriorityQueueCopier> cg = CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFragmentContext().getOptionSet());
+    CodeGenerator<PriorityQueueCopier> cg = CodeGenerator.get(PriorityQueueCopier.TEMPLATE_DEFINITION, context.getFragmentContext().getOptions());
     ClassGenerator<PriorityQueueCopier> g = cg.getRoot();
     cg.plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
index 2d53c3b..9fb478e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortImpl.java
@@ -36,6 +36,8 @@ import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Implementation of the external sort which is wrapped into the Drill
  * "next" protocol by the {@link ExternalSortBatch} class.
@@ -105,7 +107,6 @@ public class SortImpl {
     public VectorContainer getContainer() { return dest; }
   }
 
-
   /**
    * Return results for a single input batch. No merge is needed;
    * the original (sorted) input batch is simply passed as the result.
@@ -200,7 +201,7 @@ public class SortImpl {
     allocator = opContext.getAllocator();
     config = sortConfig;
     memManager = new SortMemoryManager(config, allocator.getLimit());
-    metrics = new SortMetrics(opContext.getStatsWriter());
+    metrics = new SortMetrics(opContext.getStats());
     bufferedBatches = new BufferedBatches(opContext);
 
     // Request leniency from the allocator. Leniency
@@ -215,6 +216,9 @@ public class SortImpl {
     logger.debug("Config: Is allocator lenient? {}", allowed);
   }
 
+  @VisibleForTesting
+  public OperatorContext opContext() { return context; }
+
   public void setSchema(BatchSchema schema) {
     bufferedBatches.setSchema(schema);
     spilledRuns.setSchema(schema);
@@ -541,6 +545,11 @@ public class SortImpl {
     } catch (RuntimeException e) {
       ex = ex == null ? e : ex;
     }
+
+    // Note: don't close the operator context here. It must
+    // remain open until all containers are cleared, which
+    // is done in the ExternalSortBatch class.
+
     if (ex != null) {
       throw ex;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
index 8d20cca..ae436bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SortMetrics.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.xsort.managed;
 
-import org.apache.drill.exec.ops.OperatorStatReceiver;
+import org.apache.drill.exec.ops.OperatorStats;
 
 public class SortMetrics {
 
@@ -38,12 +38,12 @@ public class SortMetrics {
    */
 
   private long minimumBufferSpace;
-  private OperatorStatReceiver stats;
+  private OperatorStats stats;
   private int spillCount;
   private int mergeCount;
   private long writeBytes;
 
-  public SortMetrics(OperatorStatReceiver stats) {
+  public SortMetrics(OperatorStats stats) {
     assert stats != null;
     this.stats = stats;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
index 1d43128..a9785ca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/managed/SorterWrapper.java
@@ -78,7 +78,7 @@ public class SorterWrapper extends BaseSortWrapper {
 
   private SingleBatchSorter newSorter(VectorAccessible batch) {
     CodeGenerator<SingleBatchSorter> cg = CodeGenerator.get(
-        SingleBatchSorter.TEMPLATE_DEFINITION, context.getFragmentContext().getOptionSet());
+        SingleBatchSorter.TEMPLATE_DEFINITION, context.getFragmentContext().getOptions());
     ClassGenerator<SingleBatchSorter> g = cg.getRoot();
     cg.plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
index 672af42..87cbf86 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
@@ -20,18 +20,16 @@ package org.apache.drill.exec.planner;
 
 import java.util.List;
 import java.util.Map;
-
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.SchemaPath;
 
 public class StarColumnHelper {
 
   public final static String PREFIX_DELIMITER = "\u00a6\u00a6";
 
-  public final static String STAR_COLUMN = "**";
-
-  public final static String PREFIXED_STAR_COLUMN = PREFIX_DELIMITER + STAR_COLUMN;
+  public final static String PREFIXED_STAR_COLUMN = PREFIX_DELIMITER + SchemaPath.WILDCARD;
 
   public static boolean containsStarColumn(RelDataType type) {
     if (! type.isStruct()) {
@@ -41,7 +39,7 @@ public class StarColumnHelper {
     List<String> fieldNames = type.getFieldNames();
 
     for (String s : fieldNames) {
-      if (s.startsWith(STAR_COLUMN)) {
+      if (s.startsWith(SchemaPath.WILDCARD)) {
         return true;
       }
     }
@@ -58,7 +56,7 @@ public class StarColumnHelper {
       if (expr instanceof RexInputRef) {
         String name = inputRowType.getFieldNames().get(((RexInputRef) expr).getIndex());
 
-        if (name.startsWith(STAR_COLUMN)) {
+        if (name.startsWith(SchemaPath.WILDCARD)) {
           return true;
         }
       }
@@ -72,7 +70,7 @@ public class StarColumnHelper {
   }
 
   public static boolean isNonPrefixedStarColumn(String fieldName) {
-    return fieldName.startsWith(STAR_COLUMN);
+    return fieldName.startsWith(SchemaPath.WILDCARD);
   }
 
   public static boolean isStarColumn(String fieldName) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
index 7b52eda..0cc016b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
@@ -33,7 +33,6 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers;
 import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
@@ -74,6 +73,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.TypeInferenceUtils;
+import org.apache.drill.exec.vector.DateUtilities;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
@@ -315,7 +315,7 @@ public class DrillConstExecutor implements RexExecutor {
                 milliseconds = intervalDayOut.milliseconds;
               }
               return rexBuilder.makeLiteral(
-                  new BigDecimal(days * (long) DateUtility.daysToStandardMillis + milliseconds),
+                  new BigDecimal(days * (long) DateUtilities.daysToStandardMillis + milliseconds),
                   TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.INTERVAL_DAY,
                       newCall.getType().isNullable()), false);
             }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
index 1230498..37e4ca1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/PreProcessLogicalRel.java
@@ -26,8 +26,8 @@ import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.exception.UnsupportedOperatorCollector;
-import org.apache.drill.exec.planner.StarColumnHelper;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
 import org.apache.drill.exec.planner.sql.parser.DrillCalciteWrapperUtility;
 import org.apache.drill.exec.util.ApproximateStringMatcher;
@@ -203,7 +203,7 @@ public class PreProcessLogicalRel extends RelShuttleImpl {
   public RelNode visit(LogicalUnion union) {
     for(RelNode child : union.getInputs()) {
       for(RelDataTypeField dataField : child.getRowType().getFieldList()) {
-        if(dataField.getName().contains(StarColumnHelper.STAR_COLUMN)) {
+        if(dataField.getName().contains(SchemaPath.WILDCARD)) {
           unsupportedOperatorCollector.setException(SqlUnsupportedException.ExceptionType.RELATIONAL,
               "Union-All over schema-less tables must specify the columns explicitly\n" +
               "See Apache Drill JIRA: DRILL-2414");

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SplitUpComplexExpressions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SplitUpComplexExpressions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SplitUpComplexExpressions.java
index 394cde3..f323991 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SplitUpComplexExpressions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SplitUpComplexExpressions.java
@@ -30,8 +30,8 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.RelConversionException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.planner.StarColumnHelper;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.physical.ProjectPrel;
@@ -107,7 +107,7 @@ public class SplitUpComplexExpressions extends BasePrelVisitor<Prel, Object, Rel
       RexBuilder builder = new RexBuilder(factory);
       allExprs.add(builder.makeInputRef( new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory), index));
 
-      if(fieldNames.get(index).contains(StarColumnHelper.STAR_COLUMN)) {
+      if(fieldNames.get(index).contains(SchemaPath.WILDCARD)) {
         relDataTypes.add(new RelDataTypeFieldImpl(fieldNames.get(index), allExprs.size(), factory.createSqlType(SqlTypeName.ANY)));
       } else {
         relDataTypes.add(new RelDataTypeFieldImpl("EXPR$" + exprIndex, allExprs.size(), factory.createSqlType(SqlTypeName.ANY)));

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
index 69458d4..c2227c4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
@@ -30,7 +30,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.planner.StarColumnHelper;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -157,7 +157,7 @@ public class SqlHandlerUtil {
             .message("Partition column %s is not in the SELECT list of CTAS!", col)
             .build(logger);
       } else {
-        if (field.getName().startsWith(StarColumnHelper.STAR_COLUMN)) {
+        if (field.getName().startsWith(SchemaPath.WILDCARD)) {
           colRefStarNames.add(col);
 
           final List<RexNode> operands = Lists.newArrayList();
@@ -191,10 +191,12 @@ public class SqlHandlerUtil {
 
       final List<RexNode> refs =
           new AbstractList<RexNode>() {
+            @Override
             public int size() {
               return originalFieldSize + colRefStarExprs.size();
             }
 
+            @Override
             public RexNode get(int index) {
               if (index < originalFieldSize) {
                 return RexInputRef.of(index, inputRowType.getFieldList());

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpandableHyperContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpandableHyperContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpandableHyperContainer.java
index 377c7af..9037340 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpandableHyperContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/ExpandableHyperContainer.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -27,6 +27,10 @@ public class ExpandableHyperContainer extends VectorContainer {
 
   public ExpandableHyperContainer(VectorAccessible batch) {
     super();
+    build(batch);
+  }
+
+  private void build(VectorAccessible batch) {
     if (batch.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.FOUR_BYTE) {
       for (VectorWrapper<?> w : batch) {
         ValueVector[] hyperVector = w.getValueVectors();
@@ -42,17 +46,7 @@ public class ExpandableHyperContainer extends VectorContainer {
 
   public void addBatch(VectorAccessible batch) {
     if (wrappers.size() == 0) {
-      if (batch.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.FOUR_BYTE) {
-        for (VectorWrapper<?> w : batch) {
-          ValueVector[] hyperVector = w.getValueVectors();
-          this.add(hyperVector, true);
-        }
-      } else {
-        for (VectorWrapper<?> w : batch) {
-          ValueVector[] hyperVector = { w.getValueVector() };
-          this.add(hyperVector, true);
-        }
-      }
+      build(batch);
       return;
     }
     if (batch.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.FOUR_BYTE) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index 3e6bf64..f180b40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -61,6 +61,8 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
     this.allocator = Preconditions.checkNotNull(allocator);
   }
 
+  public BufferAllocator allocator() { return allocator; }
+
   /**
    * Load a record batch from a single buffer.
    *
@@ -88,7 +90,7 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
 
     // Set up to recognize previous fields that no longer exist.
     final Map<String, ValueVector> oldFields = CaseInsensitiveMap.newHashMap();
-    for(final VectorWrapper<?> wrapper : container) {
+    for (final VectorWrapper<?> wrapper : container) {
       final ValueVector vector = wrapper.getValueVector();
       oldFields.put(vector.getField().getName(), vector);
     }
@@ -97,7 +99,7 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
     try {
       final List<SerializedField> fields = def.getFieldList();
       int bufOffset = 0;
-      for(final SerializedField field : fields) {
+      for (final SerializedField field : fields) {
         final MaterializedField fieldDef = MaterializedField.create(field);
         ValueVector vector = oldFields.remove(fieldDef.getName());
 
@@ -105,7 +107,7 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
           // Field did not exist previously--is schema change.
           schemaChanged = true;
           vector = TypeHelper.getNewVector(fieldDef, allocator);
-        } else if (!vector.getField().getType().equals(fieldDef.getType())) {
+        } else if (! vector.getField().getType().equals(fieldDef.getType())) {
           // Field had different type before--is schema change.
           // clear previous vector
           vector.clear();
@@ -125,7 +127,9 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
         }
 
         // Load the vector.
-        if (field.getValueCount() == 0) {
+        if (buf == null) {
+          // Schema only
+        } else if (field.getValueCount() == 0) {
           AllocationHelper.allocate(vector, 0, 0, 0);
         } else {
           vector.load(field, buf.slice(bufOffset, field.getBufferLength()));
@@ -151,9 +155,9 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
       }
       throw cause;
     } finally {
-      if (!oldFields.isEmpty()) {
+      if (! oldFields.isEmpty()) {
         schemaChanged = true;
-        for (final ValueVector vector:oldFields.values()) {
+        for (final ValueVector vector : oldFields.values()) {
           vector.clear();
         }
       }
@@ -269,5 +273,4 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
     container.clear();
     resetRecordCount();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
index e1a1031..67b2522 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaUtil.java
@@ -105,9 +105,6 @@ public class SchemaUtil {
         if (field.getType().getMinorType() == MinorType.UNION) {
           UnionVector u = (UnionVector) tp.getTo();
           for (MinorType t : field.getType().getSubTypeList()) {
-            if (u.getField().getType().getSubTypeList().contains(t)) {
-              continue;
-            }
             u.addSubType(t);
           }
         }
@@ -116,22 +113,7 @@ public class SchemaUtil {
         ValueVector newVector = TypeHelper.getNewVector(field, allocator);
         Preconditions.checkState(field.getType().getMinorType() == MinorType.UNION, "Can only convert vector to Union vector");
         UnionVector u = (UnionVector) newVector;
-        final ValueVector vv = u.addVector(tp.getTo());
-        MinorType type = v.getField().getType().getMinorType();
-        for (int i = 0; i < valueCount; i++) {
-          if (!vv.getAccessor().isNull(i)) {
-            u.getMutator().setType(i, type);
-          } else {
-            u.getMutator().setType(i, MinorType.LATE);
-          }
-        }
-        for (MinorType t : field.getType().getSubTypeList()) {
-          if (u.getField().getType().getSubTypeList().contains(t)) {
-            continue;
-          }
-          u.addSubType(t);
-        }
-        u.getMutator().setValueCount(valueCount);
+        u.setFirstType(tp.getTo(), valueCount);
         return u;
       }
     } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 9564f11..c46efaf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -18,9 +18,6 @@
 package org.apache.drill.exec.record;
 
 import java.lang.reflect.Array;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
@@ -42,13 +39,14 @@ import com.google.common.collect.Sets;
 
 public class VectorContainer implements VectorAccessible {
 
+  private final BufferAllocator allocator;
   protected final List<VectorWrapper<?>> wrappers = Lists.newArrayList();
   private BatchSchema schema;
   private int recordCount = -1;
-  private BufferAllocator allocator;
   private boolean schemaChanged = true; // Schema has changed since last built. Must rebuild schema
 
   public VectorContainer() {
+    allocator = null;
   }
 
   public VectorContainer(OperatorContext oContext) {
@@ -336,9 +334,13 @@ public class VectorContainer implements VectorAccessible {
   }
 
   public void clear() {
-    schema = null;
     zeroVectors();
+    removeAll();
+  }
+
+  public void removeAll() {
     wrappers.clear();
+    schema = null;
   }
 
   public void setRecordCount(int recordCount) {
@@ -365,13 +367,17 @@ public class VectorContainer implements VectorAccessible {
 
   /**
    * Clears the contained vectors.  (See {@link ValueVector#clear}).
+   * Note that the name <tt>zeroVector()</tt> in a value vector is
+   * used for the action to set all vectors to zero. Here it means
+   * to free the vector's memory. Sigh...
    */
+
   public void zeroVectors() {
     VectorAccessibleUtilities.clear(this);
   }
 
   public int getNumberOfColumns() {
-    return this.wrappers.size();
+    return wrappers.size();
   }
 
   public void allocateNew() {
@@ -415,4 +421,30 @@ public class VectorContainer implements VectorAccessible {
     merged.schemaChanged = false;
     return merged;
   }
+
+  /**
+   * Exchange buffers between two identical vector containers.
+   * The schemas must be identical in both column schemas and
+   * order. That is, after this call, data is exchanged between
+   * the containers. Requires that both containers be owned
+   * by the same allocator.
+   *
+   * @param other the target container with buffers to swap
+   */
+
+  public void exchange(VectorContainer other) {
+    assert schema.isEquivalent(other.schema);
+    assert wrappers.size() == other.wrappers.size();
+    assert allocator != null  &&  allocator == other.allocator;
+    for (int i = 0; i < wrappers.size(); i++) {
+      wrappers.get(i).getValueVector().exchange(
+          other.wrappers.get(i).getValueVector());
+    }
+    int temp = recordCount;
+    recordCount = other.recordCount;
+    other.recordCount = temp;
+    boolean temp2 = schemaChanged;
+    schemaChanged = other.schemaChanged;
+    other.schemaChanged = temp2;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index b3b46c2..c806669 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -177,7 +177,7 @@ public class WritableBatch implements AutoCloseable {
     return b;
   }
 
-  public static WritableBatch get(RecordBatch batch) {
+  public static WritableBatch get(VectorAccessible batch) {
     if (batch.getSchema() != null && batch.getSchema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE) {
       throw new UnsupportedOperationException("Only batches without hyper selections vectors are writable.");
     }
@@ -198,5 +198,4 @@ public class WritableBatch implements AutoCloseable {
       drillBuf.release(1);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
index 42f3473..7244148 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector2.java
@@ -86,11 +86,11 @@ public class SelectionVector2 implements AutoCloseable {
   }
 
   public void setBuffer(DrillBuf bufferHandle) {
-      /* clear the existing buffer */
-      clear();
+    /* clear the existing buffer */
+    clear();
 
-      this.buffer = bufferHandle;
-      buffer.retain(1);
+    this.buffer = bufferHandle;
+    buffer.retain(1);
   }
 
   public char getIndex(int index) {
@@ -106,7 +106,7 @@ public class SelectionVector2 implements AutoCloseable {
   }
 
   public void setIndex(int index, int value) {
-    buffer.setChar(index, value);
+    buffer.setChar(index * RECORD_SIZE, value);
   }
 
   public boolean allocateNewSafe(int size) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
index bd077fb..b51fdca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/selection/SelectionVector4.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,10 +20,10 @@ package org.apache.drill.exec.record.selection;
 import io.netty.buffer.ByteBuf;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.DeadBuf;
 
 public class SelectionVector4 implements AutoCloseable {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector4.class);
 
   private ByteBuf data;
   private int recordCount;
@@ -31,8 +31,9 @@ public class SelectionVector4 implements AutoCloseable {
   private int length;
 
   public SelectionVector4(ByteBuf vector, int recordCount, int batchRecordCount) throws SchemaChangeException {
-    if (recordCount > Integer.MAX_VALUE /4) {
-      throw new SchemaChangeException(String.format("Currently, Drill can only support allocations up to 2gb in size.  You requested an allocation of %d bytes.", recordCount * 4));
+    if (recordCount > Integer.MAX_VALUE / 4) {
+      throw new SchemaChangeException(String.format("Currently, Drill can only support allocations up to 2gb in size. " +
+          "You requested an allocation of %d bytes.", recordCount * 4L));
     }
     this.recordCount = recordCount;
     this.start = 0;
@@ -40,6 +41,17 @@ public class SelectionVector4 implements AutoCloseable {
     this.data = vector;
   }
 
+  public SelectionVector4(BufferAllocator allocator, int recordCount) {
+    if (recordCount > Integer.MAX_VALUE / 4) {
+      throw new IllegalStateException(String.format("Currently, Drill can only support allocations up to 2gb in size. " +
+          "You requested an allocation of %d bytes.", recordCount * 4L));
+    }
+    this.recordCount = recordCount;
+    this.start = 0;
+    this.length = recordCount;
+    this.data = allocator.buffer(recordCount * 4);
+  }
+
   public int getTotalCount() {
     return recordCount;
   }
@@ -54,15 +66,15 @@ public class SelectionVector4 implements AutoCloseable {
   }
 
   public void set(int index, int compound) {
-    data.setInt(index*4, compound);
+    data.setInt(index * 4, compound);
   }
 
   public void set(int index, int recordBatch, int recordIndex) {
-    data.setInt(index*4, (recordBatch << 16) | (recordIndex & 65535));
+    data.setInt(index * 4, (recordBatch << 16) | (recordIndex & 65535));
   }
 
   public int get(int index) {
-    return data.getInt( (start+index)*4);
+    return data.getInt((start+index) * 4);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/ColumnExplorer.java
----------------------------------------------------------------------
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 4b71b0f..f9d44cc 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
@@ -17,24 +17,25 @@
  */
 package org.apache.drill.exec.store;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.FragmentContextInterface;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.exec.util.Utilities;
 import org.apache.hadoop.fs.Path;
 
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
 
 public class ColumnExplorer {
 
@@ -46,13 +47,12 @@ public class ColumnExplorer {
   private final Map<String, ImplicitFileColumns> allImplicitColumns;
   private final Map<String, ImplicitFileColumns> selectedImplicitColumns;
 
-
   /**
    * Helper class that encapsulates logic for sorting out columns
    * between actual table columns, partition columns and implicit file columns.
    * Also populates map with implicit columns names as keys and their values
    */
-  public ColumnExplorer(FragmentContext context, List<SchemaPath> columns) {
+  public ColumnExplorer(FragmentContextInterface context, List<SchemaPath> columns) {
     this(context.getOptions(), columns);
   }
 
@@ -62,7 +62,7 @@ public class ColumnExplorer {
    * Also populates map with implicit columns names as keys and their values
    */
   public ColumnExplorer(OptionManager optionManager, List<SchemaPath> columns) {
-    this.partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+    this.partitionDesignator = optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
     this.columns = columns;
     this.isStarQuery = columns != null && Utilities.isStarQuery(columns);
     this.selectedPartitionColumns = Lists.newArrayList();
@@ -74,7 +74,8 @@ public class ColumnExplorer {
   }
 
   /**
-   * Creates case insensitive map with implicit file columns as keys and appropriate ImplicitFileColumns enum as values
+   * Creates case insensitive map with implicit file columns as keys and
+   * appropriate ImplicitFileColumns enum as values
    */
   public static Map<String, ImplicitFileColumns> initImplicitFileColumns(OptionManager optionManager) {
     Map<String, ImplicitFileColumns> map = CaseInsensitiveMap.newHashMap();
@@ -94,8 +95,8 @@ public class ColumnExplorer {
    * @param column column
    * @return true if given column is partition, false otherwise
    */
-  public static boolean isPartitionColumn(OptionManager optionManager, SchemaPath column){
-    String partitionDesignator = optionManager.getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
+  public static boolean isPartitionColumn(OptionManager optionManager, SchemaPath column) {
+    String partitionDesignator = optionManager.getString(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL);
     String path = column.getRootSegmentPath();
     return isPartitionColumn(partitionDesignator, path);
   }
@@ -252,11 +253,11 @@ public class ColumnExplorer {
       this.name = name;
     }
 
+    public String optionName() { return name; }
+
     /**
      * Using file path calculates value for each implicit file column
      */
     public abstract String getValue(Path path);
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/ResourceInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ResourceInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ResourceInputStream.java
index 98e460a..1aa278a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ResourceInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ResourceInputStream.java
@@ -96,5 +96,4 @@ public class ResourceInputStream extends ByteArrayInputStream implements Seekabl
       throw new EOFException();
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
index 489e03c..e97316c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFSDataInputStream.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.dfs;
 
-import org.apache.drill.exec.ops.OperatorStatReceiver;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -39,13 +38,14 @@ import java.util.EnumSet;
 public class DrillFSDataInputStream extends FSDataInputStream {
   private final FSDataInputStream underlyingIs;
   private final OpenFileTracker openFileTracker;
-  private final OperatorStatReceiver operatorStats;
+  private final OperatorStats operatorStats;
 
-  public DrillFSDataInputStream(FSDataInputStream in, OperatorStatReceiver operatorStats) throws IOException {
+  public DrillFSDataInputStream(FSDataInputStream in, OperatorStats operatorStats) throws IOException {
     this(in, operatorStats, null);
   }
 
-  public DrillFSDataInputStream(FSDataInputStream in, OperatorStatReceiver operatorStats,
+  @SuppressWarnings("resource")
+  public DrillFSDataInputStream(FSDataInputStream in, OperatorStats operatorStats,
       OpenFileTracker openFileTracker) throws IOException {
     super(new WrappedInputStream(in, operatorStats));
     underlyingIs = in;
@@ -194,9 +194,9 @@ public class DrillFSDataInputStream extends FSDataInputStream {
    */
   private static class WrappedInputStream extends InputStream implements Seekable, PositionedReadable {
     final FSDataInputStream is;
-    final OperatorStatReceiver operatorStats;
+    final OperatorStats operatorStats;
 
-    WrappedInputStream(FSDataInputStream is, OperatorStatReceiver operatorStats) {
+    WrappedInputStream(FSDataInputStream is, OperatorStats operatorStats) {
       this.is = is;
       this.operatorStats = operatorStats;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java
index fc540aa..52e1a96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/DrillFileSystem.java
@@ -26,7 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.drill.exec.ops.OperatorStatReceiver;
+import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.util.AssertionUtil;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -80,14 +80,14 @@ public class DrillFileSystem extends FileSystem implements OpenFileTracker {
   private final ConcurrentMap<DrillFSDataInputStream, DebugStackTrace> openedFiles = Maps.newConcurrentMap();
 
   private final FileSystem underlyingFs;
-  private final OperatorStatReceiver operatorStats;
+  private final OperatorStats operatorStats;
   private final CompressionCodecFactory codecFactory;
 
   public DrillFileSystem(Configuration fsConf) throws IOException {
     this(fsConf, null);
   }
 
-  public DrillFileSystem(Configuration fsConf, OperatorStatReceiver operatorStats) throws IOException {
+  public DrillFileSystem(Configuration fsConf, OperatorStats operatorStats) throws IOException {
     this.underlyingFs = FileSystem.get(fsConf);
     this.codecFactory = new CompressionCodecFactory(fsConf);
     this.operatorStats = operatorStats;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
index 80bcef2..587201e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/FileWork.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,11 +17,8 @@
  */
 package org.apache.drill.exec.store.dfs.easy;
 
-
 public interface FileWork {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileWork.class);
-
-  public String getPath();
-  public long getStart();
-  public long getLength();
+  String getPath();
+  long getStart();
+  long getLength();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/HeaderBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/HeaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/HeaderBuilder.java
index 8910c26..ef8f861 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/HeaderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/HeaderBuilder.java
@@ -24,6 +24,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.drill.common.exceptions.UserException;
+
 import com.google.common.base.Charsets;
 
 /**
@@ -67,23 +69,6 @@ public class HeaderBuilder extends TextOutput {
 
   public static final String ANONYMOUS_COLUMN_PREFIX = "column_";
 
-  /**
-   * Exception that reports header errors. Is an unchecked exception
-   * to avoid cluttering the normal field reader interface.
-   */
-  public static class HeaderError extends RuntimeException {
-
-    private static final long serialVersionUID = 1L;
-
-    public HeaderError(String msg) {
-      super(msg);
-    }
-
-    public HeaderError(int colIndex, String msg) {
-      super("Column " + (colIndex + 1) + ": " + msg);
-    }
-  }
-
   public final List<String> headers = new ArrayList<>();
   public final ByteBuffer currentField = ByteBuffer.allocate(MAX_HEADER_LEN);
 
@@ -204,14 +189,18 @@ public class HeaderBuilder extends TextOutput {
     try {
       currentField.put(data);
     } catch (BufferOverflowException e) {
-      throw new HeaderError(headers.size(), "Column exceeds maximum length of " + MAX_HEADER_LEN);
+      throw UserException.dataReadError()
+        .message("Column exceeds maximum length of %d", MAX_HEADER_LEN)
+        .build(logger);
     }
   }
 
   @Override
   public void finishRecord() {
     if (headers.isEmpty()) {
-      throw new HeaderError("The file must define at least one header.");
+      throw UserException.dataReadError()
+        .message("The file must define at least one header.")
+        .build(logger);
     }
 
     // Force headers to be unique.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextReader.java
index d218846..7a7ad0a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/compliant/TextReader.java
@@ -372,15 +372,18 @@ final class TextReader {
         throw new TextParsingException(context, "Cannot use newline character within quoted string");
       }
 
-      if(success){
+      if (success) {
         if (recordsToRead > 0 && context.currentRecord() >= recordsToRead) {
           context.stop();
         }
         return true;
-      }else{
+      } else {
         return false;
       }
 
+    } catch (UserException ex) {
+      stopParsing();
+      throw ex;
     } catch (StreamFinishedPseudoException ex) {
       stopParsing();
       return false;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
index eadbeb0..a611c6f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java
@@ -34,7 +34,6 @@ import org.apache.drill.exec.store.TimedRunnable;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.exec.store.dfs.MetadataContext;
 import org.apache.drill.exec.util.ImpersonationUtil;
-import org.apache.drill.exec.util.Utilities;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -431,7 +430,7 @@ public class Metadata {
     List<RowGroupMetadata_v3> rowGroupMetadataList = Lists.newArrayList();
 
     ArrayList<SchemaPath> ALL_COLS = new ArrayList<>();
-    ALL_COLS.add(Utilities.STAR_COLUMN);
+    ALL_COLS.add(SchemaPath.STAR_COLUMN);
     boolean autoCorrectCorruptDates = formatConfig.areCorruptDatesAutoCorrected();
     ParquetReaderUtility.DateCorruptionStatus containsCorruptDates = ParquetReaderUtility.detectCorruptDates(metadata, ALL_COLS, autoCorrectCorruptDates);
     if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetSchema.java
index 773f3d3..3935919 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetSchema.java
@@ -34,7 +34,6 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
-import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.format.SchemaElement;
@@ -226,7 +225,7 @@ public class ParquetSchema {
     for (int i = 0; i < columnsFound.length; i++) {
       SchemaPath col = projectedColumns.get(i);
       assert col != null;
-      if ( ! columnsFound[i] && ! col.equals(Utilities.STAR_COLUMN)) {
+      if ( ! columnsFound[i] && ! col.equals(SchemaPath.STAR_COLUMN)) {
         nullFilledVectors.add(createMissingColumn(col, output));
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
index 35358c2..9125e96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
@@ -31,7 +31,6 @@ import java.util.Collection;
 
 public class Utilities {
 
-  public static final SchemaPath STAR_COLUMN = SchemaPath.getSimplePath("*");
   public static final String COL_NULL_ERROR = "Columns cannot be null. Use star column to select all fields.";
 
   public static String getFileNameForQueryFragment(FragmentContext context, String location, String tag) {
@@ -87,7 +86,7 @@ public class Utilities {
     return Iterables.tryFind(Preconditions.checkNotNull(projected, COL_NULL_ERROR), new Predicate<SchemaPath>() {
       @Override
       public boolean apply(SchemaPath path) {
-        return Preconditions.checkNotNull(path).equals(STAR_COLUMN);
+        return Preconditions.checkNotNull(path).equals(SchemaPath.STAR_COLUMN);
       }
     }).isPresent();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/sql/TimePrintMillis.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/sql/TimePrintMillis.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/sql/TimePrintMillis.java
index 2611b86..d85d75b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/sql/TimePrintMillis.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/accessor/sql/TimePrintMillis.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,8 +19,8 @@ package org.apache.drill.exec.vector.accessor.sql;
 
 import java.sql.Time;
 
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+@SuppressWarnings("serial")
 public class TimePrintMillis extends Time {
   private static final String[] leadingZeroes = {"", "0", "00"};
 
@@ -33,7 +33,7 @@ public class TimePrintMillis extends Time {
 
   @Override
   public String toString () {
-    int millis = (int) (getTime() % DateUtility.secondsToMillis);
+    int millis = (int) (getTime() % org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
     StringBuilder time = new StringBuilder().append(super.toString());
 
     if (millis > 0) {

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
index bf1448e..fec9e66 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -20,7 +20,6 @@ package org.apache.drill.exec.vector.complex.fn;
 import java.io.IOException;
 
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
@@ -30,6 +29,7 @@ import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.vector.DateUtilities;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
@@ -258,9 +258,9 @@ abstract class VectorOutput {
       IntervalWriter intervalWriter = writer.interval();
       if(!isNull){
         final Period p = ISOPeriodFormat.standard().parsePeriod(parser.getValueAsString());
-        int months = DateUtility.monthsFromPeriod(p);
+        int months = DateUtilities.monthsFromPeriod(p);
         int days = p.getDays();
-        int millis = DateUtility.millisFromPeriod(p);
+        int millis = DateUtilities.periodToMillis(p);
         intervalWriter.writeInterval(months, days, millis);
       }
     }
@@ -295,6 +295,7 @@ abstract class VectorOutput {
       return innerRun();
     }
 
+    @SuppressWarnings("resource")
     @Override
     public void writeBinary(boolean isNull) throws IOException {
       VarBinaryWriter bin = writer.varBinary(fieldName);
@@ -326,6 +327,7 @@ abstract class VectorOutput {
 
     @Override
     public void writeTime(boolean isNull) throws IOException {
+      @SuppressWarnings("resource")
       TimeWriter t = writer.time(fieldName);
       if(!isNull){
         DateTimeFormatter f = ISODateTimeFormat.time();
@@ -333,6 +335,7 @@ abstract class VectorOutput {
       }
     }
 
+    @SuppressWarnings("resource")
     @Override
     public void writeTimestamp(boolean isNull) throws IOException {
       TimeStampWriter ts = writer.timeStamp(fieldName);
@@ -359,15 +362,16 @@ abstract class VectorOutput {
       IntervalWriter intervalWriter = writer.interval(fieldName);
       if(!isNull){
         final Period p = ISOPeriodFormat.standard().parsePeriod(parser.getValueAsString());
-        int months = DateUtility.monthsFromPeriod(p);
+        int months = DateUtilities.monthsFromPeriod(p);
         int days = p.getDays();
-        int millis = DateUtility.millisFromPeriod(p);
+        int millis = DateUtilities.periodToMillis(p);
         intervalWriter.writeInterval(months, days, millis);
       }
     }
 
     @Override
     public void writeInteger(boolean isNull) throws IOException {
+      @SuppressWarnings("resource")
       BigIntWriter intWriter = writer.bigInt(fieldName);
       if(!isNull){
         intWriter.writeBigInt(Long.parseLong(parser.getValueAsString()));

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
index 34c8c6c..22cd618 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
@@ -255,8 +255,11 @@ public class TestStarQueries extends BaseTestQuery {
   public void testStarView1() throws Exception {
     test("use dfs.tmp");
     test("create view vt1 as select * from cp.`tpch/region.parquet` r, cp.`tpch/nation.parquet` n where r.r_regionkey = n.n_regionkey");
-    test("select * from vt1");
-    test("drop view vt1");
+    try {
+      test("select * from vt1");
+    } finally {
+      test("drop view vt1");
+    }
   }
 
   @Test  // select star for a SchemaTable.
@@ -271,9 +274,12 @@ public class TestStarQueries extends BaseTestQuery {
         "join (select * from cp.`tpch/nation.parquet`) t2 " +
         "on t1.name = t2.n_name";
 
-    test("alter session set `planner.enable_broadcast_join` = false");
-    test(query);
-    test("alter session set `planner.enable_broadcast_join` = true");
+    try {
+      alterSession("planner.enable_broadcast_join", false);
+      test(query);
+    } finally {
+      resetSessionOption("planner.enable_broadcast_join");
+    }
     test(query);
   }
 


[11/11] drill git commit: DRILL-6049: Misc. hygiene and code cleanup changes

Posted by am...@apache.org.
DRILL-6049: Misc. hygiene and code cleanup changes

close apache/drill#1085


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

Branch: refs/heads/master
Commit: e791ed62b1c91c39676c4adef438c689fd84fd4b
Parents: d803f0c
Author: Paul Rogers <pr...@maprtech.com>
Authored: Wed Dec 20 23:59:06 2017 -0800
Committer: Aman Sinha <as...@maprtech.com>
Committed: Wed Jan 24 10:06:03 2018 -0800

----------------------------------------------------------------------
 .../drill/common/exceptions/UserException.java  |  69 +++-
 .../org/apache/drill/common/types/Types.java    |  68 +++-
 .../java/org/apache/drill/test/DrillTest.java   |   5 +-
 .../mapr/db/json/MaprDBJsonRecordReader.java    |   4 +-
 .../exec/store/kafka/KafkaRecordReader.java     |   2 +-
 .../exec/store/mongo/MongoRecordReader.java     |   3 +-
 .../main/codegen/templates/CastDateDate.java    |   1 -
 .../codegen/templates/CastIntervalInterval.java |   2 -
 .../codegen/templates/CastIntervalVarChar.java  |  35 +-
 .../main/codegen/templates/CastVarCharDate.java |   7 +-
 .../codegen/templates/CastVarCharInterval.java  |  17 +-
 .../templates/DateIntervalAggrFunctions1.java   |  24 +-
 .../DateDateArithmeticFunctions.java            |   6 +-
 .../DateIntervalArithmeticFunctions.java        |   4 +-
 .../DateTruncFunctions.java                     |  24 +-
 .../DateIntervalFunctionTemplates/Extract.java  |  28 +-
 .../IntervalIntervalArithmetic.java             |   1 -
 .../IntervalNumericArithmetic.java              |   5 +-
 .../templates/IntervalAggrFunctions2.java       |  18 +-
 .../org/apache/drill/exec/ExecConstants.java    |  11 +-
 .../exec/expr/fn/impl/DateTypeFunctions.java    | 112 ++++--
 .../drill/exec/expr/fn/impl/MappifyUtility.java |   7 +-
 .../drill/exec/ops/BaseOperatorContext.java     |   2 +-
 .../apache/drill/exec/ops/FragmentContext.java  |  10 +-
 .../exec/ops/FragmentContextInterface.java      |  16 +-
 .../apache/drill/exec/ops/OperatorContext.java  |  12 +-
 .../drill/exec/ops/OperatorContextImpl.java     |   5 -
 .../apache/drill/exec/ops/OperatorStats.java    |  64 +++-
 .../drill/exec/physical/base/GroupScan.java     |   2 +-
 .../drill/exec/physical/impl/ImplCreator.java   |   2 +-
 .../drill/exec/physical/impl/ScanBatch.java     |  12 +-
 .../exec/physical/impl/TopN/TopNBatch.java      |   2 +-
 .../impl/project/ProjectRecordBatch.java        |  12 +-
 .../IteratorValidatorBatchIterator.java         |   8 +-
 .../impl/validate/IteratorValidatorCreator.java |   2 +-
 .../impl/xsort/managed/ExternalSortBatch.java   |  27 +-
 .../impl/xsort/managed/MergeSortWrapper.java    |   2 +-
 .../managed/PriorityQueueCopierWrapper.java     |   2 +-
 .../physical/impl/xsort/managed/SortImpl.java   |  13 +-
 .../impl/xsort/managed/SortMetrics.java         |   6 +-
 .../impl/xsort/managed/SorterWrapper.java       |   2 +-
 .../drill/exec/planner/StarColumnHelper.java    |  12 +-
 .../planner/logical/DrillConstExecutor.java     |   4 +-
 .../planner/logical/PreProcessLogicalRel.java   |   4 +-
 .../visitor/SplitUpComplexExpressions.java      |   4 +-
 .../planner/sql/handlers/SqlHandlerUtil.java    |   6 +-
 .../exec/record/ExpandableHyperContainer.java   |  18 +-
 .../drill/exec/record/RecordBatchLoader.java    |  17 +-
 .../apache/drill/exec/record/SchemaUtil.java    |  20 +-
 .../drill/exec/record/VectorContainer.java      |  44 ++-
 .../apache/drill/exec/record/WritableBatch.java |   3 +-
 .../exec/record/selection/SelectionVector2.java |  10 +-
 .../exec/record/selection/SelectionVector4.java |  26 +-
 .../apache/drill/exec/store/ColumnExplorer.java |  33 +-
 .../drill/exec/store/ResourceInputStream.java   |   1 -
 .../exec/store/dfs/DrillFSDataInputStream.java  |  14 +-
 .../drill/exec/store/dfs/DrillFileSystem.java   |   6 +-
 .../drill/exec/store/dfs/easy/FileWork.java     |  11 +-
 .../easy/text/compliant/HeaderBuilder.java      |  27 +-
 .../store/easy/text/compliant/TextReader.java   |   7 +-
 .../drill/exec/store/parquet/Metadata.java      |   3 +-
 .../parquet/columnreaders/ParquetSchema.java    |   3 +-
 .../org/apache/drill/exec/util/Utilities.java   |   3 +-
 .../vector/accessor/sql/TimePrintMillis.java    |   6 +-
 .../exec/vector/complex/fn/VectorOutput.java    |  16 +-
 .../java/org/apache/drill/TestStarQueries.java  |  16 +-
 .../drill/exec/DrillSeparatePlanningTest.java   | 305 +++++-----------
 .../java/org/apache/drill/exec/ExecTest.java    |   6 +
 .../exec/compile/TestEvaluationVisitor.java     |   2 +-
 .../fn/interp/ExpressionInterpreterTest.java    |   4 +-
 .../exec/physical/impl/TopN/TopNBatchTest.java  |   2 +-
 .../physical/impl/agg/TestHashAggrSpill.java    |   3 +-
 .../physical/impl/window/TestWindowFrame.java   |   1 -
 .../physical/impl/xsort/TestExternalSort.java   |  43 ++-
 .../impl/xsort/TestSimpleExternalSort.java      |   1 +
 .../impl/xsort/managed/SortTestUtilities.java   |  46 +--
 .../physical/impl/xsort/managed/TestCopier.java |  10 +-
 .../managed/TestExternalSortInternals.java      |  40 +--
 .../impl/xsort/managed/TestSortImpl.java        |  17 +-
 .../physical/impl/xsort/managed/TestSorter.java | 105 ++++--
 .../physical/unit/PhysicalOpUnitTestBase.java   |   3 +-
 .../exec/store/easy/text/compliant/TestCsv.java |   6 -
 .../easy/text/compliant/TestHeaderBuilder.java  |   6 +-
 .../org/apache/drill/test/BaseTestQuery.java    |   5 -
 .../org/apache/drill/test/ClientFixture.java    |  27 ++
 .../org/apache/drill/test/ClusterFixture.java   |  17 +
 .../java/org/apache/drill/test/ClusterTest.java |  35 ++
 .../org/apache/drill/test/DrillTestWrapper.java |   2 +-
 .../org/apache/drill/test/OperatorFixture.java  | 111 ++----
 .../org/apache/drill/test/QueryBuilder.java     |  47 ++-
 .../org/apache/drill/test/QueryResultSet.java   | 110 ++++++
 .../drill/test/rowSet/HyperRowSetImpl.java      |  10 +
 .../drill/test/rowSet/test/PerformanceTool.java |  18 -
 .../codegen/templates/FixedValueVectors.java    | 181 ++++------
 .../src/main/codegen/templates/ListWriters.java |  56 +--
 .../codegen/templates/NullableValueVectors.java |  34 +-
 .../main/codegen/templates/UnionListWriter.java |  23 +-
 .../src/main/codegen/templates/UnionVector.java | 355 ++++++++++++++-----
 .../src/main/codegen/templates/UnionWriter.java |  12 +-
 .../templates/VariableLengthVectors.java        |  10 +-
 .../drill/exec/expr/fn/impl/DateUtility.java    | 108 +++---
 .../drill/exec/record/MaterializedField.java    | 147 +++++++-
 .../drill/exec/vector/BaseDataValueVector.java  |   3 +
 .../drill/exec/vector/BaseValueVector.java      |  10 +-
 .../apache/drill/exec/vector/DateUtilities.java | 191 ++++++++++
 .../drill/exec/vector/NullableVector.java       |   8 +-
 .../apache/drill/exec/vector/ValueVector.java   |   7 +-
 .../drill/exec/vector/VariableWidthVector.java  |  24 +-
 .../exec/vector/complex/AbstractMapVector.java  |  33 +-
 .../vector/complex/BaseRepeatedValueVector.java |   1 -
 .../drill/exec/vector/complex/MapVector.java    |  42 ++-
 .../exec/vector/complex/RepeatedMapVector.java  |  21 +-
 .../vector/complex/RepeatedValueVector.java     |   3 +-
 .../drill/exec/vector/complex/StateTool.java    |   8 +-
 .../vector/complex/impl/PromotableWriter.java   |   2 +
 .../expression/LogicalExpressionBase.java       |  11 +-
 .../drill/common/expression/PathSegment.java    |  27 +-
 .../drill/common/expression/SchemaPath.java     |  99 +++++-
 .../common/logical/FormatPluginConfigBase.java  |   6 +-
 pom.xml                                         |   2 +-
 .../apache/drill/exec/proto/UserBitShared.java  | 295 +++++++++------
 .../drill/exec/proto/beans/DrillPBError.java    |   8 +-
 protocol/src/main/protobuf/GeneralRPC.proto     |   6 +-
 protocol/src/main/protobuf/UserBitShared.proto  |  16 +-
 124 files changed, 2322 insertions(+), 1377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
index 4ea97e5..19a1f91 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
@@ -83,14 +83,6 @@ public class UserException extends DrillRuntimeException {
    * <p>The cause message will be used unless {@link Builder#message(String, Object...)} is called.
    * <p>If the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build(Logger)}
    * instead of creating a new exception. Any added context will be added to the user exception as well.
-   * <p>
-   * This exception, previously deprecated, has been repurposed to indicate unspecified
-   * errors. In particular, the case in which a lower level bit of code throws an
-   * exception other than UserException. The catching code then only knows "something went
-   * wrong", but not enough information to categorize the error.
-   * <p>
-   * System errors also indicate illegal internal states, missing functionality, and other
-   * code-related errors -- all of which "should never occur."
    *
    * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#SYSTEM
    *
@@ -98,6 +90,8 @@ public class UserException extends DrillRuntimeException {
    *              returned by the builder instead of creating a new user exception
    * @return user exception builder
    *
+   * @deprecated This method should never need to be used explicitly, unless you are passing the exception to the
+   *             Rpc layer or UserResultListener.submitFailed()
    */
 
   public static Builder systemError(final Throwable cause) {
@@ -365,6 +359,47 @@ public class UserException extends DrillRuntimeException {
   }
 
   /**
+   * Wraps an error that arises from execution due to issues in the query, in
+   * the environment and so on -- anything other than "this should never occur"
+   * type checks.
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+
+  public static Builder executionError(final Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.EXECUTION_ERROR, cause);
+  }
+
+  /**
+   * Indicates an internal validation failed or similar unexpected error. Indicates
+   * the problem is likely within Drill itself rather than due to the environment,
+   * query, etc.
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+
+  public static Builder internalError(final Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.INTERNAL_ERROR, cause);
+  }
+
+  /**
+   * Indicates an unspecified error: code caught the exception, but does not have
+   * visibility into the cause well enough to pick one of the more specific
+   * error types. In practice, using this exception indicates that error handling
+   * should be moved closer to the source of the exception so we can provide the
+   * user with a better explanation than "something went wrong."
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder unspecifiedError(final Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.UNSPECIFIED_ERROR, cause);
+  }
+
+
+  /**
    * Builder class for DrillUserException. You can wrap an existing exception, in this case it will first check if
    * this exception is, or wraps, a DrillUserException. If it does then the builder will use the user exception as it is
    * (it will ignore the message passed to the constructor) and will add any additional context information to the
@@ -402,6 +437,14 @@ public class UserException extends DrillRuntimeException {
       }
     }
 
+    private Builder(UserException uex) {
+      this.uex = uex;
+      cause = uex.getCause();
+      errorType = uex.errorType;
+      context = uex.context;
+      message = uex.getOriginalMessage();
+    }
+
     /**
      * sets or replaces the error message.
      * <p>This will be ignored if this builder is wrapping a user exception
@@ -415,7 +458,11 @@ public class UserException extends DrillRuntimeException {
     public Builder message(final String format, final Object... args) {
       // we can't replace the message of a user exception
       if (uex == null && format != null) {
-        this.message = String.format(format, args);
+        if (args.length == 0) {
+          message = format;
+        } else {
+          message = String.format(format, args);
+        }
       }
       return this;
     }
@@ -636,6 +683,10 @@ public class UserException extends DrillRuntimeException {
     this.context = builder.context;
   }
 
+  public Builder rebuild() {
+    return new Builder(this);
+  }
+
   /**
    * generates the message that will be displayed to the client without the stack trace.
    *

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index 7c7026b..8f5d1f9 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -20,6 +20,9 @@ package org.apache.drill.common.types;
 import static org.apache.drill.common.types.TypeProtos.DataMode.REPEATED;
 
 import java.sql.ResultSetMetaData;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -54,9 +57,9 @@ public class Types {
     case LIST:
     case MAP:
       return true;
+    default:
+      return false;
     }
-
-    return false;
   }
 
   public static boolean isRepeated(final MajorType type) {
@@ -460,9 +463,9 @@ public class Types {
 
   public static boolean softEquals(final MajorType a, final MajorType b, final boolean allowNullSwap) {
     if (a.getMinorType() != b.getMinorType()) {
-        return false;
+      return false;
     }
-    if(allowNullSwap) {
+    if (allowNullSwap) {
       switch (a.getMode()) {
       case OPTIONAL:
       case REQUIRED:
@@ -470,7 +473,9 @@ public class Types {
         case OPTIONAL:
         case REQUIRED:
           return true;
+        default:
         }
+      default:
       }
     }
     return a.getMode() == b.getMode();
@@ -728,4 +733,59 @@ public class Types {
     return type.getMinorType() == MinorType.LATE;
   }
 
+  public static boolean isEquivalent(MajorType type1, MajorType type2) {
+
+    // Requires full type equality, including fields such as precision and scale.
+    // But, unset fields are equivalent to 0. Can't use the protobuf-provided
+    // isEquals() which treats set and unset fields as different.
+
+    if (type1.getMinorType() != type2.getMinorType() ||
+        type1.getMode() != type2.getMode() ||
+        type1.getScale() != type2.getScale() ||
+        type1.getPrecision() != type2.getPrecision()) {
+      return false;
+    }
+
+    // Subtypes are only for unions and are seldom used.
+
+    if (type1.getMinorType() != MinorType.UNION) {
+      return true;
+    }
+
+    List<MinorType> subtypes1 = type1.getSubTypeList();
+    List<MinorType> subtypes2 = type2.getSubTypeList();
+    if (subtypes1 == subtypes2) { // Only occurs if both are null
+      return true;
+    }
+    if (subtypes1 == null || subtypes2 == null) {
+      return false;
+    }
+    if (subtypes1.size() != subtypes2.size()) {
+      return false;
+    }
+
+    // Now it gets slow because subtype lists are not ordered.
+
+    List<MinorType> copy1 = new ArrayList<>();
+    List<MinorType> copy2 = new ArrayList<>();
+    copy1.addAll(subtypes1);
+    copy2.addAll(subtypes2);
+    Collections.sort(copy1);
+    Collections.sort(copy2);
+    return copy1.equals(copy2);
+  }
+
+  /**
+   * The union vector is a map of types. The following method provides
+   * the standard name to use in the type map. It replaces the many
+   * ad-hoc appearances of this code in each reference to the map.
+   *
+   * @param type Drill data type
+   * @return string key to use for this type in a union vector type
+   * map
+   */
+
+  public static String typeKey(MinorType type) {
+    return type.name().toLowerCase();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/common/src/test/java/org/apache/drill/test/DrillTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index d949d97..24ec381 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -29,6 +29,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
+import org.junit.rules.DisableOnDebug;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
@@ -40,6 +41,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class DrillTest {
+
   protected static final ObjectMapper objectMapper;
   static {
     System.setProperty("line.separator", "\n");
@@ -54,8 +56,7 @@ public class DrillTest {
   static MemWatcher memWatcher;
   static String className;
 
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(100_000);
-
+  @Rule public final TestRule TIMEOUT = new DisableOnDebug(TestTools.getTimeoutRule(100_000));
   @Rule public final TestLogReporter logOutcome = LOG_OUTCOME;
 
   @Rule public final TestRule REPEAT_RULE = TestTools.getRepeatRule(false);

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 113b3ad..1327541 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -129,13 +129,13 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
   protected Collection<SchemaPath> transformColumns(Collection<SchemaPath> columns) {
     Set<SchemaPath> transformed = Sets.newLinkedHashSet();
     if (disablePushdown) {
-      transformed.add(Utilities.STAR_COLUMN);
+      transformed.add(SchemaPath.STAR_COLUMN);
       includeId = true;
       return transformed;
     }
 
     if (isStarQuery()) {
-      transformed.add(Utilities.STAR_COLUMN);
+      transformed.add(SchemaPath.STAR_COLUMN);
       includeId = true;
       if (isSkipQuery()) {
     	// `SELECT COUNT(*)` query

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
index f034a8a..c08c86e 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaRecordReader.java
@@ -85,7 +85,7 @@ public class KafkaRecordReader extends AbstractRecordReader {
         transformed.add(column);
       }
     } else {
-      transformed.add(Utilities.STAR_COLUMN);
+      transformed.add(SchemaPath.STAR_COLUMN);
     }
     return transformed;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
index cacb318..da516dd 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
@@ -35,7 +35,6 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.bson.BsonRecordReader;
-import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.complex.fn.JsonReader;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
@@ -113,7 +112,7 @@ public class MongoRecordReader extends AbstractRecordReader {
     } else {
       // Tale all the fields including the _id
       this.fields.remove(DrillMongoConstants.ID);
-      transformed.add(Utilities.STAR_COLUMN);
+      transformed.add(SchemaPath.STAR_COLUMN);
     }
     return transformed;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/CastDateDate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastDateDate.java b/exec/java-exec/src/main/codegen/templates/CastDateDate.java
index 21e9c21..f4ba51d 100644
--- a/exec/java-exec/src/main/codegen/templates/CastDateDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastDateDate.java
@@ -39,7 +39,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java b/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
index ab3e378..f1659ad 100644
--- a/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
@@ -41,7 +41,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -83,7 +82,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
index 43f9303..eb36263 100644
--- a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
+++ b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
@@ -44,7 +44,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -67,19 +66,19 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
 
   public void eval() {
 
-      int years  = (in.months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      int months = (in.months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      int years  = (in.months / org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
+      int months = (in.months % org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
 
       long millis = in.milliseconds;
 
-      long hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      long hours  = millis / (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
+      millis     = millis % (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
 
-      long minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      long minutes = millis / (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
+      millis      = millis % (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
 
-      long seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      long seconds = millis / (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
+      millis      = millis % (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
 
       String yearString = (Math.abs(years) == 1) ? " year " : " years ";
       String monthString = (Math.abs(months) == 1) ? " month " : " months ";
@@ -124,7 +123,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}",
@@ -143,8 +141,8 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
   }
 
   public void eval() {
-      int years  = (in.value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      int months = (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      int years  = (in.value / org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
+      int months = (in.value % org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
 
       String yearString = (Math.abs(years) == 1) ? " year " : " years ";
       String monthString = (Math.abs(months) == 1) ? " month " : " months ";
@@ -184,7 +182,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
@@ -208,14 +205,14 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
   public void eval() {
       long millis = in.milliseconds;
 
-      long hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      long hours  = millis / (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
+      millis     = millis % (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
 
-      long minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      long minutes = millis / (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
+      millis      = millis % (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
 
-      long seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      long seconds = millis / (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
+      millis      = millis % (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
 
       String dayString = (Math.abs(in.days) == 1) ? " day " : " days ";
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
index 4c51ba8..d035a99 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -39,7 +39,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
@@ -54,8 +53,7 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
   @Param ${type.from}Holder in;
   @Output ${type.to}Holder out;
   
-  public void setup() {
-  }
+  public void setup() { }
 
   public void eval() {
 
@@ -76,7 +74,6 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
       org.joda.time.format.DateTimeFormatter f = org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeFormatter();
       out.value = (int) ((f.parseDateTime(input)).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
       </#if>
-
   }
 }
 </#if> <#-- type.major -->

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java b/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
index d8b2024..8f68ff6 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
@@ -39,7 +39,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
@@ -66,24 +65,24 @@ public class Cast${type.from}To${type.to} implements DrillSimpleFunc {
       org.joda.time.Period period = org.joda.time.Period.parse(input);
 
       <#if type.to == "Interval">
-      out.months       = (period.getYears() * org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths) + period.getMonths();
+      out.months       = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
 
       out.days         = period.getDays();
 
-      out.milliseconds = (period.getHours() * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                         (period.getMinutes() * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                         (period.getSeconds() * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
+      out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                         (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                         (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
                          (period.getMillis());
 
       <#elseif type.to == "IntervalDay">
       out.days         = period.getDays();
 
-      out.milliseconds = (period.getHours() * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                         (period.getMinutes() * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                         (period.getSeconds() * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
+      out.milliseconds = (period.getHours() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                         (period.getMinutes() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                         (period.getSeconds() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
                          (period.getMillis());
       <#elseif type.to == "IntervalYear">
-      out.value = (period.getYears() * org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths) + period.getMonths();
+      out.value = (period.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) + period.getMonths();
       </#if>
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
index b2a0525..18be0b7 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
@@ -86,14 +86,14 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
     <#if type.outputType?ends_with("Interval")>
 
-    long inMS = (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis+
-                       in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    long inMS = (long) in.months * org.apache.drill.exec.vector.DateUtilities.monthsToMillis+
+                       in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                        in.milliseconds;
 
     value.value = Math.min(value.value, inMS);
 
     <#elseif type.outputType?ends_with("IntervalDay")>
-    long inMS = (long) in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    long inMS = (long) in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                        in.milliseconds;
 
     value.value = Math.min(value.value, inMS);
@@ -104,13 +104,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
     </#if>
 	  <#elseif aggrtype.funcName == "max">
     <#if type.outputType?ends_with("Interval")>
-    long inMS = (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis+
-                       in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    long inMS = (long) in.months * org.apache.drill.exec.vector.DateUtilities.monthsToMillis+
+                       in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                        in.milliseconds;
 
     value.value = Math.max(value.value, inMS);
     <#elseif type.outputType?ends_with("IntervalDay")>
-    long inMS = (long) in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    long inMS = (long) in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                        in.milliseconds;
 
     value.value = Math.max(value.value, inMS);
@@ -145,13 +145,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
       out.isSet = 1;
       <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
       <#if type.outputType?ends_with("Interval")>
-      out.months = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
-      value.value = value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
-      out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-      out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.months = (int) (value.value / org.apache.drill.exec.vector.DateUtilities.monthsToMillis);
+      value.value = value.value % org.apache.drill.exec.vector.DateUtilities.monthsToMillis;
+      out.days = (int) (value.value / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      out.milliseconds = (int) (value.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
       <#elseif type.outputType?ends_with("IntervalDay")>
-      out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-      out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.days = (int) (value.value / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      out.milliseconds = (int) (value.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
       <#else>
       out.value = value.value;
       </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
index 04eb327..03db5e6 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
@@ -61,11 +61,11 @@ public static class G${type}Difference implements DrillSimpleFunc {
         <#if type == "Time">
         out.milliseconds = left.value - right.value;
         <#elseif type == "Date">
-        out.days = (int) ((left.value - right.value) / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+        out.days = (int) ((left.value - right.value) / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         <#elseif type == "TimeStamp">
         long difference = (left.value - right.value);
-        out.milliseconds = (int) (difference % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-        out.days = (int) (difference / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+        out.milliseconds = (int) (difference % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+        out.days = (int) (difference / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         </#if>
     }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
index 57e7f68..5c9f5de 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
@@ -41,7 +41,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -160,7 +159,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -178,7 +176,7 @@ public class ${datetype}${intervaltype}Functions {
     <#else>
     ${output} = ${left}.value ${op} ${right}.milliseconds;
     // Wrap around 24 hour clock if we exceeded it while adding the time component
-    ${output} = ${output} % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
+    ${output} = ${output} % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
     </#if>
 </#macro>
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
index 702f717..480d501 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
@@ -213,19 +213,19 @@ public class GDateTimeTruncateFunctions {
         <#if toUnit == "Second"> <#--  Start UnitType -->
       out.months = right.months;
       out.days = right.days;
-      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis))*
-          (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.secondsToMillis))*
+          (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
         <#elseif toUnit == "Minute">
       out.months = right.months;
       out.days = right.days;
-      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis))*
-          (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.minutesToMillis))*
+          (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
         <#elseif toUnit == "Hour">
       out.months = right.months;
       out.days = right.days;
       out.milliseconds =
-          (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis))*
-              (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+          (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.hoursToMillis))*
+              (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
         <#elseif toUnit == "Day">
       out.months = right.months;
       out.days = right.days;
@@ -258,17 +258,17 @@ public class GDateTimeTruncateFunctions {
       <#elseif type == "IntervalDay">
         <#if toUnit == "Second"> <#--  Start UnitType -->
       out.days = right.days;
-      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis))*
-        (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.secondsToMillis))*
+        (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
         <#elseif toUnit == "Minute">
       out.days = right.days;
-      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis))*
-          (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      out.milliseconds = (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.minutesToMillis))*
+          (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
         <#elseif toUnit == "Hour">
       out.days = right.days;
       out.milliseconds =
-          (right.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis))*
-              (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+          (right.milliseconds/(org.apache.drill.exec.vector.DateUtilities.hoursToMillis))*
+              (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
         <#elseif toUnit == "Day">
       out.days = right.days;
       out.milliseconds = 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
index a64d655..2442672 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
@@ -92,19 +92,19 @@ public class ${className} {
     public void eval() {
   <#if fromUnit == "Interval">
     <#if toUnit == "Year">
-      out.value = (in.months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      out.value = (in.months / org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
     <#elseif toUnit == "Month">
-      out.value = (in.months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      out.value = (in.months % org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
     <#elseif toUnit == "Day">
       out.value = in.days;
     <#elseif toUnit == "Hour">
-      out.value = in.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      out.value = in.milliseconds/(org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
     <#elseif toUnit == "Minute">
-      int millis = in.milliseconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      int millis = in.milliseconds % (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
+      out.value = millis / (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
     <#elseif toUnit == "Second">
-      long millis = in.milliseconds % org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis;
-      out.value = (double) millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      long millis = in.milliseconds % org.apache.drill.exec.vector.DateUtilities.minutesToMillis;
+      out.value = (double) millis / (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
     </#if>
   <#elseif fromUnit == "IntervalDay">
     <#if toUnit == "Year" || toUnit == "Month">
@@ -112,19 +112,19 @@ public class ${className} {
     <#elseif toUnit == "Day">
       out.value = in.days;
     <#elseif toUnit == "Hour">
-      out.value = in.milliseconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      out.value = in.milliseconds/(org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
     <#elseif toUnit == "Minute">
-      int millis = in.milliseconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      int millis = in.milliseconds % (org.apache.drill.exec.vector.DateUtilities.hoursToMillis);
+      out.value = millis / (org.apache.drill.exec.vector.DateUtilities.minutesToMillis);
     <#elseif toUnit == "Second">
-      long millis = in.milliseconds % org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis;
-      out.value = (double) millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      long millis = in.milliseconds % org.apache.drill.exec.vector.DateUtilities.minutesToMillis;
+      out.value = (double) millis / (org.apache.drill.exec.vector.DateUtilities.secondsToMillis);
     </#if>
   <#else> <#-- IntervalYear type -->
     <#if toUnit == "Year">
-      out.value = (in.value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      out.value = (in.value / org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
     <#elseif toUnit == "Month">
-      out.value = (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      out.value = (in.value % org.apache.drill.exec.vector.DateUtilities.yearsToMonths);
     <#else>
       out.value = 0;
     </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
index b248c35..41af7eb 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
@@ -39,7 +39,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
index 8a8e966..6e06c0c 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
@@ -39,7 +39,6 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.joda.time.MutableDateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /*
  * This class is generated using freemarker and the ${.template_name} template.
@@ -82,12 +81,12 @@ public class ${intervaltype}${numerictype}Functions {
 
     // Transfer fractional part to days
     fractionalMonths = fractionalMonths - (long) fractionalMonths;
-    fractionalDays += fractionalMonths * org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays;
+    fractionalDays += fractionalMonths * org.apache.drill.exec.vector.DateUtilities.monthToStandardDays;
     ${out}.days = (int) fractionalDays;
 
     // Transfer fractional part to millis
     fractionalDays = fractionalDays - (long) fractionalDays;
-    fractionalMillis += fractionalDays * org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
+    fractionalMillis += fractionalDays * org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
 
     ${out}.milliseconds = (int) fractionalMillis;
 </#macro>

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
index 8e7fed5..7973629 100644
--- a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
@@ -82,11 +82,11 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
     nonNullCount.value = 1;
 	  <#if aggrtype.funcName == "avg">
     <#if type.inputType.endsWith("Interval")>
-    sum.value += (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays +
-                          in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    sum.value += (long) in.months * org.apache.drill.exec.vector.DateUtilities.monthToStandardDays +
+                          in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                           in.milliseconds;
     <#elseif type.inputType.endsWith("IntervalDay")>
-    sum.value += (long) in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
+    sum.value += (long) in.days * (org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis) +
                         in.milliseconds;
     <#else>
     sum.value += in.value;
@@ -107,14 +107,14 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
       out.isSet = 1;
       double millis = sum.value / ((double) count.value);
       <#if type.inputType.endsWith("Interval") || type.inputType.endsWith("IntervalYear")>
-      out.months = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
-      millis = millis % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
-      out.days =(int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-      out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.months = (int) (millis / org.apache.drill.exec.vector.DateUtilities.monthsToMillis);
+      millis = millis % org.apache.drill.exec.vector.DateUtilities.monthsToMillis;
+      out.days =(int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      out.milliseconds = (int) (millis % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
       <#elseif type.inputType.endsWith("IntervalDay")>
       out.months = 0;
-      out.days = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-      out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.days = (int) (millis / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+      out.milliseconds = (int) (millis % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
       </#if>
     } else {
       out.isSet = 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
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 c3e9d46..25f6135 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
@@ -252,7 +252,7 @@ public final class ExecConstants {
   public static final String PARQUET_COLUMNREADER_ASYNC = "store.parquet.reader.columnreader.async";
   public static final OptionValidator PARQUET_COLUMNREADER_ASYNC_VALIDATOR = new BooleanValidator(PARQUET_COLUMNREADER_ASYNC);
 
-  // Use a buffering reader for parquet page reader
+  // Use a buffering reader for Parquet page reader
   public static final String PARQUET_PAGEREADER_USE_BUFFERED_READ = "store.parquet.reader.pagereader.bufferedread";
   public static final OptionValidator PARQUET_PAGEREADER_USE_BUFFERED_READ_VALIDATOR = new  BooleanValidator(PARQUET_PAGEREADER_USE_BUFFERED_READ);
 
@@ -289,13 +289,13 @@ public final class ExecConstants {
   public static final BooleanValidator JSON_READER_NAN_INF_NUMBERS_VALIDATOR = new BooleanValidator(JSON_READER_NAN_INF_NUMBERS);
   /**
    * The column label (for directory levels) in results when querying files in a directory
-   * E.g.  labels: dir0   dir1
+   * E.g.  labels: dir0   dir1<pre>
    *    structure: foo
    *                |-    bar  -  a.parquet
-   *                |-    baz  -  b.parquet
+   *                |-    baz  -  b.parquet</pre>
    */
   public static final String FILESYSTEM_PARTITION_COLUMN_LABEL = "drill.exec.storage.file.partition.column.label";
-  public static final OptionValidator FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR = new StringValidator(FILESYSTEM_PARTITION_COLUMN_LABEL);
+  public static final StringValidator FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR = new StringValidator(FILESYSTEM_PARTITION_COLUMN_LABEL);
 
   /**
    * Implicit file columns
@@ -319,7 +319,8 @@ public final class ExecConstants {
   public static final String MONGO_BSON_RECORD_READER = "store.mongo.bson.record.reader";
   public static final OptionValidator MONGO_BSON_RECORD_READER_VALIDATOR = new BooleanValidator(MONGO_BSON_RECORD_READER);
 
-  public static final BooleanValidator ENABLE_UNION_TYPE = new BooleanValidator("exec.enable_union_type");
+  public static final String ENABLE_UNION_TYPE_KEY = "exec.enable_union_type";
+  public static final BooleanValidator ENABLE_UNION_TYPE = new BooleanValidator(ENABLE_UNION_TYPE_KEY);
 
   // Kafka plugin related options.
   public static final String KAFKA_ALL_TEXT_MODE = "store.kafka.all_text_mode";

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
index e136d15..a719ec0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
@@ -65,8 +65,10 @@ public class DateTypeFunctions {
       @Param NullableVarCharHolder in;
       @Output BitHolder out;
 
+      @Override
       public void setup() { }
 
+      @Override
       public void eval() {
         // for a null input return false
         if (in.isSet == 0) {
@@ -86,8 +88,10 @@ public class DateTypeFunctions {
       @Param VarCharHolder in;
       @Output BitHolder out;
 
+      @Override
       public void setup() { }
 
+      @Override
       public void eval() {
         // for a null input return false
         out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.isReadableAsDate(in.buffer, in.start, in.end) ? 1 : 0;
@@ -106,17 +110,19 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputMilliSeconds;
         @Output IntervalHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
 
-            out.months       =  (int) ((inputYears.value * org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths) +
+            out.months       =  (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) +
                                        (inputMonths.value));
             out.days         =  (int) inputDays.value;
-            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                                       (inputMinutes.value * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                                       (inputSeconds.value * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
+            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                                       (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                                       (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
                                        (inputMilliSeconds.value));
         }
     }
@@ -128,12 +134,14 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputMonths;
         @Output IntervalYearHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
 
-            out.value       = (int) ((inputYears.value * org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths) +
+            out.value       = (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) +
                                       (inputMonths.value));
         }
     }
@@ -148,15 +156,17 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputMillis;
         @Output IntervalDayHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
 
             out.days  = (int) inputDays.value;
-            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                                       (inputMinutes.value * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                                       (inputSeconds.value * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
+            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                                       (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                                       (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
                                  (inputMillis.value));
         }
     }
@@ -169,9 +179,11 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputDays;
         @Output DateHolder   out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             out.value = ((new org.joda.time.MutableDateTime((int) inputYears.value,
                                                             (int) inputMonths.value,
@@ -196,9 +208,11 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputMilliSeconds;
         @Output TimeStampHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             out.value = ((new org.joda.time.MutableDateTime((int)inputYears.value,
                                                             (int)inputMonths.value,
@@ -220,13 +234,15 @@ public class DateTypeFunctions {
         @Param  BigIntHolder inputMilliSeconds;
         @Output TimeHolder   out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
-            out.value = (int) ((inputHours.value * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                               (inputMinutes.value * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                               (inputSeconds.value * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
+            out.value = (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                               (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                               (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
                                 inputMilliSeconds.value);
         }
     }
@@ -237,6 +253,7 @@ public class DateTypeFunctions {
         @Output DateHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
 
             int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
@@ -246,6 +263,7 @@ public class DateTypeFunctions {
                 withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
         }
 
+        @Override
         public void eval() {
             out.value = queryStartDate;
         }
@@ -257,9 +275,11 @@ public class DateTypeFunctions {
         @Inject DrillBuf buffer;
         @Output VarCharHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             org.joda.time.DateTime temp = new org.joda.time.DateTime();
             String str = org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStampTZ.print(temp);
@@ -287,10 +307,12 @@ public class DateTypeFunctions {
         @Output TimeStampHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
             queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
         }
 
+        @Override
         public void eval() {
             out.value = queryStartDate;
         }
@@ -305,10 +327,12 @@ public class DateTypeFunctions {
         @Output TimeStampHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
             queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
         }
 
+        @Override
         public void eval() {
             out.value = queryStartDate;
         }
@@ -320,17 +344,19 @@ public class DateTypeFunctions {
         @Output TimeHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
 
             int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
             org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
-            queryStartTime= (int) ((now.getHourOfDay() * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
-                                   (now.getMinuteOfHour() * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
-                                   (now.getSecondOfMinute() * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
-                                   (now.getMillisOfSecond()));
+            queryStartTime= (now.getHourOfDay() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
+                                   (now.getMinuteOfHour() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
+                                   (now.getSecondOfMinute() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
+                                   (now.getMillisOfSecond());
         }
 
+        @Override
         public void eval() {
             out.value = queryStartTime;
         }
@@ -343,9 +369,11 @@ public class DateTypeFunctions {
     @Param TimeHolder right;
     @Output TimeStampHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             out.value = left.value + right.value;
         }
@@ -358,9 +386,11 @@ public class DateTypeFunctions {
         @Param DateHolder left;
         @Output TimeStampHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             out.value = left.value + right.value;
         }
@@ -377,9 +407,11 @@ public class DateTypeFunctions {
         @Param DateHolder right;
         @Output BigIntHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             if (1 == 1) {
                 throw new UnsupportedOperationException("date_part function should be rewritten as extract() functions");
@@ -387,26 +419,26 @@ public class DateTypeFunctions {
         }
     }
 
-    @SuppressWarnings("unused")
     @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public static class AgeTimeStampFunction implements DrillSimpleFunc {
         @Param TimeStampHolder left;
         @Param TimeStampHolder right;
         @Output IntervalHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
             long diff = left.value - right.value;
-            long days = diff / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         }
     }
 
-    @SuppressWarnings("unused")
     @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public static class AgeTimeStamp2Function implements DrillSimpleFunc {
         @Param TimeStampHolder right;
@@ -414,6 +446,7 @@ public class DateTypeFunctions {
         @Output IntervalHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
             int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
@@ -421,35 +454,36 @@ public class DateTypeFunctions {
             queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
         }
 
+        @Override
         public void eval() {
             long diff = queryStartDate - right.value;
-            long days = diff / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         }
     }
 
-    @SuppressWarnings("unused")
     @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public static class AgeDateFunction implements DrillSimpleFunc {
         @Param DateHolder left;
         @Param DateHolder right;
         @Output IntervalHolder out;
 
+        @Override
         public void setup() {
         }
 
+        @Override
         public void eval() {
           long diff = left.value - right.value;
-          long days = diff / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
-          out.months = (int) (days / org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-          out.days = (int) (days % org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-          out.milliseconds = (int) (diff % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+          long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+          out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+          out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+          out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         }
     }
 
-    @SuppressWarnings("unused")
     @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public static class AgeDate2Function implements DrillSimpleFunc {
         @Param DateHolder right;
@@ -457,6 +491,7 @@ public class DateTypeFunctions {
         @Output IntervalHolder out;
         @Inject ContextInformation contextInfo;
 
+        @Override
         public void setup() {
             int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
@@ -464,12 +499,13 @@ public class DateTypeFunctions {
             queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
         }
 
+        @Override
         public void eval() {
             long diff = queryStartDate - right.value;
-            long days = diff / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
+            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
+            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         }
     }
 
@@ -484,7 +520,7 @@ public class DateTypeFunctions {
 
         @Override
         public void eval() {
-            out.value = (int) (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+            out.value = (int) (in.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
         }
     }
 
@@ -520,7 +556,7 @@ public class DateTypeFunctions {
       @Override
       public void eval() {
         String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
-        date = (org.joda.time.DateTime) formatter.parseDateTime(inputDate);
+        date = formatter.parseDateTime(inputDate);
         out.value = date.getMillis() / 1000;
       }
     }
@@ -542,7 +578,7 @@ public class DateTypeFunctions {
       @Override
       public void eval() {
           String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
-          date = (org.joda.time.DateTime) formatter.parseDateTime(inputDate);
+          date = formatter.parseDateTime(inputDate);
           out.value = date.getMillis() / 1000;
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
index b7877df..97e0099 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MappifyUtility.java
@@ -17,22 +17,23 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
-import com.google.common.base.Charsets;
+import java.util.Iterator;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 //import org.apache.drill.common.types.DataMode;
 import org.apache.drill.common.types.MinorType;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.vector.complex.MapUtility;
 import org.apache.drill.exec.vector.complex.impl.SingleMapReaderImpl;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
-import io.netty.buffer.DrillBuf;
+import com.google.common.base.Charsets;
 
-import java.util.Iterator;
+import io.netty.buffer.DrillBuf;
 
 public class MappifyUtility {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java
index 123f8fa..7c87570 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java
@@ -180,7 +180,7 @@ public abstract class BaseOperatorContext implements OperatorContext {
   @Override
   public DrillFileSystem newFileSystem(Configuration conf) throws IOException {
     Preconditions.checkState(fs == null, "Tried to create a second FileSystem. Can only be called once per OperatorContext");
-    fs = new DrillFileSystem(conf, getStatsWriter());
+    fs = new DrillFileSystem(conf, getStats());
     return fs;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 210d0d4..d77d0b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -48,7 +48,6 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.FragmentOptionManager;
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.server.options.OptionSet;
 import org.apache.drill.exec.store.PartitionExplorer;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.testing.ExecutionControls;
@@ -190,12 +189,8 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     this(dbContext, fragment, null, connection, funcRegistry);
   }
 
-  public OptionManager getOptions() {
-    return fragmentOptions;
-  }
-
   @Override
-  public OptionSet getOptionSet() {
+  public OptionManager getOptions() {
     return fragmentOptions;
   }
 
@@ -345,6 +340,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     return buffers;
   }
 
+  @Override
   public OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats)
       throws OutOfMemoryException {
     OperatorContextImpl context = new OperatorContextImpl(popConfig, this, stats);
@@ -352,6 +348,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     return context;
   }
 
+  @Override
   public OperatorContext newOperatorContext(PhysicalOperator popConfig)
       throws OutOfMemoryException {
     OperatorContextImpl context = new OperatorContextImpl(popConfig, this);
@@ -385,6 +382,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     return executionControls;
   }
 
+  @Override
   public String getQueryUserName() {
     return fragment.getCredentials().getUserName();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextInterface.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextInterface.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextInterface.java
index 7d4ba18..9dbc411 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextInterface.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextInterface.java
@@ -22,11 +22,13 @@ import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.options.OptionSet;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.testing.ExecutionControls;
 
 import io.netty.buffer.DrillBuf;
@@ -57,10 +59,10 @@ public interface FragmentContextInterface {
    */
   FunctionImplementationRegistry getFunctionRegistry();
   /**
-   * Returns a read-only version of the session options.
+   * Returns the session options.
    * @return the session options
    */
-  OptionSet getOptionSet();
+  OptionManager getOptions();
 
   /**
    * Generates code for a class given a {@link ClassGenerator},
@@ -146,4 +148,12 @@ public interface FragmentContextInterface {
   DrillBuf getManagedBuffer();
 
   DrillBuf getManagedBuffer(int size);
+
+  OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats)
+      throws OutOfMemoryException;
+  OperatorContext newOperatorContext(PhysicalOperator popConfig)
+      throws OutOfMemoryException;
+
+  String getQueryUserName();
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 37653e0..3d2fdd8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -71,21 +71,11 @@ public interface OperatorContext {
   ExecutionControls getExecutionControls();
 
   /**
-   * A write-only interface to the Drill statistics mechanism. Allows
+   * Drill statistics mechanism. Allows
    * operators to update statistics.
    * @return operator statistics
    */
 
-  OperatorStatReceiver getStatsWriter();
-
-  /**
-   * Full operator stats (for legacy code). Prefer
-   * <tt>getStatsWriter()</tt> to allow code to easily run in a
-   * test environment.
-   *
-   * @return operator statistics
-   */
-
   OperatorStats getStats();
 
   ExecutorService getExecutor();

http://git-wip-us.apache.org/repos/asf/drill/blob/e791ed62/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index bc85c39..e4c7dd9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -88,11 +88,6 @@ class OperatorContextImpl extends BaseOperatorContext implements AutoCloseable {
   }
 
   @Override
-  public OperatorStatReceiver getStatsWriter() {
-    return stats;
-  }
-
-  @Override
   public <RESULT> ListenableFuture<RESULT> runCallableAs(final UserGroupInformation proxyUgi,
                                                          final Callable<RESULT> callable) {
     synchronized (this) {