You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2015/04/15 17:56:02 UTC

[1/6] drill git commit: DRILL-2675: Implement a subset of User Exceptions to improve how errors are reported to the user

Repository: drill
Updated Branches:
  refs/heads/master 859e6a86b -> 314e5a2a8


http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
index f62f060..111c3c1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 package org.apache.drill;
-import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedDataTypeException;
 import org.apache.drill.exec.work.foreman.UnsupportedFunctionException;
@@ -26,26 +26,31 @@ import org.junit.Test;
 public class TestDisabledFunctionality extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExampleQueries.class);
 
-  @Test(expected = RpcException.class)  // see DRILL-2054
+  @Test(expected = DrillUserException.class)  // see DRILL-2054
   public void testBooleanORExpression() throws Exception {
         test("select (1 = 1) || (1 > 0) from cp.`tpch/nation.parquet` ");
     }
 
-  @Test(expected = RpcException.class)  // see DRILL-2054
+  @Test(expected = DrillUserException.class)  // see DRILL-2054
   public void testBooleanORSelectClause() throws Exception {
     test("select true || true from cp.`tpch/nation.parquet` ");
   }
 
-  @Test(expected = RpcException.class)  // see DRILL-2054
+  @Test(expected = DrillUserException.class)  // see DRILL-2054
   public void testBooleanORWhereClause() throws Exception {
     test("select * from cp.`tpch/nation.parquet` where (true || true) ");
   }
 
-  @Test(expected = RpcException.class)  // see DRILL-2054
+  @Test(expected = DrillUserException.class)  // see DRILL-2054
   public void testBooleanAND() throws Exception {
     test("select true && true from cp.`tpch/nation.parquet` ");
   }
 
+  private static void throwAsUnsupportedException(DrillUserException ex) throws Exception {
+    SqlUnsupportedException.errorClassNameToException(ex.getOrCreatePBError(false).getException().getExceptionClass());
+    throw ex;
+  }
+
   @Test(expected = UnsupportedFunctionException.class)  // see DRILL-1937
   public void testDisabledExplainplanForComparisonWithNonscalarSubquery() throws Exception {
     try {
@@ -53,9 +58,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
            "where n_nationkey = " +
            "(select r_regionkey from cp.`tpch/region.parquet` " +
            "where r_regionkey = 1)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -66,9 +70,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
            "where n_nationkey = " +
            "(select r_regionkey from cp.`tpch/region.parquet` " +
            "where r_regionkey = 1)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -76,9 +79,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledUnion() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) UNION (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -86,9 +88,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledUnionDistinct() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) UNION DISTINCT (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -96,9 +97,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledIntersect() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) INTERSECT (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -106,9 +106,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledIntersectALL() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) INTERSECT ALL (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -116,9 +115,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledExceptALL() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) EXCEPT ALL (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -126,9 +124,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledExcept() throws Exception {
     try {
       test("(select n_name as name from cp.`tpch/nation.parquet`) EXCEPT (select r_name as name from cp.`tpch/region.parquet`)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -136,9 +133,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledNaturalJoin() throws Exception {
     try {
       test("select * from cp.`tpch/nation.parquet` NATURAL JOIN cp.`tpch/region.parquet`");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -146,9 +142,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCrossJoin() throws Exception {
     try {
       test("select * from cp.`tpch/nation.parquet` CROSS JOIN cp.`tpch/region.parquet`");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -156,9 +151,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastTINYINT() throws Exception {
     try {
       test("select cast(n_name as tinyint) from cp.`tpch/nation.parquet`;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -166,9 +160,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastSMALLINT() throws Exception {
     try {
       test("select cast(n_name as smallint) from cp.`tpch/nation.parquet`;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -176,9 +169,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastREAL() throws Exception {
     try {
       test("select cast(n_name as real) from cp.`tpch/nation.parquet`;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -186,9 +178,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCardinality() throws Exception {
     try {
       test("select cardinality(employee_id) from cp.`employee.json`;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -197,9 +188,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
     try {
       test("select a.*, b.user_port " +
           "from cp.`employee.json` a, sys.drillbits b;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -209,9 +199,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.*, b.user_port " +
           "from cp.`employee.json` a, sys.drillbits b " +
           "where a.position_id <> b.user_port;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -221,9 +210,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.last_name, b.n_name, c.r_name " +
           "from cp.`employee.json` a, cp.`tpch/nation.parquet` b, cp.`tpch/region.parquet` c " +
           "where a.position_id > b.n_nationKey and b.n_nationKey = c.r_regionkey;");
-      } catch(Exception ex) {
-        SqlUnsupportedException.errorMessageToException(ex.getMessage());
-        throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -233,9 +221,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.lastname, b.n_name " +
           "from cp.`employee.json` a LEFT JOIN cp.`tpch/nation.parquet` b " +
           "ON a.position_id > b.n_nationKey;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -245,9 +232,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.lastname, b.n_name " +
           "from cp.`employee.json` a INNER JOIN cp.`tpch/nation.parquet` b " +
           "ON a.position_id > b.n_nationKey;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -259,9 +245,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
           "where b.n_nationkey = " +
           "(select r_regionkey from cp.`tpch/region.parquet` " +
           "where r_regionkey = 1)");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -271,9 +256,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.lastname " +
           "from cp.`employee.json` a " +
           "where exists (select n_name from cp.`tpch/nation.parquet` b) AND a.position_id = 10");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -283,9 +267,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("explain plan for (select a.lastname, b.n_name " +
           "from cp.`employee.json` a INNER JOIN cp.`tpch/nation.parquet` b " +
           "ON a.position_id > b.n_nationKey);");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -295,9 +278,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("explain plan for (select a.lastname, b.n_name " +
           "from cp.`employee.json` a LEFT OUTER JOIN cp.`tpch/nation.parquet` b " +
           "ON (a.position_id > b.n_nationKey AND a.employee_id = b.n_regionkey));");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -307,9 +289,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
       test("select a.lastname, b.n_name " +
           "from cp.`employee.json` a RIGHT OUTER JOIN cp.`tpch/nation.parquet` b " +
           "ON (a.position_id > b.n_nationKey AND a.employee_id = b.n_regionkey);");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 
@@ -320,9 +301,8 @@ public class TestDisabledFunctionality extends BaseTestQuery{
           "OVER (PARTITION BY position_id order by position_id) " +
           "FROM cp.`employee.json` " +
           "order by employee_id;");
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
-      throw ex;
+    } catch(DrillUserException ex) {
+      throwAsUnsupportedException(ex);
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/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 effef9b..3dfaf10 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
@@ -17,10 +17,10 @@
  */
 package org.apache.drill;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.TestTools;
 import org.junit.Test;
-import org.apache.drill.exec.rpc.RpcException;
 
 public class TestStarQueries extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestStarQueries.class);
@@ -263,11 +263,11 @@ public class TestStarQueries extends BaseTestQuery{
     test("select *, first_name, *, last_name from cp.`employee.json`;");
   }
 
-  @Test(expected = RpcException.class)  // Should get "At line 1, column 8: Column 'n_nationkey' is ambiguous"
+  @Test(expected = DrillUserException.class)  // Should get "At line 1, column 8: Column 'n_nationkey' is ambiguous"
   public void testSelStarAmbiguousJoin() throws Exception {
     try {
       test("select x.n_nationkey, x.n_name, x.n_regionkey, x.r_name from (select * from cp.`tpch/nation.parquet` n, cp.`tpch/region.parquet` r where n.n_regionkey = r.r_regionkey) x " ) ;
-    } catch (RpcException e) {
+    } catch (DrillUserException e) {
       logger.info("***** Test resulted in expected failure: " + e.getMessage());
       throw e;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
index 11d83f9..3e00423 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
@@ -17,9 +17,9 @@
  */
 package org.apache.drill;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedRelOperatorException;
 import org.junit.Test;
@@ -227,8 +227,8 @@ public class TestUnionAll extends BaseTestQuery{
       String query = "(select * from dfs_test.tmp.`nation_view_testunionall`) " +
                      "union all (select * from cp.`tpch/region.parquet`)";
       test(query);
-    } catch(Exception ex) {
-      SqlUnsupportedException.errorMessageToException(ex.getMessage());
+    } catch(DrillUserException ex) {
+      SqlUnsupportedException.errorClassNameToException(ex.getOrCreatePBError(false).getException().getExceptionClass());
       throw ex;
     } finally {
       test("drop view nation_view_testunionall");
@@ -356,7 +356,7 @@ public class TestUnionAll extends BaseTestQuery{
         .build().run();
   }
 
-  @Test(expected = RpcException.class) // see DRILL-2590
+  @Test(expected = DrillUserException.class) // see DRILL-2590
   public void testUnionAllImplicitCastingFailure() throws Exception {
     String rootInt = FileUtils.getResourceAsFile("/store/json/intData.json").toURI().toString();
     String rootBoolean = FileUtils.getResourceAsFile("/store/json/booleanData.json").toURI().toString();

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
index e03098a..69c274c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
@@ -29,6 +29,7 @@ import org.apache.drill.QueryTestUtil;
 import org.apache.drill.SingleRowListener;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
@@ -39,13 +40,13 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.apache.drill.exec.proto.UserBitShared.ExceptionWrapper;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.testing.ExceptionInjectionUtil;
 import org.apache.drill.exec.testing.SimulatedExceptions.InjectionOption;
@@ -311,13 +312,10 @@ public class TestDrillbitResilience extends ExecTest {
    * @param desc the expected exception site description
    */
   private static void assertInjected(
-      final Throwable caught, final Class<? extends Throwable> exceptionClass, final String desc) {
-    final String cause = caught.getMessage();
-    final String[] causeParts = cause.split(":");
-    final String causeShortName = causeParts[0].trim();
-    final String causeDesc = causeParts[1].trim();
-    assertTrue(exceptionClass.getName().endsWith(causeShortName));
-    assertEquals(desc, causeDesc);
+      final DrillUserException caught, final Class<? extends Throwable> exceptionClass, final String desc) {
+    ExceptionWrapper cause = caught.getOrCreatePBError(false).getException();
+    assertEquals(exceptionClass.getName(), cause.getExceptionClass());
+    assertEquals(desc, cause.getMessage());
   }
 
   @Test
@@ -341,8 +339,8 @@ public class TestDrillbitResilience extends ExecTest {
     try {
       QueryTestUtil.test(drillClient, "select * from sys.drillbits");
       fail();
-    } catch(RpcException rpce) {
-      assertInjected(rpce, ForemanException.class, desc);
+    } catch(DrillUserException dre) {
+      assertInjected(dre, ForemanException.class, desc);
     }
   }
 
@@ -367,8 +365,8 @@ public class TestDrillbitResilience extends ExecTest {
     try {
       QueryTestUtil.test(drillClient, "select * from sys.drillbits");
       fail();
-    } catch(RpcException rpce) {
-      assertInjected(rpce, ForemanException.class, exceptionDesc);
+    } catch(DrillUserException dre) {
+      assertInjected(dre, ForemanException.class, exceptionDesc);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
index 55f0d75..3bcfbdf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
@@ -23,6 +23,8 @@ import java.io.UnsupportedEncodingException;
 import java.util.Arrays;
 import java.util.HashMap;
 
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -60,7 +62,7 @@ public class ParquetResultListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(RpcException ex) {
+  public void submissionFailed(DrillUserException ex) {
     logger.error("Submission failed.", ex);
     future.setException(ex);
   }
@@ -167,7 +169,9 @@ public class ParquetResultListener implements UserResultsListener {
           assertEquals("Mismatched record counts in vectors.", recordsInBatch, valuesChecked.get(s).intValue());
         }
         assertEquals("Record count incorrect for column: " + s, totalRecords, (long) valuesChecked.get(s));
-      } catch (AssertionError e) { submissionFailed(new RpcException(e)); }
+      } catch (AssertionError e) {
+        submissionFailed(ErrorHelper.wrap(e));
+      }
     }
 
     assert valuesChecked.keySet().size() > 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
index 882cdbd..cde2e95 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
@@ -23,12 +23,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -90,7 +90,7 @@ public class TestParquetPhysicalPlan extends ExecTest {
     private CountDownLatch latch = new CountDownLatch(1);
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       logger.error("submission failed", ex);
       latch.countDown();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
index d666d06..857a885 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
@@ -25,10 +25,8 @@ import net.hydromatic.avatica.ArrayImpl.Factory;
 import net.hydromatic.avatica.ColumnMetaData;
 import net.hydromatic.avatica.Cursor;
 
-import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.jdbc.impl.DrillResultSetImpl;
 
@@ -141,7 +139,7 @@ public class DrillCursor implements Cursor {
           }
           return true;
         }
-      } catch (RpcException | InterruptedException | SchemaChangeException e) {
+      } catch (Exception e) {
         throw new SQLException("Failure while executing query.", e);
       }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 24ef62b..a9753bc 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -29,12 +29,12 @@ import net.hydromatic.avatica.AvaticaPrepareResult;
 import net.hydromatic.avatica.AvaticaResultSet;
 import net.hydromatic.avatica.AvaticaStatement;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -160,7 +160,7 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
   // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
   public class ResultsListener implements UserResultsListener {
     private static final int MAX = 100;
-    private volatile RpcException ex;
+    private volatile DrillUserException ex;
     volatile boolean completed = false;
     private volatile boolean autoread = true;
     private volatile ConnectionThrottle throttle;
@@ -183,7 +183,7 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
     }
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       this.ex = ex;
       completed = true;
       close();
@@ -219,7 +219,7 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
     }
 
     // TODO:  Doc.:  Specify whether result can be null and what that means.
-    public QueryDataBatch getNext() throws RpcException, InterruptedException {
+    public QueryDataBatch getNext() throws Exception {
       while (true) {
         if (ex != null) {
           throw ex;

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
index f72d5e1..d9dba6e 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
@@ -270,7 +270,7 @@ public final class SchemaUserBitShared
                     output.writeObject(2, message.getEndpoint(), org.apache.drill.exec.proto.SchemaCoordinationProtos.DrillbitEndpoint.WRITE, false);
 
                 if(message.hasErrorType())
-                    output.writeInt32(3, message.getErrorType(), false);
+                    output.writeEnum(3, message.getErrorType().getNumber(), false);
                 if(message.hasMessage())
                     output.writeString(4, message.getMessage(), false);
                 if(message.hasException())
@@ -326,7 +326,7 @@ public final class SchemaUserBitShared
 
                             break;
                         case 3:
-                            builder.setErrorType(input.readInt32());
+                            builder.setErrorType(org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.valueOf(input.readEnum()));
                             break;
                         case 4:
                             builder.setMessage(input.readString());

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/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 96a921b..4b4e558 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
@@ -1705,15 +1705,15 @@ public final class UserBitShared {
      */
     org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
 
-    // optional int32 error_type = 3;
+    // optional .exec.shared.DrillPBError.ErrorType error_type = 3;
     /**
-     * <code>optional int32 error_type = 3;</code>
+     * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
      */
     boolean hasErrorType();
     /**
-     * <code>optional int32 error_type = 3;</code>
+     * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
      */
-    int getErrorType();
+    org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType getErrorType();
 
     // optional string message = 4;
     /**
@@ -1859,8 +1859,14 @@ public final class UserBitShared {
               break;
             }
             case 24: {
-              bitField0_ |= 0x00000004;
-              errorType_ = input.readInt32();
+              int rawValue = input.readEnum();
+              org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType value = org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(3, rawValue);
+              } else {
+                bitField0_ |= 0x00000004;
+                errorType_ = value;
+              }
               break;
             }
             case 34: {
@@ -1931,6 +1937,160 @@ public final class UserBitShared {
       return PARSER;
     }
 
+    /**
+     * Protobuf enum {@code exec.shared.DrillPBError.ErrorType}
+     */
+    public enum ErrorType
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>CONNECTION = 0;</code>
+       */
+      CONNECTION(0, 0),
+      /**
+       * <code>DATA_READ = 1;</code>
+       */
+      DATA_READ(1, 1),
+      /**
+       * <code>DATA_WRITE = 2;</code>
+       */
+      DATA_WRITE(2, 2),
+      /**
+       * <code>FUNCTION = 3;</code>
+       */
+      FUNCTION(3, 3),
+      /**
+       * <code>PARSE = 4;</code>
+       */
+      PARSE(4, 4),
+      /**
+       * <code>PERMISSION = 5;</code>
+       */
+      PERMISSION(5, 5),
+      /**
+       * <code>PLAN = 6;</code>
+       */
+      PLAN(6, 6),
+      /**
+       * <code>RESOURCE = 7;</code>
+       */
+      RESOURCE(7, 7),
+      /**
+       * <code>SYSTEM = 8;</code>
+       */
+      SYSTEM(8, 8),
+      /**
+       * <code>UNSUPPORTED_OPERATION = 9;</code>
+       */
+      UNSUPPORTED_OPERATION(9, 9),
+      ;
+
+      /**
+       * <code>CONNECTION = 0;</code>
+       */
+      public static final int CONNECTION_VALUE = 0;
+      /**
+       * <code>DATA_READ = 1;</code>
+       */
+      public static final int DATA_READ_VALUE = 1;
+      /**
+       * <code>DATA_WRITE = 2;</code>
+       */
+      public static final int DATA_WRITE_VALUE = 2;
+      /**
+       * <code>FUNCTION = 3;</code>
+       */
+      public static final int FUNCTION_VALUE = 3;
+      /**
+       * <code>PARSE = 4;</code>
+       */
+      public static final int PARSE_VALUE = 4;
+      /**
+       * <code>PERMISSION = 5;</code>
+       */
+      public static final int PERMISSION_VALUE = 5;
+      /**
+       * <code>PLAN = 6;</code>
+       */
+      public static final int PLAN_VALUE = 6;
+      /**
+       * <code>RESOURCE = 7;</code>
+       */
+      public static final int RESOURCE_VALUE = 7;
+      /**
+       * <code>SYSTEM = 8;</code>
+       */
+      public static final int SYSTEM_VALUE = 8;
+      /**
+       * <code>UNSUPPORTED_OPERATION = 9;</code>
+       */
+      public static final int UNSUPPORTED_OPERATION_VALUE = 9;
+
+
+      public final int getNumber() { return value; }
+
+      public static ErrorType valueOf(int value) {
+        switch (value) {
+          case 0: return CONNECTION;
+          case 1: return DATA_READ;
+          case 2: return DATA_WRITE;
+          case 3: return FUNCTION;
+          case 4: return PARSE;
+          case 5: return PERMISSION;
+          case 6: return PLAN;
+          case 7: return RESOURCE;
+          case 8: return SYSTEM;
+          case 9: return UNSUPPORTED_OPERATION;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<ErrorType>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<ErrorType>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<ErrorType>() {
+              public ErrorType findValueByNumber(int number) {
+                return ErrorType.valueOf(number);
+              }
+            };
+
+      public final com.google.protobuf.Descriptors.EnumValueDescriptor
+          getValueDescriptor() {
+        return getDescriptor().getValues().get(index);
+      }
+      public final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptorForType() {
+        return getDescriptor();
+      }
+      public static final com.google.protobuf.Descriptors.EnumDescriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final ErrorType[] VALUES = values();
+
+      public static ErrorType valueOf(
+          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+        if (desc.getType() != getDescriptor()) {
+          throw new java.lang.IllegalArgumentException(
+            "EnumValueDescriptor is not for this type.");
+        }
+        return VALUES[desc.getIndex()];
+      }
+
+      private final int index;
+      private final int value;
+
+      private ErrorType(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:exec.shared.DrillPBError.ErrorType)
+    }
+
     private int bitField0_;
     // optional string error_id = 1;
     public static final int ERROR_ID_FIELD_NUMBER = 1;
@@ -2009,19 +2169,19 @@ public final class UserBitShared {
       return endpoint_;
     }
 
-    // optional int32 error_type = 3;
+    // optional .exec.shared.DrillPBError.ErrorType error_type = 3;
     public static final int ERROR_TYPE_FIELD_NUMBER = 3;
-    private int errorType_;
+    private org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType errorType_;
     /**
-     * <code>optional int32 error_type = 3;</code>
+     * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
      */
     public boolean hasErrorType() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional int32 error_type = 3;</code>
+     * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
      */
-    public int getErrorType() {
+    public org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType getErrorType() {
       return errorType_;
     }
 
@@ -2149,7 +2309,7 @@ public final class UserBitShared {
     private void initFields() {
       errorId_ = "";
       endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
-      errorType_ = 0;
+      errorType_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.CONNECTION;
       message_ = "";
       exception_ = org.apache.drill.exec.proto.UserBitShared.ExceptionWrapper.getDefaultInstance();
       parsingError_ = java.util.Collections.emptyList();
@@ -2173,7 +2333,7 @@ public final class UserBitShared {
         output.writeMessage(2, endpoint_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt32(3, errorType_);
+        output.writeEnum(3, errorType_.getNumber());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeBytes(4, getMessageBytes());
@@ -2203,7 +2363,7 @@ public final class UserBitShared {
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(3, errorType_);
+          .computeEnumSize(3, errorType_.getNumber());
       }
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
@@ -2344,7 +2504,7 @@ public final class UserBitShared {
           endpointBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
-        errorType_ = 0;
+        errorType_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.CONNECTION;
         bitField0_ = (bitField0_ & ~0x00000004);
         message_ = "";
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -2728,35 +2888,38 @@ public final class UserBitShared {
         return endpointBuilder_;
       }
 
-      // optional int32 error_type = 3;
-      private int errorType_ ;
+      // optional .exec.shared.DrillPBError.ErrorType error_type = 3;
+      private org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType errorType_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.CONNECTION;
       /**
-       * <code>optional int32 error_type = 3;</code>
+       * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
        */
       public boolean hasErrorType() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional int32 error_type = 3;</code>
+       * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
        */
-      public int getErrorType() {
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType getErrorType() {
         return errorType_;
       }
       /**
-       * <code>optional int32 error_type = 3;</code>
+       * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
        */
-      public Builder setErrorType(int value) {
+      public Builder setErrorType(org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
         bitField0_ |= 0x00000004;
         errorType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 error_type = 3;</code>
+       * <code>optional .exec.shared.DrillPBError.ErrorType error_type = 3;</code>
        */
       public Builder clearErrorType() {
         bitField0_ = (bitField0_ & ~0x00000004);
-        errorType_ = 0;
+        errorType_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType.CONNECTION;
         onChanged();
         return this;
       }
@@ -19611,103 +19774,108 @@ 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\"\323\001\n\014DrillPBError\022\020\n\010error_id\030\001 \001(\t\022(" +
+      "\001(\020\"\235\003\n\014DrillPBError\022\020\n\010error_id\030\001 \001(\t\022(" +
       "\n\010endpoint\030\002 \001(\0132\026.exec.DrillbitEndpoint" +
-      "\022\022\n\nerror_type\030\003 \001(\005\022\017\n\007message\030\004 \001(\t\0220\n" +
-      "\texception\030\005 \001(\0132\035.exec.shared.Exception" +
-      "Wrapper\0220\n\rparsing_error\030\006 \003(\0132\031.exec.sh" +
-      "ared.ParsingError\"\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.Stac" +
-      "kTraceElementWrapper\022,\n\005cause\030\004 \001(\0132\035.ex" +
-      "ec.shared.ExceptionWrapper\"\205\001\n\030StackTrac" +
-      "eElementWrapper\022\022\n\nclass_name\030\001 \001(\t\022\021\n\tf" +
-      "ile_name\030\002 \001(\t\022\023\n\013line_number\030\003 \001(\005\022\023\n\013m" +
-      "ethod_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\014re",
-      "cord_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exec.s" +
-      "hared.SerializedField\022)\n!carries_two_byt" +
-      "e_selection_vector\030\003 \001(\010\"\205\001\n\010NamePart\022(\n" +
-      "\004type\030\001 \001(\0162\032.exec.shared.NamePart.Type\022" +
-      "\014\n\004name\030\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.shar" +
-      "ed.NamePart\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY\020\001" +
-      "\"\351\001\n\017SerializedField\022%\n\nmajor_type\030\001 \001(\013" +
-      "2\021.common.MajorType\022(\n\tname_part\030\002 \001(\0132\025" +
-      ".exec.shared.NamePart\022+\n\005child\030\003 \003(\0132\034.e" +
-      "xec.shared.SerializedField\022\023\n\013value_coun",
-      "t\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\023\n\013grou" +
-      "p_count\030\006 \001(\005\022\025\n\rbuffer_length\030\007 \001(\005\"7\n\n" +
-      "NodeStatus\022\017\n\007node_id\030\001 \001(\005\022\030\n\020memory_fo" +
-      "otprint\030\002 \001(\003\"\206\002\n\013QueryResult\0228\n\013query_s" +
-      "tate\030\001 \001(\0162#.exec.shared.QueryResult.Que" +
-      "ryState\022&\n\010query_id\030\002 \001(\0132\024.exec.shared." +
-      "QueryId\022(\n\005error\030\003 \003(\0132\031.exec.shared.Dri" +
-      "llPBError\"k\n\nQueryState\022\013\n\007PENDING\020\000\022\013\n\007" +
-      "RUNNING\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\"p",
-      "\n\tQueryData\022&\n\010query_id\030\001 \001(\0132\024.exec.sha" +
-      "red.QueryId\022\021\n\trow_count\030\002 \001(\005\022(\n\003def\030\003 " +
-      "\001(\0132\033.exec.shared.RecordBatchDef\"\224\001\n\tQue" +
-      "ryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222\n\005" +
-      "state\030\003 \001(\0162#.exec.shared.QueryResult.Qu" +
-      "eryState\022\014\n\004user\030\004 \001(\t\022\'\n\007foreman\030\005 \001(\0132" +
-      "\026.exec.DrillbitEndpoint\"\336\002\n\014QueryProfile" +
-      "\022 \n\002id\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004ty" +
-      "pe\030\002 \001(\0162\026.exec.shared.QueryType\022\r\n\005star" +
-      "t\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n\004p",
-      "lan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.Drillb" +
-      "itEndpoint\0222\n\005state\030\010 \001(\0162#.exec.shared." +
-      "QueryResult.QueryState\022\027\n\017total_fragment" +
-      "s\030\t \001(\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n\020f" +
-      "ragment_profile\030\013 \003(\0132!.exec.shared.Majo" +
-      "rFragmentProfile\"t\n\024MajorFragmentProfile" +
-      "\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026minor_fra" +
-      "gment_profile\030\002 \003(\0132!.exec.shared.MinorF" +
-      "ragmentProfile\"\274\002\n\024MinorFragmentProfile\022" +
-      ")\n\005state\030\001 \001(\0162\032.exec.shared.FragmentSta",
-      "te\022(\n\005error\030\002 \001(\0132\031.exec.shared.DrillPBE" +
-      "rror\022\031\n\021minor_fragment_id\030\003 \001(\005\0226\n\020opera" +
-      "tor_profile\030\004 \003(\0132\034.exec.shared.Operator" +
-      "Profile\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\017max_memory" +
-      "_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.exec.Dri" +
-      "llbitEndpoint\"\377\001\n\017OperatorProfile\0221\n\rinp" +
-      "ut_profile\030\001 \003(\0132\032.exec.shared.StreamPro" +
-      "file\022\023\n\013operator_id\030\003 \001(\005\022\025\n\roperator_ty" +
-      "pe\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_alloca" +
-      "ted\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\rStrea" +
-      "mProfile\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\013MetricValue\022\021\n\tme" +
-      "tric_id\030\001 \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014dou" +
-      "ble_value\030\003 \001(\001*5\n\nRpcChannel\022\017\n\013BIT_CON" +
-      "TROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*/\n\tQueryT" +
-      "ype\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*" +
-      "k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITIN",
-      "G_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*\312\005\n\020CoreOpe" +
-      "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS" +
-      "T_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\025HAS" +
-      "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI" +
-      "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE" +
-      "R\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\030SELECT" +
-      "ION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREG",
-      "ATE\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\026" +
-      "PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCA" +
-      "N\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_S" +
-      "CAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_" +
-      "SCAN\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\006WINDO" +
-      "W\020\"\022\024\n\020NESTED_LOOP_JOIN\020#B.\n\033org.apache.",
-      "drill.exec.protoB\rUserBitSharedH\001"
+      "\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\"\242\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\"\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\"\351\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\023\n\013gr" +
+      "oup_count\030\006 \001(\005\022\025\n\rbuffer_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\"\206\002\n\013QueryResult\0228\n\013query" +
+      "_state\030\001 \001(\0162#.exec.shared.QueryResult.Q",
+      "ueryState\022&\n\010query_id\030\002 \001(\0132\024.exec.share" +
+      "d.QueryId\022(\n\005error\030\003 \003(\0132\031.exec.shared.D" +
+      "rillPBError\"k\n\nQueryState\022\013\n\007PENDING\020\000\022\013" +
+      "\n\007RUNNING\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" +
+      "\"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\"\224\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\014\n\004user\030\004 \001(\t\022\'\n\007foreman\030\005 \001(" +
+      "\0132\026.exec.DrillbitEndpoint\"\336\002\n\014QueryProfi" +
+      "le\022 \n\002id\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004" +
+      "type\030\002 \001(\0162\026.exec.shared.QueryType\022\r\n\005st" +
+      "art\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.Dril" +
+      "lbitEndpoint\0222\n\005state\030\010 \001(\0162#.exec.share" +
+      "d.QueryResult.QueryState\022\027\n\017total_fragme" +
+      "nts\030\t \001(\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n" +
+      "\020fragment_profile\030\013 \003(\0132!.exec.shared.Ma",
+      "jorFragmentProfile\"t\n\024MajorFragmentProfi" +
+      "le\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026minor_f" +
+      "ragment_profile\030\002 \003(\0132!.exec.shared.Mino" +
+      "rFragmentProfile\"\274\002\n\024MinorFragmentProfil" +
+      "e\022)\n\005state\030\001 \001(\0162\032.exec.shared.FragmentS" +
+      "tate\022(\n\005error\030\002 \001(\0132\031.exec.shared.DrillP" +
+      "BError\022\031\n\021minor_fragment_id\030\003 \001(\005\0226\n\020ope" +
+      "rator_profile\030\004 \003(\0132\034.exec.shared.Operat" +
+      "orProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n\010end_tim" +
+      "e\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017max_memo",
+      "ry_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.exec.D" +
+      "rillbitEndpoint\"\377\001\n\017OperatorProfile\0221\n\ri" +
+      "nput_profile\030\001 \003(\0132\032.exec.shared.StreamP" +
+      "rofile\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\rproce" +
+      "ss_nanos\030\006 \001(\003\022#\n\033peak_local_memory_allo" +
+      "cated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132\030.exec.share" +
+      "d.MetricValue\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStr" +
+      "eamProfile\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\013MetricValue\022\021\n\t",
+      "metric_id\030\001 \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014d" +
+      "ouble_value\030\003 \001(\001*5\n\nRpcChannel\022\017\n\013BIT_C" +
+      "ONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*/\n\tQuer" +
+      "yType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020" +
+      "\003*k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAIT" +
+      "ING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHE" +
+      "D\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005*\312\005\n\020CoreO" +
+      "peratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADC" +
+      "AST_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGA" +
+      "TE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025H",
+      "ASH_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MER" +
+      "GING_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SEN" +
+      "DER\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\030SELE" +
+      "CTION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGR" +
+      "EGATE\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\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_S" +
+      "CAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB" +
+      "_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SU",
+      "B_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SC" +
+      "AN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_S" +
+      "UB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUC" +
+      "ER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WIN" +
+      "DOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#B.\n\033org.apach" +
+      "e.drill.exec.protoB\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/99b6d0e6/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 ac9cef5..873ffa4 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
@@ -35,6 +35,50 @@ import com.dyuproject.protostuff.Schema;
 
 public final class DrillPBError implements Externalizable, Message<DrillPBError>, Schema<DrillPBError>
 {
+    public enum ErrorType implements com.dyuproject.protostuff.EnumLite<ErrorType>
+    {
+        CONNECTION(0),
+        DATA_READ(1),
+        DATA_WRITE(2),
+        FUNCTION(3),
+        PARSE(4),
+        PERMISSION(5),
+        PLAN(6),
+        RESOURCE(7),
+        SYSTEM(8),
+        UNSUPPORTED_OPERATION(9);
+        
+        public final int number;
+        
+        private ErrorType (int number)
+        {
+            this.number = number;
+        }
+        
+        public int getNumber()
+        {
+            return number;
+        }
+        
+        public static ErrorType valueOf(int number)
+        {
+            switch(number) 
+            {
+                case 0: return CONNECTION;
+                case 1: return DATA_READ;
+                case 2: return DATA_WRITE;
+                case 3: return FUNCTION;
+                case 4: return PARSE;
+                case 5: return PERMISSION;
+                case 6: return PLAN;
+                case 7: return RESOURCE;
+                case 8: return SYSTEM;
+                case 9: return UNSUPPORTED_OPERATION;
+                default: return null;
+            }
+        }
+    }
+
 
     public static Schema<DrillPBError> getSchema()
     {
@@ -51,7 +95,7 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
     
     private String errorId;
     private DrillbitEndpoint endpoint;
-    private int errorType;
+    private ErrorType errorType;
     private String message;
     private ExceptionWrapper exception;
     private List<ParsingError> parsingError;
@@ -91,12 +135,12 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
 
     // errorType
 
-    public int getErrorType()
+    public ErrorType getErrorType()
     {
-        return errorType;
+        return errorType == null ? ErrorType.CONNECTION : errorType;
     }
 
-    public DrillPBError setErrorType(int errorType)
+    public DrillPBError setErrorType(ErrorType errorType)
     {
         this.errorType = errorType;
         return this;
@@ -203,7 +247,7 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
                     break;
 
                 case 3:
-                    message.errorType = input.readInt32();
+                    message.errorType = ErrorType.valueOf(input.readEnum());
                     break;
                 case 4:
                     message.message = input.readString();
@@ -234,8 +278,8 @@ public final class DrillPBError implements Externalizable, Message<DrillPBError>
              output.writeObject(2, message.endpoint, DrillbitEndpoint.getSchema(), false);
 
 
-        if(message.errorType != 0)
-            output.writeInt32(3, message.errorType, false);
+        if(message.errorType != null)
+             output.writeEnum(3, message.errorType.number, false);
 
         if(message.message != null)
             output.writeString(4, message.message, false);

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 2938114..0f86958 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -30,9 +30,21 @@ message QueryId {
 }
 
 message DrillPBError{
+  enum ErrorType {
+    CONNECTION = 0;
+    DATA_READ = 1;
+    DATA_WRITE = 2;
+    FUNCTION = 3;
+    PARSE = 4;
+    PERMISSION = 5;
+    PLAN = 6;
+    RESOURCE = 7;
+    SYSTEM = 8;
+    UNSUPPORTED_OPERATION = 9;
+  }
   optional string error_id = 1; // for debug tracing purposes
   optional DrillbitEndpoint endpoint = 2;
-  optional int32 error_type = 3;
+  optional ErrorType error_type = 3;
   optional string message = 4;
   optional ExceptionWrapper exception = 5;
   repeated ParsingError parsing_error = 6; //optional, used when providing location of error within a piece of text.


[6/6] drill git commit: DRILL-2695: Add Support for large in conditions through the use of the Values operator. Update JSON reader to support reading Extended JSON. Update JSON writer to support writing extended JSON data. Update JSON reader to automatic

Posted by ja...@apache.org.
DRILL-2695: Add Support for large in conditions through the use of the Values operator.
Update JSON reader to support reading Extended JSON.
Update JSON writer to support writing extended JSON data.
Update JSON reader to automatically unwrap a file that includes a single top-level array (used by values).
Update Options manager to use getOption(<Type>Validator) to directly retrieve typed value.
Remove JSON rewinding
Add support for CONVERT_TO( [], 'SIMPLEJSON') to disable extended types as part of udf use.


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

Branch: refs/heads/master
Commit: 314e5a2a8f476f059153fde1b7e7da7d882db94e
Parents: 75455e8
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Mar 8 21:37:44 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Apr 15 08:10:35 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/JSONOptions.java    |  12 +
 .../drill/common/logical/data/Constant.java     |  50 --
 .../drill/common/logical/data/Values.java       |  69 +++
 .../data/visitors/AbstractLogicalVisitor.java   |   4 +-
 .../logical/data/visitors/LogicalVisitor.java   |   4 +-
 .../org/apache/drill/common/types/Types.java    |  48 +-
 .../exec/store/mongo/MongoRecordReader.java     |   2 +-
 .../src/main/codegen/templates/BaseWriter.java  |   5 -
 .../main/codegen/templates/ComplexReaders.java  |  24 +-
 .../main/codegen/templates/ComplexWriters.java  |  61 +--
 .../templates/JsonOutputRecordWriter.java       | 119 ++---
 .../src/main/codegen/templates/ListWriters.java |  48 +-
 .../src/main/codegen/templates/MapWriters.java  |  44 +-
 .../org/apache/drill/exec/ExecConstants.java    |   3 +-
 .../drill/exec/expr/EvaluationVisitor.java      |   5 +-
 .../drill/exec/expr/fn/impl/DateUtility.java    |  13 +
 .../exec/expr/fn/impl/conv/JsonConvertFrom.java |   4 +-
 .../exec/expr/fn/impl/conv/JsonConvertTo.java   |  32 +-
 .../physical/base/AbstractPhysicalVisitor.java  |   6 +
 .../exec/physical/base/PhysicalVisitor.java     |   2 +
 .../drill/exec/physical/config/Values.java      |  71 +++
 .../physical/impl/flatten/FlattenTemplate.java  |   5 +-
 .../impl/values/ValuesBatchCreator.java         |  44 ++
 .../drill/exec/planner/logical/DrillOptiq.java  |   4 +-
 .../exec/planner/logical/DrillRuleSets.java     |   3 +
 .../exec/planner/logical/DrillValuesRel.java    | 255 ++++++++-
 .../planner/logical/ScanFieldDeterminer.java    |   4 +-
 .../drill/exec/planner/physical/ValuesPrel.java |  78 +++
 .../exec/planner/physical/ValuesPrule.java      |  48 ++
 .../exec/server/options/BaseOptionManager.java  |  56 ++
 .../server/options/FallbackOptionManager.java   |   2 +-
 .../exec/server/options/OptionManager.java      |   9 +
 .../server/options/SystemOptionManager.java     |   7 +-
 .../exec/store/easy/json/JSONFormatPlugin.java  |   2 +
 .../exec/store/easy/json/JSONRecordReader.java  | 127 +++--
 .../exec/store/easy/json/JsonProcessor.java     |   4 +-
 .../exec/store/easy/json/JsonRecordWriter.java  |  14 +-
 .../store/easy/json/RewindableUtf8Reader.java   | 157 ------
 .../easy/json/reader/BaseJsonProcessor.java     |  38 +-
 .../easy/json/reader/CountingJsonReader.java    |   4 +-
 .../exec/store/parquet2/DrillParquetReader.java |   9 -
 .../DrillParquetRecordMaterializer.java         |   4 -
 .../drill/exec/vector/RepeatedVector.java       |   2 +-
 .../drill/exec/vector/complex/WriteState.java   |  46 --
 .../exec/vector/complex/fn/BasicJsonOutput.java | 530 +++++++++++++++++++
 .../vector/complex/fn/DateOutputFormat.java     |  37 ++
 .../vector/complex/fn/ExtendedJsonOutput.java   | 183 +++++++
 .../exec/vector/complex/fn/ExtendedType.java    |  40 ++
 .../vector/complex/fn/ExtendedTypeName.java     |  29 +
 .../exec/vector/complex/fn/JsonOutput.java      | 109 ++++
 .../exec/vector/complex/fn/JsonReader.java      | 335 +++++++-----
 .../exec/vector/complex/fn/JsonWriter.java      | 104 ++--
 .../exec/vector/complex/fn/VectorOutput.java    | 295 +++++++++++
 .../exec/vector/complex/fn/WorkingBuffer.java   |  70 +++
 .../vector/complex/impl/AbstractBaseWriter.java |  21 -
 .../vector/complex/impl/ComplexWriterImpl.java  |   5 -
 .../complex/impl/RepeatedListReaderImpl.java    |   4 +-
 .../complex/impl/RepeatedMapReaderImpl.java     |   4 +-
 .../complex/impl/SingleMapReaderImpl.java       |  12 +-
 .../complex/impl/VectorContainerWriter.java     |   5 -
 .../physical/impl/filter/TestLargeInClause.java |  62 +++
 .../exec/store/json/TestJsonRecordReader.java   |   5 +
 .../complex/writer/TestExtendedTypes.java       |  57 ++
 .../vector/complex/writer/TestRepeated.java     |   3 +-
 .../src/test/resources/store/json/listdoc.json  |   4 +
 .../test/resources/vector/complex/extended.json |  41 ++
 66 files changed, 2708 insertions(+), 795 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/JSONOptions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/JSONOptions.java b/common/src/main/java/org/apache/drill/common/JSONOptions.java
index 945cd92..9e0514c 100644
--- a/common/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/common/src/main/java/org/apache/drill/common/JSONOptions.java
@@ -42,7 +42,9 @@ import com.fasterxml.jackson.databind.SerializerProvider;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.node.TreeTraversingParser;
 import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.base.Preconditions;
 
 @JsonSerialize(using = Se.class)
 @JsonDeserialize(using = De.class)
@@ -98,6 +100,16 @@ public class JSONOptions {
     return getListWith(config.getMapper(), t);
   }
 
+  public JsonNode asNode(){
+    Preconditions.checkArgument(this.root != null, "Attempted to grab JSONOptions as JsonNode when no root node was stored.  You can only convert non-opaque JSONOptions values to JsonNodes.");
+    return root;
+  }
+
+  public JsonParser asParser(){
+    Preconditions.checkArgument(this.root != null, "Attempted to grab JSONOptions as Parser when no root node was stored.  You can only convert non-opaque JSONOptions values to parsers.");
+    return new TreeTraversingParser(root);
+  }
+
   @SuppressWarnings("unchecked")
   public <T> T getListWith(ObjectMapper mapper, TypeReference<T> t) throws IOException {
     if (opaque != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Constant.java b/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
deleted file mode 100644
index 460803d..0000000
--- a/common/src/main/java/org/apache/drill/common/logical/data/Constant.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.common.logical.data;
-
-import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-
-@JsonTypeName("constant")
-public class Constant extends SourceOperator {
-
-    private final JSONOptions content;
-
-    @JsonCreator
-    public Constant(@JsonProperty("content") JSONOptions content){
-        super();
-        this.content = content;
-        Preconditions.checkNotNull(content, "content attribute is required for source operator 'constant'.");
-    }
-
-    public JSONOptions getContent() {
-        return content;
-    }
-
-    @Override
-    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
-      return logicalVisitor.visitConstant(this, value);
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/Values.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Values.java b/common/src/main/java/org/apache/drill/common/logical/data/Values.java
new file mode 100644
index 0000000..9276e55
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Values.java
@@ -0,0 +1,69 @@
+/**
+ * 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.common.logical.data;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("values")
+public class Values extends SourceOperator {
+
+    private final JSONOptions content;
+
+    @JsonCreator
+    public Values(@JsonProperty("content") JSONOptions content){
+        super();
+        this.content = content;
+        Preconditions.checkNotNull(content, "content attribute is required for source operator 'constant'.");
+    }
+
+    public JSONOptions getContent() {
+        return content;
+    }
+
+    @Override
+    public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+      return logicalVisitor.visitValues(this, value);
+    }
+
+    public static Builder builder(){
+      return new Builder();
+    }
+
+    public static class Builder extends AbstractBuilder<Values>{
+      private JSONOptions content;
+
+      public Builder content(JsonNode n){
+        content = new JSONOptions(n, JsonLocation.NA);
+        return this;
+      }
+
+      @Override
+      public Values build() {
+        return new Values(content);
+      }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java b/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
index 92e370f..165ee66 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.common.logical.data.visitors;
 
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
 import org.apache.drill.common.logical.data.GroupingAggregate;
@@ -115,7 +115,7 @@ public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implemen
     }
 
     @Override
-    public T visitConstant(Constant constant, X value) throws E {
+    public T visitValues(Values constant, X value) throws E {
        return visitOp(constant, value);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java b/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
index 3a426bf..1e07dc6 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@ -18,7 +18,7 @@
 package org.apache.drill.common.logical.data.visitors;
 
 
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
 import org.apache.drill.common.logical.data.GroupingAggregate;
@@ -53,7 +53,7 @@ public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
     public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
 
     public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
-    public RETURN visitConstant(Constant constant, EXTRA value) throws EXCEP;
+    public RETURN visitValues(Values constant, EXTRA value) throws EXCEP;
     public RETURN visitOrder(Order order, EXTRA value) throws EXCEP;
     public RETURN visitJoin(Join join, EXTRA value) throws EXCEP;
     public RETURN visitLimit(Limit limit, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/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 c41727d..cec433f 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
@@ -18,7 +18,6 @@
 package org.apache.drill.common.types;
 
 import static org.apache.drill.common.types.TypeProtos.DataMode.REPEATED;
-import static org.apache.drill.common.types.TypeProtos.MinorType.*;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.types.TypeProtos.DataMode;
@@ -39,7 +38,7 @@ public class Types {
     UNKNOWN, NONE, EQUAL, ORDERED;
   }
 
-  public static boolean isComplex(MajorType type) {
+  public static boolean isComplex(final MajorType type) {
     switch(type.getMinorType()) {
     case LIST:
     case MAP:
@@ -49,11 +48,11 @@ public class Types {
     return false;
   }
 
-  public static boolean isRepeated(MajorType type) {
+  public static boolean isRepeated(final MajorType type) {
     return type.getMode() == REPEATED ;
   }
 
-  public static boolean isNumericType(MajorType type) {
+  public static boolean isNumericType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -85,7 +84,7 @@ public class Types {
   /***
    * Gets JDBC type code for given Drill RPC-/protobuf-level type.
    */
-  public static int getJdbcType(MajorType type) {
+  public static int getJdbcType(final MajorType type) {
     if (type.getMode() == DataMode.REPEATED) {
       return java.sql.Types.ARRAY;
     }
@@ -159,7 +158,7 @@ public class Types {
     }
   }
 
-  public static boolean isUnSigned(MajorType type) {
+  public static boolean isUnSigned(final MajorType type) {
     switch(type.getMinorType()) {
     case UINT1:
     case UINT2:
@@ -171,7 +170,7 @@ public class Types {
     }
 
   }
-  public static boolean usesHolderForGet(MajorType type) {
+  public static boolean usesHolderForGet(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return true;
     }
@@ -199,7 +198,7 @@ public class Types {
 
   }
 
-  public static boolean isFixedWidthType(MajorType type) {
+  public static boolean isFixedWidthType(final MajorType type) {
     switch(type.getMinorType()) {
     case VARBINARY:
     case VAR16CHAR:
@@ -211,7 +210,7 @@ public class Types {
   }
 
 
-  public static boolean isStringScalarType(MajorType type) {
+  public static boolean isStringScalarType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -226,7 +225,7 @@ public class Types {
     }
   }
 
-  public static boolean isBytesScalarType(MajorType type) {
+  public static boolean isBytesScalarType(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return false;
     }
@@ -239,7 +238,7 @@ public class Types {
     }
   }
 
-  public static Comparability getComparability(MajorType type) {
+  public static Comparability getComparability(final MajorType type) {
     if (type.getMode() == REPEATED) {
       return Comparability.NONE;
     }
@@ -261,7 +260,7 @@ public class Types {
   }
 
 
-  public static boolean softEquals(MajorType a, MajorType b, boolean allowNullSwap) {
+  public static boolean softEquals(final MajorType a, final MajorType b, final boolean allowNullSwap) {
     if (a.getMinorType() != b.getMinorType()) {
         return false;
     }
@@ -279,31 +278,31 @@ public class Types {
     return a.getMode() == b.getMode();
   }
 
-  public static boolean isLateBind(MajorType type) {
+  public static boolean isLateBind(final MajorType type) {
     return type.getMinorType() == MinorType.LATE;
   }
 
-  public static MajorType withMode(MinorType type, DataMode mode) {
+  public static MajorType withMode(final MinorType type, final DataMode mode) {
     return MajorType.newBuilder().setMode(mode).setMinorType(type).build();
   }
 
-  public static MajorType withScaleAndPrecision(MinorType type, DataMode mode, int scale, int precision) {
+  public static MajorType withScaleAndPrecision(final MinorType type, final DataMode mode, final int scale, final int precision) {
     return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
   }
 
-  public static MajorType required(MinorType type) {
+  public static MajorType required(final MinorType type) {
     return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
   }
 
-  public static MajorType repeated(MinorType type) {
+  public static MajorType repeated(final MinorType type) {
     return MajorType.newBuilder().setMode(REPEATED).setMinorType(type).build();
   }
 
-  public static MajorType optional(MinorType type) {
+  public static MajorType optional(final MinorType type) {
     return MajorType.newBuilder().setMode(DataMode.OPTIONAL).setMinorType(type).build();
   }
 
-  public static MajorType overrideMinorType(MajorType originalMajorType, MinorType overrideMinorType) {
+  public static MajorType overrideMinorType(final MajorType originalMajorType, final MinorType overrideMinorType) {
     switch (originalMajorType.getMode()) {
       case REPEATED:
         return repeated(overrideMinorType);
@@ -316,11 +315,11 @@ public class Types {
     }
   }
 
-  public static MajorType overrideMode(MajorType originalMajorType, DataMode overrideMode) {
+  public static MajorType overrideMode(final MajorType originalMajorType, final DataMode overrideMode) {
     return withScaleAndPrecision(originalMajorType.getMinorType(), overrideMode, originalMajorType.getScale(), originalMajorType.getPrecision());
   }
 
-  public static MajorType getMajorTypeFromName(String typeName) {
+  public static MajorType getMajorTypeFromName(final String typeName) {
     return getMajorTypeFromName(typeName, DataMode.REQUIRED);
   }
 
@@ -376,6 +375,7 @@ public class Types {
     case "binary":
       return MinorType.VARBINARY;
     case "json":
+    case "simplejson":
       return MinorType.LATE;
     case "null":
     case "any":
@@ -385,11 +385,11 @@ public class Types {
     }
   }
 
-  public static MajorType getMajorTypeFromName(String typeName, DataMode mode) {
+  public static MajorType getMajorTypeFromName(final String typeName, final DataMode mode) {
     return withMode(getMinorTypeFromName(typeName), mode);
   }
 
-  public static String getNameOfMinorType(MinorType type) {
+  public static String getNameOfMinorType(final MinorType type) {
     switch (type) {
       case BIT:
         return "bool";
@@ -440,7 +440,7 @@ public class Types {
     }
   }
 
-  public static String toString(MajorType type) {
+  public static String toString(final MajorType type) {
     return type != null ? "MajorType[" + TextFormat.shortDebugString(type) + "]" : "null";
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/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 15ef197..6eff72f 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
@@ -146,7 +146,7 @@ public class MongoRecordReader extends AbstractRecordReader {
   @Override
   public void setup(OutputMutator output) throws ExecutionSetupException {
     this.writer = new VectorContainerWriter(output);
-    this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), Lists.newArrayList(getColumns()), enableAllTextMode);
+    this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), Lists.newArrayList(getColumns()), enableAllTextMode, false);
     logger.info("Filters Applied : " + filters);
     logger.info("Fields Selected :" + fields);
     cursor = collection.find(filters, fields);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/BaseWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/BaseWriter.java b/exec/java-exec/src/main/codegen/templates/BaseWriter.java
index 2ce4c3c..ada410d 100644
--- a/exec/java-exec/src/main/codegen/templates/BaseWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/BaseWriter.java
@@ -30,15 +30,11 @@ package org.apache.drill.exec.vector.complex.writer;
 @SuppressWarnings("unused")
 public interface BaseWriter extends Positionable{
   FieldWriter getParent();
-  boolean ok();
-  WriteState getState();
   int getValueCapacity();
-  void resetState();
 
   public interface MapWriter extends BaseWriter{
 
     MaterializedField getField();
-    void checkValueCapacity();
 
     <#list vv.types as type><#list type.minor as minor>
     <#assign lowerName = minor.class?uncap_first />
@@ -83,7 +79,6 @@ public interface BaseWriter extends Positionable{
     void copyReader(FieldReader reader);
     MapWriter rootAsMap();
     ListWriter rootAsList();
-    boolean ok();
     
     public void setPosition(int index);
     public void setValueCount(int count);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java b/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
index cd69d0d..fa1dac4 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexReaders.java
@@ -79,17 +79,13 @@ public class ${nullMode}${name}ReaderImpl extends AbstractFieldReader {
   <#if mode == "Repeated">
 
   public void copyAsValue(${minor.class?cap_first}Writer writer){
-    if (writer.ok()) {
-      Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer;
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer;
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void copyAsField(String name, MapWriter writer){
-    if (writer.ok()) {
-      Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl)  writer.list(name).${lowerName}();
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl)  writer.list(name).${lowerName}();
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public int size(){
@@ -115,17 +111,13 @@ public class ${nullMode}${name}ReaderImpl extends AbstractFieldReader {
   <#else>
   
   public void copyAsValue(${minor.class?cap_first}Writer writer){
-    if (writer.ok()) {
-      ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer;
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer;
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void copyAsField(String name, MapWriter writer){
-    if (writer.ok()) {
-      ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name);
-      impl.vector.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name);
+    impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
   public void read(Nullable${minor.class?cap_first}Holder h){

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
index 576fd83..d40dfd2 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexWriters.java
@@ -61,12 +61,8 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
     return vector.getValueCapacity();
   }
 
-  public void checkValueCapacity() {
-    inform(vector.getValueCapacity() > idx());
-  }
-
   public void allocate(){
-    inform(vector.allocateNewSafe());
+    vector.allocateNew();
   }
   
   public void clear(){
@@ -77,76 +73,53 @@ public class ${eName}WriterImpl extends AbstractFieldWriter {
     return super.idx();
   }
   
-  protected void inform(boolean ok){
-    super.inform(ok);
-  }
-  
   <#if mode == "Repeated">
 
   public void write(${minor.class?cap_first}Holder h){
-    if(ok()){
-      // update to inform(addSafe) once available for all repeated vector types for holders.
-      mutator.addSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   public void write(Nullable${minor.class?cap_first}Holder h){
-    if(ok()){
-      // update to inform(addSafe) once available for all repeated vector types for holders.
-      mutator.addSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
 
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
-    if(ok()){
-      // update to inform(setSafe) once available for all vector types for holders.
-      mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
+    vector.setCurrentValueCount(idx());
   }
   </#if>
   
   public void setPosition(int idx){
-    if (ok()){
-      super.setPosition(idx);
-      mutator.startNewGroup(idx);
-    }
+    super.setPosition(idx);
+    mutator.startNewGroup(idx);
   }
   
   
   <#else>
   
   public void write(${minor.class}Holder h){
-    if(ok()){
-      mutator.setSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   public void write(Nullable${minor.class}Holder h){
-    if(ok()){
-      mutator.setSafe(idx(), h);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), h);
+    vector.setCurrentValueCount(idx());
   }
   
   <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>){
-    if(ok()){
-      mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
+    vector.setCurrentValueCount(idx());
   }
 
   <#if mode == "Nullable">
   public void writeNull(){
-    if(ok()){
-      mutator.setNull(idx());
-      vector.setCurrentValueCount(idx());
-    }
+    mutator.setNull(idx());
+    vector.setCurrentValueCount(idx());
   }
   </#if>
   </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
index 1d0dc9d..1bea326 100644
--- a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
@@ -16,12 +16,6 @@
  * limitations under the License.
  */
 
-import org.joda.time.DateTimeUtils;
-import parquet.io.api.Binary;
-
-import java.lang.Override;
-import java.lang.RuntimeException;
-
 <@pp.dropOutputFile />
 <@pp.changeOutputFile name="org/apache/drill/exec/store/JSONOutputRecordWriter.java" />
 <#include "/@includes/license.ftl" />
@@ -46,6 +40,8 @@ import io.netty.buffer.ByteBuf;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+
 
 
 import org.apache.drill.common.types.TypeProtos;
@@ -68,7 +64,7 @@ import java.util.Map;
  */
 public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implements RecordWriter {
 
-  protected JsonGenerator gen;
+  protected JsonOutput gen;
 
 <#list vv.types as type>
   <#list type.minor as minor>
@@ -92,12 +88,7 @@ public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implem
 
     @Override
     public void writeField() throws IOException {
-  <#if mode.prefix == "Nullable" >
-    if (!reader.isSet()) {
-      gen.writeNull();
-      return;
-    }
-  <#elseif mode.prefix == "Repeated" >
+  <#if mode.prefix == "Repeated" >
     // empty lists are represented by simply not starting a field, rather than starting one and putting in 0 elements
     if (reader.size() == 0) {
       return;
@@ -106,57 +97,59 @@ public abstract class JSONOutputRecordWriter extends AbstractRecordWriter implem
     for (int i = 0; i < reader.size(); i++) {
   <#else>
   </#if>
-
-  <#if  minor.class == "TinyInt" ||
-        minor.class == "UInt1" ||
-        minor.class == "UInt2" ||
-        minor.class == "SmallInt" ||
-        minor.class == "Int" ||
-        minor.class == "Decimal9" ||
-        minor.class == "Float4" ||
-        minor.class == "BigInt" ||
-        minor.class == "Decimal18" ||
-        minor.class == "UInt8" ||
-        minor.class == "UInt4" ||
-        minor.class == "Float8" ||
-        minor.class == "Decimal28Sparse" ||
-        minor.class == "Decimal28Dense" ||
-        minor.class == "Decimal38Dense" ||
-        minor.class == "Decimal38Sparse">
-    <#if mode.prefix == "Repeated" >
-      gen.writeNumber(reader.read${friendlyType}(i));
-    <#else>
-      gen.writeNumber(reader.read${friendlyType}());
-    </#if>
-  <#elseif minor.class == "Date" ||
-              minor.class == "Time" ||
-              minor.class == "TimeStamp" ||
-              minor.class == "TimeTZ" ||
-              minor.class == "IntervalDay" ||
-              minor.class == "Interval" ||
-              minor.class == "VarChar" ||
-              minor.class == "Var16Char" ||
-              minor.class == "IntervalYear">
-    <#if mode.prefix == "Repeated" >
-              gen.writeString(reader.read${friendlyType}(i).toString());
-    <#else>
-      gen.writeString(reader.read${friendlyType}().toString());
-    </#if>
-  <#elseif
-        minor.class == "Bit">
-      <#if mode.prefix == "Repeated" >
-              gen.writeBoolean(reader.read${friendlyType}(i));
-      <#else>
-      gen.writeBoolean(reader.read${friendlyType}());
-      </#if>
-  <#elseif
-            minor.class == "VarBinary">
-      <#if mode.prefix == "Repeated" >
-              gen.writeBinary(reader.readByteArray(i));
-      <#else>
-      gen.writeBinary(reader.readByteArray());
-      </#if>
+  
+  <#assign typeName = minor.class >
+  
+  <#switch minor.class>
+  <#case "UInt1">
+  <#case "UInt2">
+  <#case "UInt4">
+  <#case "UInt8">
+    <#assign typeName = "unsupported">
+    <#break>
+    
+  <#case "Decimal9">
+  <#case "Decimal18">
+  <#case "Decimal28Sparse">
+  <#case "Decimal28Dense">
+  <#case "Decimal38Dense">
+  <#case "Decimal38Sparse">
+    <#assign typeName = "Decimal">
+    <#break>
+  <#case "Float4">
+    <#assign typeName = "Float">
+    <#break>
+  <#case "Float8">
+    <#assign typeName = "Double">
+    <#break>
+    
+  <#case "IntervalDay">
+  <#case "IntervalYear">
+    <#assign typeName = "Interval">
+    <#break>
+    
+  <#case "Bit">
+    <#assign typeName = "Boolean">
+    <#break>  
+
+  <#case "TimeStamp">
+    <#assign typeName = "Timestamp">
+    <#break>  
+    
+  <#case "VarBinary">
+    <#assign typeName = "Binary">
+    <#break>  
+    
+  </#switch>
+  
+  <#if typeName == "unsupported">
+    throw new UnsupportedOperationException("Unable to currently write ${minor.class} type to JSON.");
+  <#elseif mode.prefix == "Repeated" >
+    gen.write${typeName}(i, reader);
+  <#else>
+    gen.write${typeName}(reader);
   </#if>
+
   <#if mode.prefix == "Repeated">
     }
       gen.writeEndArray();

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/ListWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ListWriters.java b/exec/java-exec/src/main/codegen/templates/ListWriters.java
index ff4c3d8..29708d7 100644
--- a/exec/java-exec/src/main/codegen/templates/ListWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/ListWriters.java
@@ -46,7 +46,7 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   protected final ${containerClass} container;
   private Mode mode = Mode.INIT;
   private FieldWriter writer;
-  protected ValueVector innerVector;
+  protected RepeatedVector innerVector;
   
   <#if mode == "Repeated">private int currentChildIndex = 0;</#if>
   public ${mode}ListWriter(String name, ${containerClass} container, FieldWriter parent){
@@ -62,9 +62,12 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   }
 
   public void allocate(){
-    if(writer != null) writer.allocate();
+    if(writer != null){
+      writer.allocate();
+    }
+    
     <#if mode == "Repeated">
-    inform(container.allocateNewSafe());
+    container.allocateNew();
     </#if>
   }
   
@@ -149,27 +152,27 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
     return container.getField();
   }
 
-  public void checkValueCapacity() {
-    inform(container.getValueCapacity() > idx());
-  }
-
   <#if mode == "Repeated">
+  
   public void start(){
-    if(ok()){
-      checkValueCapacity();
-      if (!ok()) return;
-      // update the repeated vector to state that there is current+1 objects.
-      RepeatedListHolder h = new RepeatedListHolder();
-      container.getAccessor().get(idx(), h);
-      if(h.start >= h.end){
-        container.getMutator().startNewGroup(idx());  
-      }
-      currentChildIndex = container.getMutator().add(idx());
-      if(currentChildIndex == -1){
-        inform(false);
-      }else{
-        if(writer != null) writer.setPosition(currentChildIndex);  
-      }
+    
+    final RepeatedListVector list = (RepeatedListVector) container;
+    final RepeatedListVector.Mutator mutator = list.getMutator();
+    
+    // make sure that the current vector can support the end position of this list.
+    if(container.getValueCapacity() <= idx()){
+      mutator.setValueCount(idx()+1);
+    }
+    
+    // update the repeated vector to state that there is current+1 objects.
+    RepeatedListHolder h = new RepeatedListHolder();
+    list.getAccessor().get(idx(), h);
+    if(h.start >= h.end){
+      mutator.startNewGroup(idx());  
+    }
+    currentChildIndex = container.getMutator().add(idx());
+    if(writer != null){
+      writer.setPosition(currentChildIndex);  
     }
   }
   
@@ -180,6 +183,7 @@ public class ${mode}ListWriter extends AbstractFieldWriter{
   }
   <#else>
   
+  
   public void setPosition(int index){
     super.setPosition(index);
     if(writer != null) writer.setPosition(index);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/codegen/templates/MapWriters.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/MapWriters.java b/exec/java-exec/src/main/codegen/templates/MapWriters.java
index 4dab990..6ee8035 100644
--- a/exec/java-exec/src/main/codegen/templates/MapWriters.java
+++ b/exec/java-exec/src/main/codegen/templates/MapWriters.java
@@ -63,14 +63,6 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
       return container.getField();
   }
 
-  public void checkValueCapacity(){
-    <#if mode == "Repeated">
-    if (container.getValueCapacity() <= idx()) {
-      container.reAlloc();
-    }
-    </#if>
-  }
-
   public MapWriter map(String name){
     FieldWriter writer = fields.get(name);
     if(writer == null){
@@ -85,7 +77,7 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
   }
   
   public void allocate(){
-    inform(container.allocateNewSafe());
+    container.allocateNew();
     for(FieldWriter w : fields.values()){
       w.allocate();
     }
@@ -96,10 +88,8 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
     for(FieldWriter w : fields.values()){
       w.clear();
     }
-    
   }
   
-  
   public ListWriter list(String name){
     FieldWriter writer = fields.get(name);
     if(writer == null){
@@ -113,24 +103,24 @@ public class ${mode}MapWriter extends AbstractFieldWriter{
 
   <#if mode == "Repeated">
   public void start(){
-    if(ok()){
-      checkValueCapacity();
-      if (!ok()) return;
       // update the repeated vector to state that there is current+1 objects.
       
-      RepeatedMapHolder h = new RepeatedMapHolder();
-      container.getAccessor().get(idx(), h);
-      if(h.start >= h.end){
-        container.getMutator().startNewGroup(idx());  
-      }
-      currentChildIndex = container.getMutator().add(idx());
-      if(currentChildIndex == -1){
-        inform(false);
-      }else{
-        for(FieldWriter w: fields.values()){
-          w.setPosition(currentChildIndex);  
-        }
-      }
+    final RepeatedMapHolder h = new RepeatedMapHolder();
+    final RepeatedMapVector map = (RepeatedMapVector) container;
+    final RepeatedMapVector.Mutator mutator = map.getMutator();
+    
+    // make sure that the current vector can support the end position of this list.
+    if(container.getValueCapacity() <= idx()){
+      mutator.setValueCount(idx()+1);
+    }
+
+    map.getAccessor().get(idx(), h);
+    if(h.start >= h.end){
+      container.getMutator().startNewGroup(idx());  
+    }
+    currentChildIndex = container.getMutator().add(idx());
+    for(FieldWriter w: fields.values()){
+      w.setPosition(currentChildIndex);  
     }
   }
   

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/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 bd93206..7d89ac9 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
@@ -107,7 +107,8 @@ public interface ExecConstants {
   public static OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
 
   public static String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
-  public static OptionValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
+  public static BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
+  public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types", true);
 
   /**
    * The column label (for directory levels) in results when querying files in a directory

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index b6e3858..19aa3c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -310,15 +310,12 @@ public class EvaluationVisitor {
             TypeHelper.getWriterInterface(inputContainer.getMinorType(), inputContainer.getMajorType().getMode()));
         JVar writer = generator.declareClassField("writer", writerIFace);
         generator.getSetupBlock().assign(writer, JExpr._new(writerImpl).arg(vv).arg(JExpr._null()));
-        generator.getEvalBlock().add(writer.invoke("resetState"));
         generator.getEvalBlock().add(writer.invoke("setPosition").arg(outIndex));
         String copyMethod = inputContainer.isSingularRepeated() ? "copyAsValueSingle" : "copyAsValue";
         generator.getEvalBlock().add(inputContainer.getHolder().invoke(copyMethod).arg(writer));
         if (e.isSafe()) {
           HoldingContainer outputContainer = generator.declare(Types.REQUIRED_BIT);
-          JConditional ifOut = generator.getEvalBlock()._if(writer.invoke("ok"));
-          ifOut._then().assign(outputContainer.getValue(), JExpr.lit(1));
-          ifOut._else().assign(outputContainer.getValue(), JExpr.lit(0));
+          generator.getEvalBlock().assign(outputContainer.getValue(), JExpr.lit(1));
           return outputContainer;
         }
       } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
index a031bee..98a0b61 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateUtility.java
@@ -18,6 +18,7 @@
 
 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;
@@ -666,4 +667,16 @@ public class DateUtility {
         }
         return timeFormat;
     }
+
+    public static int monthsFromPeriod(Period period){
+      return (period.getYears() * yearsToMonths) + period.getMonths();
+    }
+
+    public static int millisFromPeriod(final Period period){
+      return (period.getHours() * hoursToMillis) +
+      (period.getMinutes() * minutesToMillis) +
+      (period.getSeconds() * secondsToMillis) +
+      (period.getMillis());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
index c828cf4..3be2c9d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertFrom.java
@@ -50,7 +50,7 @@ public class JsonConvertFrom {
     @Output ComplexWriter writer;
 
     public void setup(){
-      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false);
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false);
     }
 
     public void eval(){
@@ -76,7 +76,7 @@ public class JsonConvertFrom {
     @Output ComplexWriter writer;
 
     public void setup(){
-      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false);
+      jsonReader = new org.apache.drill.exec.vector.complex.fn.JsonReader(buffer, false, false);
     }
 
     public void eval(){

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
index ccfae14..1d2292e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/JsonConvertTo.java
@@ -52,7 +52,37 @@ public class JsonConvertTo {
 
       java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
       try {
-        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true);
+        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, true);
+
+        jsonWriter.write(input);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+
+      byte [] bytea = stream.toByteArray();
+
+      out.buffer = buffer = buffer.reallocIfNeeded(bytea.length);
+      out.buffer.setBytes(0, bytea);
+      out.end = bytea.length;
+    }
+  }
+
+  @FunctionTemplate(name = "convert_toSIMPLEJSON", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class ConvertToSimpleJson implements DrillSimpleFunc{
+
+    @Param FieldReader input;
+    @Output VarBinaryHolder out;
+    @Inject DrillBuf buffer;
+
+    public void setup(){
+    }
+
+    public void eval(){
+      out.start = 0;
+
+      java.io.ByteArrayOutputStream stream = new java.io.ByteArrayOutputStream();
+      try {
+        org.apache.drill.exec.vector.complex.fn.JsonWriter jsonWriter = new org.apache.drill.exec.vector.complex.fn.JsonWriter(stream, true, false);
 
         jsonWriter.write(input);
       } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 1633912..a3b5f27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnionExchange;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
 public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
@@ -215,6 +216,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitValues(Values op, X value) throws E {
+    return visitOp(op, value);
+  }
+
+  @Override
   public T visitOp(PhysicalOperator op, X value) throws E{
     throw new UnsupportedOperationException(String.format(
         "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index fde27af..ae6ff60 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnionExchange;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
 /**
@@ -73,6 +74,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitWriter(Writer op, EXTRA value) throws EXCEP;
+  public RETURN visitValues(Values op, EXTRA value) throws EXCEP;
   public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
 
   public RETURN visitHashPartitionSender(HashPartitionSender op, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
new file mode 100644
index 0000000..fbb069b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
@@ -0,0 +1,71 @@
+/**
+ * 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.config;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.Leaf;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Iterators;
+
+public class Values extends AbstractBase implements Leaf {
+
+  @SuppressWarnings("unused")
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Values.class);
+
+  private final JSONOptions content;
+
+  @JsonCreator
+  public Values(@JsonProperty("content") JSONOptions content){
+    this.content = content;
+  }
+
+  public JSONOptions getContent(){
+    return content;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitValues(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+    assert children.isEmpty();
+    return this;
+  }
+
+  @Override
+  public int getOperatorType() {
+    return -1;
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
index c5d3d93..96209a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/flatten/FlattenTemplate.java
@@ -37,6 +37,8 @@ import org.apache.drill.exec.vector.RepeatedVector;
 public abstract class FlattenTemplate implements Flattener {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FlattenTemplate.class);
 
+  private static final int OUTPUT_BATCH_SIZE = 4*1024;
+
   private ImmutableList<TransferPair> transfers;
   private SelectionVector2 vector2;
   private SelectionVector4 vector4;
@@ -86,9 +88,10 @@ public abstract class FlattenTemplate implements Flattener {
           for ( ; groupIndex < groupCount; groupIndex++) {
             currGroupSize = accessor.getGroupSizeAtIndex(groupIndex);
             for ( ; childIndexWithinCurrGroup < currGroupSize; childIndexWithinCurrGroup++) {
-              if (!doEval(groupIndex, firstOutputIndex)) {
+              if (firstOutputIndex == OUTPUT_BATCH_SIZE) {
                 break outer;
               }
+              doEval(groupIndex, firstOutputIndex);
               firstOutputIndex++;
               childIndex++;
             }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
new file mode 100644
index 0000000..d526a84
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/values/ValuesBatchCreator.java
@@ -0,0 +1,44 @@
+/**
+ * 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.values;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Values;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.easy.json.JSONRecordReader;
+
+import com.google.common.collect.Iterators;
+
+public class ValuesBatchCreator implements BatchCreator<Values> {
+  @Override
+  public RecordBatch getBatch(FragmentContext context, Values config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    assert children.isEmpty();
+
+    JSONRecordReader reader = new JSONRecordReader(context, config.getContent().asNode(), null, Collections.singletonList(SchemaPath.getSimplePath("*")));
+    return new ScanBatch(config, context, Iterators.singletonIterator((RecordReader) reader));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 5dd6ed5..b96ff3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -61,7 +61,7 @@ import com.google.common.collect.Lists;
  * Utilities for Drill's planner.
  */
 public class DrillOptiq {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOptiq.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOptiq.class);
 
   /**
    * Converts a tree of {@link RexNode} operators into a scalar expression in Drill syntax.
@@ -499,7 +499,7 @@ public class DrillOptiq {
     return new TypedNullConstant(Types.optional(type));
   }
 
-  private static boolean isLiteralNull(RexLiteral literal) {
+  public static boolean isLiteralNull(RexLiteral literal) {
     return literal.getTypeName().getName().equals("NULL");
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
index 92a804e..84a0b51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
@@ -39,6 +39,7 @@ import org.apache.drill.exec.planner.physical.ScreenPrule;
 import org.apache.drill.exec.planner.physical.SortConvertPrule;
 import org.apache.drill.exec.planner.physical.SortPrule;
 import org.apache.drill.exec.planner.physical.StreamAggPrule;
+import org.apache.drill.exec.planner.physical.ValuesPrule;
 import org.apache.drill.exec.planner.physical.WindowPrule;
 import org.apache.drill.exec.planner.physical.UnionAllPrule;
 import org.apache.drill.exec.planner.physical.WriterPrule;
@@ -141,6 +142,7 @@ public class DrillRuleSets {
       DrillMergeProjectRule.getInstance(true, RelFactories.DEFAULT_PROJECT_FACTORY, context.getFunctionRegistry()),
       RemoveDistinctAggregateRule.INSTANCE, //
       // ReduceAggregatesRule.INSTANCE, // replaced by DrillReduceAggregatesRule
+      DrillValuesRule.INSTANCE,
 
       /*
       Projection push-down related rules
@@ -206,6 +208,7 @@ public class DrillRuleSets {
     ruleList.add(WindowPrule.INSTANCE);
     ruleList.add(PushLimitToTopN.INSTANCE);
     ruleList.add(UnionAllPrule.INSTANCE);
+    ruleList.add(ValuesPrule.INSTANCE);
 
     // ruleList.add(UnionDistinctPrule.INSTANCE);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
index a3551e7..4fbc4d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
@@ -17,41 +17,276 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import static org.apache.drill.exec.planner.logical.DrillOptiq.isLiteralNull;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.GregorianCalendar;
 import java.util.List;
 
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.Values;
+import org.apache.drill.exec.vector.complex.fn.ExtendedJsonOutput;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+import org.eigenbase.rel.AbstractRelNode;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.ValuesRelBase;
+import org.eigenbase.rel.RelWriter;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeField;
 import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.sql.SqlExplainLevel;
+import org.eigenbase.sql.type.SqlTypeUtil;
+import org.eigenbase.util.NlsString;
+import org.eigenbase.util.Pair;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+
+import com.fasterxml.jackson.core.JsonLocation;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.util.TokenBuffer;
+import com.google.common.base.Functions;
 
 /**
  * Values implemented in Drill.
  */
-public class DrillValuesRel extends ValuesRelBase implements DrillRel {
+public class DrillValuesRel extends AbstractRelNode implements DrillRel {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillValuesRel.class);
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+
+  private static final long MILLIS_IN_DAY = 1000*60*60*24;
+
+  private final JSONOptions options;
+  private final double rowCount;
+
   protected DrillValuesRel(RelOptCluster cluster, RelDataType rowType, List<List<RexLiteral>> tuples, RelTraitSet traits) {
-    super(cluster, rowType, tuples, traits);
+    super(cluster, traits);
     assert getConvention() == DRILL_LOGICAL;
+    verifyRowType(tuples, rowType);
+
+    this.rowType = rowType;
+    this.rowCount = tuples.size();
+
+    try{
+      this.options = new JSONOptions(convertToJsonNode(rowType, tuples), JsonLocation.NA);
+    }catch(IOException e){
+      throw new DrillRuntimeException("Failure while attempting to encode ValuesRel in JSON.", e);
+    }
+
+  }
+
+  private DrillValuesRel(RelOptCluster cluster, RelDataType rowType, RelTraitSet traits, JSONOptions options, double rowCount){
+    super(cluster, traits);
+    this.options = options;
+    this.rowCount = rowCount;
+    this.rowType = rowType;
+  }
+
+  private static void verifyRowType(final List<List<RexLiteral>> tuples, RelDataType rowType){
+      for (List<RexLiteral> tuple : tuples) {
+        assert (tuple.size() == rowType.getFieldCount());
+
+        for (Pair<RexLiteral, RelDataTypeField> pair : Pair.zip(tuple, rowType.getFieldList())) {
+          RexLiteral literal = (RexLiteral) pair.left;
+          RelDataType fieldType = ((RelDataTypeField) pair.right).getType();
+
+          if ((!(RexLiteral.isNullLiteral(literal)))
+              && (!(SqlTypeUtil.canAssignFrom(fieldType, literal.getType())))) {
+            throw new AssertionError("to " + fieldType + " from " + literal);
+          }
+        }
+      }
+
+  }
+
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return planner.getCostFactory().makeCost(this.rowCount, 1.0d, 0.0d);
   }
 
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     assert inputs.isEmpty();
-    return new DrillValuesRel(getCluster(), rowType, tuples, traitSet);
+    return new DrillValuesRel(getCluster(), rowType, traitSet, options, rowCount);
   }
 
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
+  public LogicalOperator implement(DrillImplementor implementor) {
+      return Values.builder()
+          .content(options.asNode())
+          .build();
   }
 
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    // Update when https://issues.apache.org/jira/browse/DRILL-57 fixed
-    throw new UnsupportedOperationException();
+  public JSONOptions getTuplesAsJsonOptions() throws IOException {
+    return options;
+  }
+
+  public double getRows() {
+    return rowCount;
+  }
+
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .itemIf("type", this.rowType, pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES)
+        .itemIf("type", this.rowType.getFieldList(), pw.nest())
+        .itemIf("tuplesCount", rowCount, pw.getDetailLevel() != SqlExplainLevel.ALL_ATTRIBUTES)
+        .itemIf("tuples", options.asNode(), pw.getDetailLevel() == SqlExplainLevel.ALL_ATTRIBUTES);
+  }
+
+  private static JsonNode convertToJsonNode(RelDataType rowType, List<List<RexLiteral>> tuples) throws IOException{
+    TokenBuffer out = new TokenBuffer(MAPPER.getFactory().getCodec(), false);
+    JsonOutput json = new ExtendedJsonOutput(out);
+    json.writeStartArray();
+    String[] fields = rowType.getFieldNames().toArray(new String[rowType.getFieldCount()]);
+
+    for(List<RexLiteral> row : tuples){
+      json.writeStartObject();
+      int i =0;
+      for(RexLiteral field : row){
+        json.writeFieldName(fields[i]);
+        writeLiteral(field, json);
+        i++;
+      }
+      json.writeEndObject();
+    }
+    json.writeEndArray();
+    json.flush();
+    return out.asParser().readValueAsTree();
+  }
+
+
+  private static void writeLiteral(RexLiteral literal, JsonOutput out) throws IOException{
+
+    switch(literal.getType().getSqlTypeName()){
+    case BIGINT:
+      if (isLiteralNull(literal)) {
+        out.writeBigIntNull();
+      }else{
+        out.writeBigInt((((BigDecimal) literal.getValue()).setScale(0, BigDecimal.ROUND_HALF_UP)).longValue());
+      }
+      return;
+
+    case BOOLEAN:
+      if (isLiteralNull(literal)) {
+        out.writeBooleanNull();
+      }else{
+        out.writeBoolean((Boolean) literal.getValue());
+      }
+      return;
+
+    case CHAR:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar(((NlsString)literal.getValue()).getValue());
+      }
+      return ;
+
+    case DOUBLE:
+      if (isLiteralNull(literal)){
+        out.writeDoubleNull();
+      }else{
+        out.writeDouble(((BigDecimal) literal.getValue()).doubleValue());
+      }
+      return;
+
+    case FLOAT:
+      if (isLiteralNull(literal)) {
+        out.writeFloatNull();
+      }else{
+        out.writeFloat(((BigDecimal) literal.getValue()).floatValue());
+      }
+      return;
+
+    case INTEGER:
+      if (isLiteralNull(literal)) {
+        out.writeIntNull();
+      }else{
+        out.writeInt((((BigDecimal) literal.getValue()).setScale(0, BigDecimal.ROUND_HALF_UP)).intValue());
+      }
+      return;
+
+    case DECIMAL:
+      if (isLiteralNull(literal)) {
+        out.writeDoubleNull();
+      }else{
+        out.writeDouble(((BigDecimal) literal.getValue()).doubleValue());
+      }
+      logger.warn("Converting exact decimal into approximate decimal.  Should be fixed once decimal is implemented.");
+      return;
+
+    case VARCHAR:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar( ((NlsString)literal.getValue()).getValue());
+      }
+      return;
+
+    case SYMBOL:
+      if (isLiteralNull(literal)) {
+        out.writeVarcharNull();
+      }else{
+        out.writeVarChar(literal.getValue().toString());
+      }
+      return;
+
+    case DATE:
+      if (isLiteralNull(literal)) {
+        out.writeDateNull();
+      }else{
+        out.writeDate(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case TIME:
+      if (isLiteralNull(literal)) {
+        out.writeTimeNull();
+      }else{
+        out.writeTime(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case TIMESTAMP:
+      if (isLiteralNull(literal)) {
+        out.writeTimestampNull();
+      }else{
+        out.writeTimestamp(new DateTime((GregorianCalendar)literal.getValue()));
+      }
+      return;
+
+    case INTERVAL_YEAR_MONTH:
+      if (isLiteralNull(literal)) {
+        out.writeIntervalNull();
+      }else{
+        int months = ((BigDecimal) (literal.getValue())).intValue();
+        out.writeInterval(new Period().plusMonths(months));
+      }
+      return;
+
+    case INTERVAL_DAY_TIME:
+      if (isLiteralNull(literal)) {
+        out.writeIntervalNull();
+      }else{
+        long millis = ((BigDecimal) (literal.getValue())).longValue();
+        int days = (int) (millis/MILLIS_IN_DAY);
+        millis = millis - (days * MILLIS_IN_DAY);
+        out.writeInterval(new Period().plusDays(days).plusMillis( (int) millis));
+      }
+      return;
+
+    case NULL:
+      out.writeUntypedNull();
+      return;
+
+    case ANY:
+    default:
+      throw new UnsupportedOperationException(String.format("Unable to convert the value of %s and type %s to a Drill constant expression.", literal, literal.getType().getSqlTypeName()));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
index 59c65f9..1178ebd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ScanFieldDeterminer.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.data.Constant;
+import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.GroupingAggregate;
 import org.apache.drill.common.logical.data.Join;
@@ -146,7 +146,7 @@ public class ScanFieldDeterminer extends AbstractLogicalVisitor<Void, ScanFieldD
   }
 
   @Override
-  public Void visitConstant(Constant constant, FieldList value) {
+  public Void visitValues(Values constant, FieldList value) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
new file mode 100644
index 0000000..f896345
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.exec.physical.config.Values;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.AbstractRelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+
+import com.google.common.collect.Iterators;
+
+public class ValuesPrel extends AbstractRelNode implements Prel {
+
+  @SuppressWarnings("unused")
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValuesPrel.class);
+
+  private JSONOptions content;
+
+  public ValuesPrel(RelOptCluster cluster, RelTraitSet traitSet, RelDataType rowType, JSONOptions content) {
+    super(cluster, traitSet);
+    this.rowType = rowType;
+    this.content = content;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public Values getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    return new Values(content);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public SelectionVectorMode[] getSupportedEncodings() {
+    return SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
new file mode 100644
index 0000000..dbe45ca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.planner.logical.DrillValuesRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+
+public class ValuesPrule extends RelOptRule {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValuesPrule.class);
+
+  public static final ValuesPrule INSTANCE = new ValuesPrule();
+
+  private ValuesPrule(){
+    super(RelOptHelper.any(DrillValuesRel.class), "Prel.ValuesPrule");
+  }
+
+  @Override
+  public void onMatch(final RelOptRuleCall call) {
+    final DrillValuesRel rel = (DrillValuesRel) call.rel(0);
+    try{
+      call.transformTo(new ValuesPrel(rel.getCluster(), rel.getTraitSet().plus(Prel.DRILL_PHYSICAL), rel.getRowType(), rel.getTuplesAsJsonOptions()));
+    }catch(IOException e){
+      logger.warn("Failure while converting JSONOptions.", e);
+    }
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
new file mode 100644
index 0000000..e15eddf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/BaseOptionManager.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.server.options;
+
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
+
+abstract class BaseOptionManager implements OptionManager {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseOptionManager.class);
+
+
+  private OptionValue getOptionSafe(OptionValidator validator){
+    OptionValue value = getOption(validator.getOptionName());
+    if(value == null){
+      throw new IllegalArgumentException(String.format("Unknown value for boolean option `%s`.", validator.getOptionName()));
+    }
+    return value;
+  }
+
+  @Override
+  public boolean getOption(BooleanValidator validator) {
+    return getOptionSafe(validator).bool_val;
+  }
+
+  @Override
+  public double getOption(DoubleValidator validator) {
+    return getOptionSafe(validator).float_val;
+  }
+
+  @Override
+  public long getOption(LongValidator validator) {
+    return getOptionSafe(validator).num_val;
+  }
+
+  @Override
+  public String getOption(StringValidator validator) {
+    return getOptionSafe(validator).string_val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
index 45d393c..4e90616 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/FallbackOptionManager.java
@@ -24,7 +24,7 @@ import org.eigenbase.sql.SqlLiteral;
 
 import com.google.common.collect.Iterables;
 
-public abstract class FallbackOptionManager implements OptionManager {
+public abstract class FallbackOptionManager extends BaseOptionManager {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FallbackOptionManager.class);
 
   protected final OptionManager fallback;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
index 4ffe9a3..0b8811a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionManager.java
@@ -18,6 +18,10 @@
 package org.apache.drill.exec.server.options;
 
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.eigenbase.sql.SqlLiteral;
 
 public interface OptionManager extends Iterable<OptionValue> {
@@ -28,6 +32,11 @@ public interface OptionManager extends Iterable<OptionValue> {
   public OptionManager getSystemManager();
   public OptionList getOptionList();
 
+  public boolean getOption(BooleanValidator validator);
+  public double getOption(DoubleValidator validator);
+  public long getOption(LongValidator validator);
+  public String getOption(StringValidator validator);
+
   public interface OptionAdmin {
     public void registerOptionType(OptionValidator validator);
     public void validate(OptionValue v) throws SetOptionException;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 1a8559e..4471d4f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -30,6 +30,10 @@ import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.QueryClassLoader;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
+import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.store.sys.PStore;
 import org.apache.drill.exec.store.sys.PStoreConfig;
 import org.apache.drill.exec.store.sys.PStoreProvider;
@@ -37,7 +41,7 @@ import org.eigenbase.sql.SqlLiteral;
 
 import com.google.common.collect.Maps;
 
-public class SystemOptionManager implements OptionManager {
+public class SystemOptionManager extends BaseOptionManager {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemOptionManager.class);
 
   private static final OptionValidator[] VALIDATORS = {
@@ -72,6 +76,7 @@ public class SystemOptionManager implements OptionManager {
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,
+      ExecConstants.JSON_EXTENDED_TYPES,
       ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR,
       ExecConstants.MONGO_READER_ALL_TEXT_MODE_VALIDATOR,
       ExecConstants.SLICE_TARGET_OPTION,


[5/6] drill git commit: DRILL-2695: Add Support for large in conditions through the use of the Values operator. Update JSON reader to support reading Extended JSON. Update JSON writer to support writing extended JSON data. Update JSON reader to automatic

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index 6cf1ce5..2e65466 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -25,6 +25,7 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
@@ -75,6 +76,7 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
     options.put(FileSystem.FS_DEFAULT_NAME_KEY, ((FileSystemConfig)writer.getStorageConfig()).connection);
 
     options.put("extension", "json");
+    options.put("extended", Boolean.toString(context.getOptions().getOption(ExecConstants.JSON_EXTENDED_TYPES)));
 
     RecordWriter recordWriter = new JsonRecordWriter();
     recordWriter.init(options);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
index 91e0b21..4c44dbd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -33,76 +32,134 @@ import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.easy.json.JsonProcessor.ReadState;
 import org.apache.drill.exec.store.easy.json.reader.CountingJsonReader;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.complex.fn.JsonReader;
-import org.apache.drill.exec.store.easy.json.JsonProcessor.ReadState;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
-import com.fasterxml.jackson.core.JsonParseException;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
 public class JSONRecordReader extends AbstractRecordReader {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader.class);
 
-  private OutputMutator mutator;
   private VectorContainerWriter writer;
+
+  // Data we're consuming
   private Path hadoopPath;
+  private JsonNode embeddedContent;
   private InputStream stream;
-  private DrillFileSystem fileSystem;
+  private final DrillFileSystem fileSystem;
   private JsonProcessor jsonReader;
   private int recordCount;
-  private FragmentContext fragmentContext;
+  private final FragmentContext fragmentContext;
   private OperatorContext operatorContext;
-  private boolean enableAllTextMode;
+  private final boolean enableAllTextMode;
+
+  /**
+   * Create a JSON Record Reader that uses a file based input stream.
+   * @param fragmentContext
+   * @param inputPath
+   * @param fileSystem
+   * @param columns
+   * @throws OutOfMemoryException
+   */
+  public JSONRecordReader(final FragmentContext fragmentContext, final String inputPath, final DrillFileSystem fileSystem,
+      final List<SchemaPath> columns) throws OutOfMemoryException {
+    this(fragmentContext, inputPath, null, fileSystem, columns);
+  }
+
+  /**
+   * Create a new JSON Record Reader that uses a in memory materialized JSON stream.
+   * @param fragmentContext
+   * @param inputPath
+   * @param fileSystem
+   * @param columns
+   * @throws OutOfMemoryException
+   */
+  public JSONRecordReader(final FragmentContext fragmentContext, final JsonNode embeddedContent, final DrillFileSystem fileSystem,
+      final List<SchemaPath> columns) throws OutOfMemoryException {
+    this(fragmentContext, null, embeddedContent, fileSystem, columns);
+  }
+
+  private JSONRecordReader(final FragmentContext fragmentContext, final String inputPath, final JsonNode embeddedContent, final DrillFileSystem fileSystem,
+                          final List<SchemaPath> columns) throws OutOfMemoryException {
+
+    Preconditions.checkArgument(
+        (inputPath == null && embeddedContent != null) ||
+        (inputPath != null && embeddedContent == null),
+        "One of inputPath or embeddedContent must be set but not both."
+        );
+
+    if(inputPath != null){
+      this.hadoopPath = new Path(inputPath);
+    }else{
+      this.embeddedContent = embeddedContent;
+    }
 
-  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, DrillFileSystem fileSystem,
-                          List<SchemaPath> columns) throws OutOfMemoryException {
-    this.hadoopPath = new Path(inputPath);
     this.fileSystem = fileSystem;
     this.fragmentContext = fragmentContext;
-    this.enableAllTextMode = fragmentContext.getOptions().getOption(ExecConstants.JSON_ALL_TEXT_MODE).bool_val;
+
+    // only enable all text mode if we aren't using embedded content mode.
+    this.enableAllTextMode = embeddedContent == null && fragmentContext.getOptions().getOption(ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR);
     setColumns(columns);
   }
 
   @Override
-  public void setup(OutputMutator output) throws ExecutionSetupException {
+  public void setup(final OutputMutator output) throws ExecutionSetupException {
     try{
-      CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration());
-      CompressionCodec codec = factory.getCodec(hadoopPath); // infers from file ext.
-      if (codec != null) {
-        this.stream = codec.createInputStream(fileSystem.open(hadoopPath));
-      } else {
-        this.stream = fileSystem.open(hadoopPath);
-      }
+      setupData();
       this.writer = new VectorContainerWriter(output);
-      this.mutator = output;
       if (isSkipQuery()) {
         this.jsonReader = new CountingJsonReader(fragmentContext.getManagedBuffer());
       } else {
-        this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), ImmutableList.copyOf(getColumns()), enableAllTextMode);
+        this.jsonReader = new JsonReader(fragmentContext.getManagedBuffer(), ImmutableList.copyOf(getColumns()), enableAllTextMode, true);
       }
-      this.jsonReader.setSource(stream);
-    }catch(Exception e){
+      setupParser();
+    }catch(final Exception e){
       handleAndRaise("Failure reading JSON file.", e);
     }
   }
 
-  protected void handleAndRaise(String suffix, Exception e) throws DrillUserException {
+  private void setupData() throws IOException{
+    if(hadoopPath != null){
+      final CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration());
+      final CompressionCodec codec = factory.getCodec(hadoopPath); // infers from file ext.
+      if (codec != null) {
+        this.stream = codec.createInputStream(fileSystem.open(hadoopPath));
+      } else {
+        this.stream = fileSystem.open(hadoopPath);
+      }
+    }
+  }
+
+  private void setupParser() throws IOException{
+    if(hadoopPath != null){
+      jsonReader.setSource(stream);
+    }else{
+      jsonReader.setSource(embeddedContent);
+    }
+  }
+
+  protected void handleAndRaise(final String suffix, final Exception e) throws DrillUserException {
 
     String message = e.getMessage();
     int columnNr = -1;
 
     if (e instanceof JsonParseException) {
-      JsonParseException ex = (JsonParseException) e;
+      final JsonParseException ex = (JsonParseException) e;
       message = ex.getOriginalMessage();
       columnNr = ex.getLocation().getColumnNr();
     }
 
-    DrillUserException.Builder builder = new DrillUserException.Builder(ErrorType.DATA_READ, e, "%s - %s", suffix, message);
+    final DrillUserException.Builder builder = new DrillUserException.Builder(ErrorType.DATA_READ, e, "%s - %s", suffix, message);
 
     // add context information
     builder.add("Filename: " + hadoopPath.toUri().getPath());
@@ -119,7 +176,7 @@ public class JSONRecordReader extends AbstractRecordReader {
     return operatorContext;
   }
 
-  public void setOperatorContext(OperatorContext operatorContext) {
+  public void setOperatorContext(final OperatorContext operatorContext) {
     this.operatorContext = operatorContext;
   }
 
@@ -151,15 +208,13 @@ public class JSONRecordReader extends AbstractRecordReader {
       writer.setValueCount(recordCount);
 //      p.stop();
 //      System.out.println(String.format("Wrote %d records in %dms.", recordCount, p.elapsed(TimeUnit.MILLISECONDS)));
-      if (recordCount == 0 && write == ReadState.WRITE_FAILURE) {
-        throw new IOException("Record was too large to copy into vector.");
-      }
+
 
       return recordCount;
 
-    } catch (JsonParseException e) {
+    } catch (final JsonParseException e) {
       handleAndRaise("Error parsing JSON.", e);
-    } catch (IOException e) {
+    } catch (final IOException e) {
       handleAndRaise("Error reading JSON.", e);
     }
     // this is never reached
@@ -169,8 +224,10 @@ public class JSONRecordReader extends AbstractRecordReader {
   @Override
   public void cleanup() {
     try {
-      stream.close();
-    } catch (IOException e) {
+      if(stream != null){
+        stream.close();
+      }
+    } catch (final IOException e) {
       logger.warn("Failure while closing stream.", e);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java
index ce6017b..b310818 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonProcessor.java
@@ -22,10 +22,11 @@ import java.io.InputStream;
 
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
+import com.fasterxml.jackson.databind.JsonNode;
+
 public interface JsonProcessor {
 
   public static enum ReadState {
-    WRITE_FAILURE,
     END_OF_STREAM,
     WRITE_SUCCEED
   }
@@ -33,6 +34,7 @@ public interface JsonProcessor {
   ReadState write(BaseWriter.ComplexWriter writer) throws IOException;
 
   void setSource(InputStream is) throws IOException;
+  void setSource(JsonNode node);
 
   void ensureAtLeastOneField(BaseWriter.ComplexWriter writer);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
index 76c4ace..a43a4a0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
@@ -26,6 +26,8 @@ import org.apache.drill.exec.store.EventBasedRecordWriter;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.store.JSONOutputRecordWriter;
 import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.vector.complex.fn.BasicJsonOutput;
+import org.apache.drill.exec.vector.complex.fn.ExtendedJsonOutput;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -33,17 +35,19 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
 import com.google.common.collect.Lists;
 
 public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWriter {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonRecordWriter.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonRecordWriter.class);
 
   private String location;
   private String prefix;
 
   private String fieldDelimiter;
   private String extension;
+  private boolean useExtendedOutput;
 
   private int index;
   private FileSystem fs = null;
@@ -63,6 +67,7 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
     this.prefix = writerOptions.get("prefix");
     this.fieldDelimiter = writerOptions.get("separator");
     this.extension = writerOptions.get("extension");
+    this.useExtendedOutput = Boolean.parseBoolean(writerOptions.get("extended"));
 
     Configuration conf = new Configuration();
     conf.set(FileSystem.FS_DEFAULT_NAME_KEY, writerOptions.get(FileSystem.FS_DEFAULT_NAME_KEY));
@@ -71,7 +76,12 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
     Path fileName = new Path(location, prefix + "_" + index + "." + extension);
     try {
       stream = fs.create(fileName);
-      gen = factory.createGenerator(stream).useDefaultPrettyPrinter();
+      JsonGenerator generator = factory.createGenerator(stream).useDefaultPrettyPrinter();
+      if(useExtendedOutput){
+        gen = new ExtendedJsonOutput(generator);
+      }else{
+        gen = new BasicJsonOutput(generator);
+      }
       logger.debug("Created file: {}", fileName);
     } catch (IOException ex) {
       logger.error("Unable to create file: " + fileName, ex);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/RewindableUtf8Reader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/RewindableUtf8Reader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/RewindableUtf8Reader.java
deleted file mode 100644
index b9075de..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/RewindableUtf8Reader.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.easy.json;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.hadoop.fs.Seekable;
-
-import com.fasterxml.jackson.core.JsonStreamContextExposer;
-import com.fasterxml.jackson.core.JsonToken;
-import com.fasterxml.jackson.core.JsonParser.Feature;
-import com.fasterxml.jackson.core.io.IOContext;
-import com.fasterxml.jackson.core.json.JsonReadContext;
-import com.fasterxml.jackson.core.json.JsonReadContextExposer;
-import com.fasterxml.jackson.core.json.UTF8StreamJsonParser;
-import com.fasterxml.jackson.core.sym.BytesToNameCanonicalizer;
-
-/**
- * An extended version of Jaskon's UTF8StreamJsonParser that supports rewind the stream to the previous record.
- */
-public class RewindableUtf8Reader<T extends InputStream & Seekable> extends UTF8StreamJsonParser {
-
-  private T in;
-
-  /**
-   * Index of character after last available one in the buffer.
-   */
-  private long markFilePos;
-  private int markInputPtr;
-  private int markInputEnd;
-  private long markInputProcessed;
-  private int markInputRow;
-  private int markInputRowStart;
-  private long markInputTotal;
-  private int markTokenInputRow;
-  private int markTokenInputCol;
-  private JsonToken markToken;
-  private JsonToken markLastToken;
-  private JsonReadContext markContext;
-  private JsonReadContext rootContext;
-
-  private int type;
-  private int lineNr;
-  private int colNr;
-
-  private boolean closed = false;
-
-  public RewindableUtf8Reader(IOContext ctxt, int features, BytesToNameCanonicalizer sym, byte[] inputBuffer) {
-    super(ctxt, features, null, null, sym, inputBuffer, 0, 0, true);
-    this.rootContext = this._parsingContext;
-  }
-
-  public void mark() throws IOException{
-    this.markFilePos = this.in.getPos();
-    this.markInputPtr = this._inputPtr;
-    this.markInputEnd = this._inputEnd;
-    this.markInputProcessed = this._currInputProcessed;
-    this.markInputRow = this._currInputRow;
-    this.markInputRowStart = this._currInputRowStart;
-    this.markInputTotal = this._tokenInputTotal;
-    this.markTokenInputCol = this._tokenInputCol;
-    this.markTokenInputRow = this._tokenInputRow;
-    this.markToken = this._currToken;
-    this.markLastToken = this._lastClearedToken;
-    this.markContext = this._parsingContext;
-    this.type = JsonStreamContextExposer.getType(markContext);
-    this.lineNr = JsonReadContextExposer.getLineNmbr(markContext);
-    this.colNr = JsonReadContextExposer.getColNmbr(markContext);
-  }
-
-  public void resetToMark() throws IOException{
-    if(markFilePos != in.getPos()){
-      in.seek(markFilePos - _inputBuffer.length);
-      in.read(_inputBuffer, 0, _inputBuffer.length);
-    }
-    this._inputPtr = this.markInputPtr;
-    this._inputEnd = this.markInputEnd;
-    this._currInputProcessed = this.markInputProcessed;
-    this._currInputRow = this.markInputRow;
-    this._currInputRowStart = this.markInputRowStart;
-    this._tokenInputTotal = this.markInputTotal;
-    this._tokenInputCol = this.markTokenInputCol;
-    this._tokenInputRow = this.markTokenInputRow;
-    this._currToken = this.markToken;
-    this._lastClearedToken = this.markLastToken;
-    this._parsingContext = this.markContext;
-    JsonReadContextExposer.reset(markContext, type, lineNr, colNr);
-
-  }
-
-  @Override
-  protected void _closeInput() throws IOException {
-    super._closeInput();
-
-      if (_inputStream != null) {
-          if (_ioContext.isResourceManaged() || isEnabled(Feature.AUTO_CLOSE_SOURCE)) {
-              _inputStream.close();
-          }
-          _inputStream = null;
-      }
-      this.closed = true;
-
-  }
-
-  public void setInputStream(T in) throws IOException{
-    if(this.in != null){
-      in.close();
-    }
-
-    this._inputStream = in;
-    this.in = in;
-    this._parsingContext = rootContext;
-    this._inputPtr = 0;
-    this._inputEnd = 0;
-    this._currInputProcessed = 0;
-    this._currInputRow = 0;
-    this._currInputRowStart = 0;
-    this._tokenInputTotal = 0;
-    this._tokenInputCol = 0;
-    this._tokenInputRow = 0;
-    this._currToken = null;
-    this._lastClearedToken = null;
-    this.closed = false;
-  }
-
-  public boolean hasDataAvailable() throws IOException{
-    return !closed;
-  }
-
-  @Override
-  public String toString() {
-    return "RewindableUtf8Reader [markFilePos=" + markFilePos + ", markInputPtr=" + markInputPtr + ", markInputEnd="
-        + markInputEnd + ", markInputProcessed=" + markInputProcessed + ", markInputRow=" + markInputRow
-        + ", markInputRowStart=" + markInputRowStart + ", markInputTotal=" + markInputTotal + ", markTokenInputRow="
-        + markTokenInputRow + ", markTokenInputCol=" + markTokenInputCol + ", markToken=" + markToken
-        + ", markLastToken=" + markLastToken + ", markContext=" + markContext + ", rootContext=" + rootContext
-        + ", type=" + type + ", lineNr=" + lineNr + ", colNr=" + colNr + "]";
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
index 509798a..718bb09 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
@@ -17,42 +17,40 @@
  */
 package org.apache.drill.exec.store.easy.json.reader;
 
+import io.netty.buffer.DrillBuf;
+
 import java.io.IOException;
 import java.io.InputStream;
 
-import com.fasterxml.jackson.core.JsonFactory;
+import org.apache.drill.exec.store.easy.json.JsonProcessor;
+
 import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.io.IOContext;
-import com.fasterxml.jackson.core.sym.BytesToNameCanonicalizer;
-import com.fasterxml.jackson.core.util.BufferRecycler;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.TreeTraversingParser;
 import com.google.common.base.Preconditions;
-import io.netty.buffer.DrillBuf;
-import org.apache.drill.exec.store.easy.json.JsonProcessor;
-import org.apache.drill.exec.store.easy.json.RewindableUtf8Reader;
 
 public abstract class BaseJsonProcessor implements JsonProcessor {
 
-  protected final RewindableUtf8Reader parser;
+  private static final ObjectMapper MAPPER = new ObjectMapper() //
+    .configure(JsonParser.Feature.ALLOW_COMMENTS, true)
+    .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+
+  protected JsonParser parser;
   protected DrillBuf workBuf;
 
   public BaseJsonProcessor(DrillBuf workBuf) {
     this.workBuf = Preconditions.checkNotNull(workBuf);
-    this.parser = Preconditions.checkNotNull(createParser());
   }
 
-  protected RewindableUtf8Reader createParser() {
-    final BufferRecycler recycler = new BufferRecycler();
-    final IOContext context = new IOContext(recycler, this, false);
-    final int features = JsonParser.Feature.collectDefaults() //
-        | JsonParser.Feature.ALLOW_COMMENTS.getMask() //
-        | JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES.getMask();
-
-    final BytesToNameCanonicalizer can = BytesToNameCanonicalizer.createRoot();
-    return new RewindableUtf8Reader<>(context, features, can.makeChild(JsonFactory.Feature.collectDefaults()), context.allocReadIOBuffer());
+  @Override
+  public void setSource(InputStream is) throws IOException {
+    this.parser = MAPPER.getFactory().createParser(is);
   }
 
   @Override
-  public void setSource(InputStream is) throws IOException {
-    parser.setInputStream(is);
+  public void setSource(JsonNode node) {
+    this.parser = new TreeTraversingParser(node);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/CountingJsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/CountingJsonReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/CountingJsonReader.java
index 1ef71e7..c4ab1ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/CountingJsonReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/CountingJsonReader.java
@@ -24,7 +24,7 @@ import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.store.easy.json.JsonProcessor;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 
-public class CountingJsonReader extends BaseJsonProcessor implements JsonProcessor {
+public class CountingJsonReader extends BaseJsonProcessor {
 
   public CountingJsonReader(DrillBuf workBuf) {
     super(workBuf);
@@ -40,7 +40,7 @@ public class CountingJsonReader extends BaseJsonProcessor implements JsonProcess
     }
     writer.rootAsMap().bit("count").writeBit(1);
     parser.skipChildren();
-    return writer.ok() ? ReadState.WRITE_SUCCEED : ReadState.WRITE_FAILURE;
+    return ReadState.WRITE_SUCCEED;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
index 9f8357b..4d837c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetReader.java
@@ -316,15 +316,6 @@ public class DrillParquetReader extends AbstractRecordReader {
       recordReader.read();
       count++;
       totalRead++;
-      if (count % fillLevelCheckFrequency == 0) {
-        if (getPercentFilled() > fillLevelCheckThreshold) {
-          if(!recordMaterializer.ok()){
-            String message = String.format("The setting for `%s` is too high for your Parquet records. Please set a lower check threshold and retry your query. ", ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD);
-            handleAndRaise(message, new RuntimeException(message));
-          }
-          break;
-        }
-      }
     }
     writer.setValueCount(count);
     // if we have requested columns that were not found in the file fill their vectors with null

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetRecordMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetRecordMaterializer.java
index 720e8be..574df40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetRecordMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetRecordMaterializer.java
@@ -42,10 +42,6 @@ public class DrillParquetRecordMaterializer extends RecordMaterializer<Void> {
     complexWriter.setPosition(position);
   }
 
-  public boolean ok() {
-    return complexWriter.ok();
-  }
-
   @Override
   public Void getCurrentRecord() {
     return null;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVector.java
index 2c2ff54..df4279a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/RepeatedVector.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.vector;
 
-public interface RepeatedVector {
+public interface RepeatedVector extends ValueVector {
   public static final int DEFAULT_REPEAT_PER_RECORD = 4;
 
   public RepeatedFixedWidthVector.RepeatedAccessor getAccessor();

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/WriteState.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/WriteState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/WriteState.java
deleted file mode 100644
index 43dba65..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/WriteState.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.vector.complex;
-
-import org.apache.drill.exec.vector.complex.writer.FieldWriter;
-
-
-public class WriteState {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WriteState.class);
-
-  private FieldWriter failPoint;
-
-  public boolean isFailed(){
-    return failPoint != null;
-  }
-
-  public boolean isOk(){
-    return failPoint == null;
-  }
-
-  public void fail(FieldWriter w){
-    assert failPoint == null;
-    failPoint = w;
-
-//    System.out.println("Fail Point " + failPoint);
-  }
-
-  public void reset(){
-    failPoint = null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
new file mode 100644
index 0000000..364692e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/BasicJsonOutput.java
@@ -0,0 +1,530 @@
+/**
+ * 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.complex.fn;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
+import org.joda.time.format.ISOPeriodFormat;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+
+/**
+ * A JSON output class that generates standard JSON. By default, literals are output such that they can be implicitly
+ * cast.
+ */
+public class BasicJsonOutput implements JsonOutput {
+
+  protected final JsonGenerator gen;
+  private final DateTimeFormatter dateFormatter;
+  private final DateTimeFormatter timeFormatter;
+  private final DateTimeFormatter timestampFormatter;
+
+  public BasicJsonOutput(JsonGenerator gen) {
+    this(gen, DateOutputFormat.SQL);
+  }
+
+  protected BasicJsonOutput(JsonGenerator gen, DateOutputFormat dateOutput) {
+    Preconditions.checkNotNull(dateOutput);
+    Preconditions.checkNotNull(gen);
+
+    this.gen = gen;
+
+    switch (dateOutput) {
+    case SQL: {
+      dateFormatter = DateUtility.formatDate;
+      timeFormatter = DateUtility.formatTime;
+      timestampFormatter = DateUtility.formatTimeStamp;
+      break;
+    }
+    case ISO: {
+      dateFormatter = ISODateTimeFormat.date();
+      timeFormatter = ISODateTimeFormat.time();
+      timestampFormatter = ISODateTimeFormat.dateTime();
+      break;
+    }
+
+    default:
+      throw new UnsupportedOperationException(String.format("Unable to support date output of type %s.", dateOutput));
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    gen.flush();
+  }
+
+  @Override
+  public void writeStartArray() throws IOException {
+    gen.writeStartArray();
+  }
+
+  @Override
+  public void writeEndArray() throws IOException {
+    gen.writeEndArray();
+  }
+
+  @Override
+  public void writeStartObject() throws IOException {
+    gen.writeStartObject();
+  }
+
+  @Override
+  public void writeEndObject() throws IOException {
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeUntypedNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeFieldName(String name) throws IOException {
+    gen.writeFieldName(name);
+  }
+
+  @Override
+  public void writeDecimal(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDecimal(reader.readBigDecimal());
+    } else {
+      writeDecimalNull();
+    }
+  }
+
+  @Override
+  public void writeTinyInt(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTinyInt(reader.readByte());
+    } else {
+      writeTinyIntNull();
+    }
+  }
+
+  @Override
+  public void writeSmallInt(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeSmallInt(reader.readShort());
+    } else {
+      writeSmallIntNull();
+    }
+  }
+
+  @Override
+  public void writeInt(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeInt(reader.readInteger());
+    } else {
+      writeIntNull();
+    }
+  }
+
+  @Override
+  public void writeBigInt(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBigInt(reader.readLong());
+    } else {
+      writeBigIntNull();
+    }
+  }
+
+  @Override
+  public void writeFloat(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeFloat(reader.readFloat());
+    } else {
+      writeFloatNull();
+    }
+  }
+
+  @Override
+  public void writeDouble(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDouble(reader.readDouble());
+    } else {
+      writeDoubleNull();
+    }
+  }
+
+  @Override
+  public void writeVarChar(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeVarChar(reader.readText().toString());
+    } else {
+      writeVarcharNull();
+    }
+  }
+
+  @Override
+  public void writeVar16Char(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeVar16Char(reader.readText().toString());
+    } else {
+      writeVar16charNull();
+    }
+  }
+
+  @Override
+  public void writeBinary(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBinary(reader.readByteArray());
+    } else {
+      writeBinaryNull();
+    }
+  }
+
+  @Override
+  public void writeBoolean(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBoolean(reader.readBoolean());
+    } else {
+      writeBooleanNull();
+    }
+  }
+
+  @Override
+  public void writeDate(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDate(reader.readDateTime());
+    } else {
+      writeDateNull();
+    }
+  }
+
+  @Override
+  public void writeTime(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTime(reader.readDateTime());
+    } else {
+      writeTimeNull();
+    }
+  }
+
+  @Override
+  public void writeTimestamp(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTimestamp(reader.readDateTime());
+    } else {
+      writeTimeNull();
+    }
+  }
+
+  @Override
+  public void writeInterval(FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeInterval(reader.readPeriod());
+    } else {
+      writeIntervalNull();
+    }
+  }
+
+  @Override
+  public void writeTinyInt(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTinyInt(reader.readByte(index));
+    } else {
+      writeTinyIntNull();
+    }
+  }
+
+  @Override
+  public void writeSmallInt(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeSmallInt(reader.readShort(index));
+    } else {
+      writeSmallIntNull();
+    }
+  }
+
+  @Override
+  public void writeInt(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeInt(reader.readInteger(index));
+    } else {
+      writeIntNull();
+    }
+  }
+
+  @Override
+  public void writeBigInt(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBigInt(reader.readLong(index));
+    } else {
+      writeBigIntNull();
+    }
+  }
+
+  @Override
+  public void writeFloat(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeFloat(reader.readFloat(index));
+    } else {
+      writeFloatNull();
+    }
+  }
+
+  @Override
+  public void writeDouble(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDouble(reader.readDouble(index));
+    } else {
+      writeDoubleNull();
+    }
+  }
+
+  @Override
+  public void writeVarChar(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeVarChar(reader.readText(index).toString());
+    } else {
+      writeVarcharNull();
+    }
+  }
+
+  @Override
+  public void writeVar16Char(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeVar16Char(reader.readString(index));
+    } else {
+      writeVar16charNull();
+    }
+  }
+
+  @Override
+  public void writeBinary(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBinary(reader.readByteArray(index));
+    } else {
+      writeBinaryNull();
+    }
+  }
+
+  @Override
+  public void writeBoolean(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeBoolean(reader.readBoolean(index));
+    } else {
+      writeBooleanNull();
+    }
+  }
+
+  @Override
+  public void writeDate(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDate(reader.readDateTime(index));
+    } else {
+      writeDateNull();
+    }
+  }
+
+  @Override
+  public void writeTime(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTime(reader.readDateTime(index));
+    } else {
+      writeTimeNull();
+    }
+  }
+
+  @Override
+  public void writeTimestamp(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeTimestamp(reader.readDateTime(index));
+    } else {
+      writeTimestampNull();
+    }
+  }
+
+  @Override
+  public void writeInterval(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeInterval(reader.readPeriod(index));
+    } else {
+      writeIntervalNull();
+    }
+  }
+
+  @Override
+  public void writeDecimal(int index, FieldReader reader) throws IOException {
+    if (reader.isSet()) {
+      writeDecimal(reader.readBigDecimal(index));
+    } else {
+      writeDecimalNull();
+    }
+  }
+
+  @Override
+  public void writeDecimal(BigDecimal value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeTinyInt(byte value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeSmallInt(short value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeInt(int value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeBigInt(long value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeFloat(float value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeDouble(double value) throws IOException {
+    gen.writeNumber(value);
+  }
+
+  @Override
+  public void writeVarChar(String value) throws IOException {
+    gen.writeString(value);
+  }
+
+  @Override
+  public void writeVar16Char(String value) throws IOException {
+    gen.writeString(value);
+  }
+
+  @Override
+  public void writeBinary(byte[] value) throws IOException {
+    gen.writeBinary(value);
+  }
+
+  @Override
+  public void writeBoolean(boolean value) throws IOException {
+    gen.writeBoolean(value);
+  }
+
+  @Override
+  public void writeDate(DateTime value) throws IOException {
+    gen.writeString(dateFormatter.print(value.withZone(DateTimeZone.UTC)));
+  }
+
+  @Override
+  public void writeTime(DateTime value) throws IOException {
+    gen.writeString(timeFormatter.print(value.withZone(DateTimeZone.UTC)));
+  }
+
+  @Override
+  public void writeTimestamp(DateTime value) throws IOException {
+    gen.writeString(timestampFormatter.print(value.withZone(DateTimeZone.UTC)));
+  }
+
+  @Override
+  public void writeInterval(Period value) throws IOException {
+    gen.writeString(value.toString(ISOPeriodFormat.standard()));
+  }
+
+  @Override
+  public void writeDecimalNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeTinyIntNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeSmallIntNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeIntNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeBigIntNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeFloatNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeDoubleNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeVarcharNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeVar16charNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeBinaryNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeBooleanNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeDateNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeTimeNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeTimestampNull() throws IOException {
+    gen.writeNull();
+  }
+
+  @Override
+  public void writeIntervalNull() throws IOException {
+    gen.writeNull();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DateOutputFormat.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DateOutputFormat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DateOutputFormat.java
new file mode 100644
index 0000000..fa91b75
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/DateOutputFormat.java
@@ -0,0 +1,37 @@
+/**
+ * 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.complex.fn;
+
+/**
+ * Describes the default date output format to use for JSON. Drill's default behavior for text output formats is to use
+ * a string which can be implicitly casted back to its original type (so the same format as the SQL literal format where
+ * applicable). However, in JSON, we also can use extended types to specifically identify the data type of the output.
+ * In this case, Drill outputs ISO standard formats rather than SQL formats to ensure comaptibility with other systems
+ * (namely MongoDB).
+ */
+public enum DateOutputFormat {
+  /**
+   * The SQL literal format for dates.  This means no timezone in times and a space in between the date and time for timestamp.
+   */
+  SQL,
+
+  /**
+   * The ISO standard format for dates/times.
+   */
+  ISO
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
new file mode 100644
index 0000000..9ac1dd5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedJsonOutput.java
@@ -0,0 +1,183 @@
+/**
+ * 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.complex.fn;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+
+/**
+ * Writes JSON Output that will wrap Binary, Date, Time, Timestamp, Integer, Decimal and Interval types with wrapping
+ * maps for better type resolution upon deserialization.
+ */
+public class ExtendedJsonOutput extends BasicJsonOutput {
+
+  public ExtendedJsonOutput(JsonGenerator gen) {
+    super(gen, DateOutputFormat.ISO);
+  }
+
+  @Override
+  public void writeBigInt(long value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.INTEGER.serialized);
+    super.writeBigInt(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeBinary(byte[] value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.BINARY.serialized);
+    super.writeBinary(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeDate(DateTime value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.DATE.serialized);
+    super.writeDate(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeTime(DateTime value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.TIME.serialized);
+    super.writeTime(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeTimestamp(DateTime value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.TIMESTAMP.serialized);
+    super.writeTimestamp(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeInterval(Period value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.INTERVAL.serialized);
+    super.writeInterval(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeBigIntNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.INTEGER.serialized);
+    super.writeBigIntNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeBinaryNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.BINARY.serialized);
+    super.writeBinaryNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeDateNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.DATE.serialized);
+    super.writeDateNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeTimeNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.TIME.serialized);
+    super.writeTimeNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeTimestampNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.TIMESTAMP.serialized);
+    super.writeTimestampNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeIntervalNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.INTERVAL.serialized);
+    super.writeIntervalNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeDecimal(BigDecimal value) throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.DECIMAL.serialized);
+    super.writeDecimal(value);
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeDecimalNull() throws IOException {
+    gen.writeStartObject();
+    gen.writeFieldName(ExtendedType.DECIMAL.serialized);
+    super.writeDecimalNull();
+    gen.writeEndObject();
+  }
+
+  @Override
+  public void writeTinyInt(byte value) throws IOException {
+    writeBigInt(value);
+  }
+
+  @Override
+  public void writeSmallInt(short value) throws IOException {
+    writeBigInt(value);
+  }
+
+  @Override
+  public void writeInt(int value) throws IOException {
+    writeBigInt(value);
+  }
+
+  @Override
+  public void writeTinyIntNull() throws IOException {
+    writeBigIntNull();
+  }
+
+  @Override
+  public void writeSmallIntNull() throws IOException {
+    writeBigIntNull();
+  }
+
+  @Override
+  public void writeIntNull() throws IOException {
+    writeBigIntNull();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
new file mode 100644
index 0000000..bec0fd2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
@@ -0,0 +1,40 @@
+/**
+ * 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.complex.fn;
+
+import com.fasterxml.jackson.core.io.SerializedString;
+
+public enum ExtendedType {
+
+  BINARY(ExtendedTypeName.BINARY),
+  DATE(ExtendedTypeName.DATE),
+  TIME(ExtendedTypeName.TIME),
+  TIMESTAMP(ExtendedTypeName.TIMESTAMP),
+  INTERVAL(ExtendedTypeName.INTERVAL),
+  INTEGER(ExtendedTypeName.INTEGER),
+  DECIMAL(ExtendedTypeName.DECIMAL);
+
+  public final SerializedString serialized;
+
+  ExtendedType(String name) {
+    this.serialized = new SerializedString(name);
+  }
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
new file mode 100644
index 0000000..fcef24b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
@@ -0,0 +1,29 @@
+/**
+ * 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.complex.fn;
+
+public interface ExtendedTypeName {
+  public static final String BINARY = "$binary";      // base64 encoded binary (ZHJpbGw=)  [from Mongo]
+  public static final String DATE = "$dateDay";       // ISO date with no time. such as (12-24-27)
+  public static final String TIME = "$time";          // ISO time with no timezone (19:20:30.45Z)
+  public static final String TIMESTAMP = "$date";     // ISO standard time (2009-02-23T00:00:00.000-08:00) [from Mongo]
+  public static final String INTERVAL = "$interval";  // ISO standard duration (PT26.4S)
+  public static final String INTEGER = "$numberLong"; // 8 byte signed integer (123) [from Mongo]
+  public static final String DECIMAL = "$decimal";    // exact numeric value (123.123)
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
new file mode 100644
index 0000000..72720de
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonOutput.java
@@ -0,0 +1,109 @@
+/**
+ * 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.complex.fn;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+
+/**
+ * Interface through which UDFs, RecordWriters and other systems can write out a JSON output.
+ * Generally used to control how non-json types are mapped to a json output stream.
+ */
+public interface JsonOutput {
+
+  // basic json tools.
+  void flush() throws IOException;
+  void writeStartArray() throws IOException;
+  void writeEndArray() throws IOException;
+  void writeStartObject() throws IOException;
+  void writeEndObject() throws IOException;
+  void writeUntypedNull() throws IOException;
+  void writeFieldName(String name) throws IOException;
+
+
+  // literals
+  void writeDecimal(BigDecimal value) throws IOException;
+  void writeTinyInt(byte value) throws IOException;
+  void writeSmallInt(short value) throws IOException;
+  void writeInt(int value) throws IOException;
+  void writeBigInt(long value) throws IOException;
+  void writeFloat(float value) throws IOException;
+  void writeDouble(double value) throws IOException;
+  void writeVarChar(String value) throws IOException;
+  void writeVar16Char(String value) throws IOException;
+  void writeBinary(byte[] value) throws IOException;
+  void writeBoolean(boolean value) throws IOException;
+  void writeDate(DateTime value) throws IOException;
+  void writeTime(DateTime value) throws IOException;
+  void writeTimestamp(DateTime value) throws IOException;
+  void writeInterval(Period value) throws IOException;
+  void writeDecimalNull() throws IOException;
+  void writeTinyIntNull() throws IOException;
+  void writeSmallIntNull() throws IOException;
+  void writeIntNull() throws IOException;
+  void writeBigIntNull() throws IOException;
+  void writeFloatNull() throws IOException;
+  void writeDoubleNull() throws IOException;
+  void writeVarcharNull() throws IOException;
+  void writeVar16charNull() throws IOException;
+  void writeBinaryNull() throws IOException;
+  void writeBooleanNull() throws IOException;
+  void writeDateNull() throws IOException;
+  void writeTimeNull() throws IOException;
+  void writeTimestampNull() throws IOException;
+  void writeIntervalNull() throws IOException;
+
+
+  // scalars reader
+  void writeDecimal(FieldReader reader) throws IOException;
+  void writeTinyInt(FieldReader reader) throws IOException;
+  void writeSmallInt(FieldReader reader) throws IOException;
+  void writeInt(FieldReader reader) throws IOException;
+  void writeBigInt(FieldReader reader) throws IOException;
+  void writeFloat(FieldReader reader) throws IOException;
+  void writeDouble(FieldReader reader) throws IOException;
+  void writeVarChar(FieldReader reader) throws IOException;
+  void writeVar16Char(FieldReader reader) throws IOException;
+  void writeBinary(FieldReader reader) throws IOException;
+  void writeBoolean(FieldReader reader) throws IOException;
+  void writeDate(FieldReader reader) throws IOException;
+  void writeTime(FieldReader reader) throws IOException;
+  void writeTimestamp(FieldReader reader) throws IOException;
+  void writeInterval(FieldReader reader) throws IOException;
+
+  // index positioned scalars
+  void writeDecimal(int index, FieldReader reader) throws IOException;
+  void writeTinyInt(int index, FieldReader reader) throws IOException;
+  void writeSmallInt(int index, FieldReader reader) throws IOException;
+  void writeInt(int index, FieldReader reader) throws IOException;
+  void writeBigInt(int index, FieldReader reader) throws IOException;
+  void writeFloat(int index, FieldReader reader) throws IOException;
+  void writeDouble(int index, FieldReader reader) throws IOException;
+  void writeVarChar(int index, FieldReader reader) throws IOException;
+  void writeVar16Char(int index, FieldReader reader) throws IOException;
+  void writeBinary(int index, FieldReader reader) throws IOException;
+  void writeBoolean(int index, FieldReader reader) throws IOException;
+  void writeDate(int index, FieldReader reader) throws IOException;
+  void writeTime(int index, FieldReader reader) throws IOException;
+  void writeTimestamp(int index, FieldReader reader) throws IOException;
+  void writeInterval(int index, FieldReader reader) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
index 9738ff8..c196fd2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonReader.java
@@ -21,82 +21,77 @@ import io.netty.buffer.DrillBuf;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.expr.holders.BigIntHolder;
-import org.apache.drill.exec.expr.holders.BitHolder;
-import org.apache.drill.exec.expr.holders.Float8Holder;
-import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.easy.json.JsonProcessor;
-import org.apache.drill.exec.store.easy.json.RewindableUtf8Reader;
 import org.apache.drill.exec.store.easy.json.reader.BaseJsonProcessor;
+import org.apache.drill.exec.vector.complex.fn.VectorOutput.ListVectorOutput;
+import org.apache.drill.exec.vector.complex.fn.VectorOutput.MapVectorOutput;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Seekable;
 
-import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonParser.Feature;
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.JsonToken;
-import com.fasterxml.jackson.core.io.IOContext;
-import com.fasterxml.jackson.core.sym.BytesToNameCanonicalizer;
-import com.fasterxml.jackson.core.util.BufferRecycler;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.io.compress.CompressionInputStream;
 
-public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
+public class JsonReader extends BaseJsonProcessor {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonReader.class);
-  public final static int MAX_RECORD_SIZE = 128*1024;
+  public final static int MAX_RECORD_SIZE = 128 * 1024;
 
+  private final WorkingBuffer workingBuffer;
   private final List<SchemaPath> columns;
   private final boolean allTextMode;
   private boolean atLeastOneWrite = false;
-
-  private FieldSelection selection;
+  private final MapVectorOutput mapOutput;
+  private final ListVectorOutput listOutput;
+  private final boolean extended = true;
 
   /**
-   * Whether we are in a reset state. In a reset state, we don't have to advance to the next token on write because
-   * we're already at the start of the next object
+   * Describes whether or not this reader can unwrap a single root array record and treat it like a set of distinct records.
    */
-  private boolean onReset = false;
+  private final boolean skipOuterList;
 
+  /**
+   * Whether the reader is currently in a situation where we are unwrapping an outer list.
+   */
+  private boolean inOuterList;
 
-  public JsonReader() throws IOException {
-    this(null, false);
-  }
+  private FieldSelection selection;
 
-  public JsonReader(DrillBuf managedBuf, boolean allTextMode) {
-    this(managedBuf, GroupScan.ALL_COLUMNS, allTextMode);
+  public JsonReader(DrillBuf managedBuf, boolean allTextMode, boolean skipOuterList) {
+    this(managedBuf, GroupScan.ALL_COLUMNS, allTextMode, skipOuterList);
   }
 
-  public JsonReader(DrillBuf managedBuf, List<SchemaPath> columns, boolean allTextMode) {
+  public JsonReader(DrillBuf managedBuf, List<SchemaPath> columns, boolean allTextMode, boolean skipOuterList) {
     super(managedBuf);
-
     assert Preconditions.checkNotNull(columns).size() > 0 : "json record reader requires at least a column";
     this.selection = FieldSelection.getFieldSelection(columns);
+    this.workingBuffer = new WorkingBuffer(managedBuf);
+    this.skipOuterList = skipOuterList;
     this.allTextMode = allTextMode;
     this.columns = columns;
+    this.mapOutput = new MapVectorOutput(workingBuffer);
+    this.listOutput = new ListVectorOutput(workingBuffer);
   }
 
   @Override
-  public void ensureAtLeastOneField(ComplexWriter writer){
-    if(!atLeastOneWrite){
+  public void ensureAtLeastOneField(ComplexWriter writer) {
+    if (!atLeastOneWrite) {
       // if we had no columns, create one empty one so we can return some data for count purposes.
       SchemaPath sp = columns.get(0);
       PathSegment root = sp.getRootSegment();
       BaseWriter.MapWriter fieldWriter = writer.rootAsMap();
-      while (root.getChild() != null && ! root.getChild().isArray()) {
+      while (root.getChild() != null && !root.getChild().isArray()) {
         fieldWriter = fieldWriter.map(root.getNameSegment().getPath());
         root = root.getChild();
       }
@@ -104,54 +99,50 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
     }
   }
 
+  public void setSource(int start, int end, DrillBuf buf) throws IOException {
+    setSource(DrillBufInputStream.getStream(start, end, buf));
+  }
+
+
   @Override
-  public void setSource(InputStream is) throws IOException{
-    parser.setInputStream(is);
-    this.onReset = false;
+  public void setSource(InputStream is) throws IOException {
+    super.setSource(is);
+    mapOutput.setParser(parser);
+    listOutput.setParser(parser);
   }
 
-  public void setSource(int start, int end, DrillBuf buf) throws IOException{
-    parser.setInputStream(DrillBufInputStream.getStream(start, end, buf));
+  @Override
+  public void setSource(JsonNode node) {
+    super.setSource(node);
+    mapOutput.setParser(parser);
+    listOutput.setParser(parser);
   }
 
   public void setSource(String data) throws IOException {
     setSource(data.getBytes(Charsets.UTF_8));
   }
 
-  public void setSource(byte[] bytes) throws IOException{
-    parser.setInputStream(new SeekableBAIS(bytes));
-    this.onReset = false;
+  public void setSource(byte[] bytes) throws IOException {
+    setSource(new SeekableBAIS(bytes));
   }
 
-
   @Override
   public ReadState write(ComplexWriter writer) throws IOException {
-    JsonToken t = onReset ? parser.getCurrentToken() : parser.nextToken();
+    JsonToken t = parser.nextToken();
 
-    while (!parser.hasCurrentToken() && parser.hasDataAvailable()) {
+    while (!parser.hasCurrentToken() && !parser.isClosed()) {
       t = parser.nextToken();
     }
 
-    if(!parser.hasCurrentToken()){
+    if (parser.isClosed()) {
       return ReadState.END_OF_STREAM;
     }
 
-    if(onReset){
-      onReset = false;
-    }else{
-      parser.mark();
-    }
-
     ReadState readState = writeToVector(writer, t);
 
-    switch(readState){
+    switch (readState) {
     case END_OF_STREAM:
       break;
-    case WRITE_FAILURE:
-      logger.debug("Ran out of space while writing object, rewinding to object start.");
-      parser.resetToMark();
-      onReset = true;
-      break;
     case WRITE_SUCCEED:
       break;
     default:
@@ -161,72 +152,113 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
     return readState;
   }
 
-  private ReadState writeToVector(ComplexWriter writer, JsonToken t) throws IOException {
-    if (!writer.ok()) {
-      return ReadState.WRITE_FAILURE;
+  private void confirmLast() throws IOException{
+    parser.nextToken();
+    if(!parser.isClosed()){
+      throw new JsonParseException("Drill attempted to unwrap a toplevel list "
+        + "in your document.  However, it appears that there is trailing content after this top level list.  Drill only "
+        + "supports querying a set of distinct maps or a single json array with multiple inner maps.", parser.getCurrentLocation());
     }
+  }
 
+  private ReadState writeToVector(ComplexWriter writer, JsonToken t) throws IOException {
     switch (t) {
-      case START_OBJECT:
-        writeDataSwitch(writer.rootAsMap());
-        break;
-      case START_ARRAY:
+    case START_OBJECT:
+      writeDataSwitch(writer.rootAsMap());
+      break;
+    case START_ARRAY:
+      if(inOuterList){
+        throw new JsonParseException("The top level of your document must either be a single array of maps or a set "
+            + "of white space delimited maps.", parser.getCurrentLocation());
+      }
+
+      if(skipOuterList){
+        t = parser.nextToken();
+        if(t == JsonToken.START_OBJECT){
+          inOuterList = true;
+          writeDataSwitch(writer.rootAsMap());
+        }else{
+          throw new JsonParseException("The top level of your document must either be a single array of maps or a set "
+              + "of white space delimited maps.", parser.getCurrentLocation());
+        }
+
+      }else{
         writeDataSwitch(writer.rootAsList());
-        break;
-      case NOT_AVAILABLE:
-        return ReadState.END_OF_STREAM;
-      default:
-        throw new JsonParseException(
-            String.format("Failure while parsing JSON.  Found token of [%s]  Drill currently only supports parsing "
-                + "json strings that contain either lists or maps.  The root object cannot be a scalar.",
-                t),
-            parser.getCurrentLocation());
       }
+      break;
+    case END_ARRAY:
 
-      if(writer.ok()){
-        return ReadState.WRITE_SUCCEED;
+      if(inOuterList){
+        confirmLast();
+        return ReadState.END_OF_STREAM;
       }else{
-        return ReadState.WRITE_FAILURE;
+        throw new JsonParseException(String.format("Failure while parsing JSON.  Ran across unexpected %s.", JsonToken.END_ARRAY), parser.getCurrentLocation());
       }
+
+    case NOT_AVAILABLE:
+      return ReadState.END_OF_STREAM;
+    default:
+      throw new JsonParseException(String.format(
+          "Failure while parsing JSON.  Found token of [%s]  Drill currently only supports parsing "
+              + "json strings that contain either lists or maps.  The root object cannot be a scalar.", t),
+          parser.getCurrentLocation());
+    }
+
+    return ReadState.WRITE_SUCCEED;
+
   }
 
-  private void writeDataSwitch(MapWriter w) throws IOException{
-    if(this.allTextMode){
-      writeDataAllText(w, this.selection);
-    }else{
-      writeData(w, this.selection);
+  private void writeDataSwitch(MapWriter w) throws IOException {
+    if (this.allTextMode) {
+      writeDataAllText(w, this.selection, true);
+    } else {
+      writeData(w, this.selection, true);
     }
   }
 
-  private void writeDataSwitch(ListWriter w) throws IOException{
-    if(this.allTextMode){
+  private void writeDataSwitch(ListWriter w) throws IOException {
+    if (this.allTextMode) {
       writeDataAllText(w);
-    }else{
+    } else {
       writeData(w);
     }
   }
 
   private void consumeEntireNextValue() throws IOException {
     switch (parser.nextToken()) {
-      case START_ARRAY:
-      case START_OBJECT:
-        parser.skipChildren();
-        return;
-      default:
-        // hit a single value, do nothing as the token was already read
-        // in the switch statement
-        return;
+    case START_ARRAY:
+    case START_OBJECT:
+      parser.skipChildren();
+      return;
+    default:
+      // hit a single value, do nothing as the token was already read
+      // in the switch statement
+      return;
     }
   }
 
-  private void writeData(MapWriter map, FieldSelection selection) throws IOException {
+  /**
+   *
+   * @param map
+   * @param selection
+   * @param moveForward
+   *          Whether or not we should start with using the current token or the next token. If moveForward = true, we
+   *          should start with the next token and ignore the current one.
+   * @throws IOException
+   */
+  private void writeData(MapWriter map, FieldSelection selection, boolean moveForward) throws IOException {
     //
     map.start();
-    outside: while(true) {
-      if (!map.ok()) {
-        return;
+    outside: while (true) {
+
+      JsonToken t;
+      if(moveForward){
+        t = parser.nextToken();
+      }else{
+        t = parser.getCurrentToken();
+        moveForward = true;
       }
-      JsonToken t = parser.nextToken();
+
       if (t == JsonToken.NOT_AVAILABLE || t == JsonToken.END_OBJECT) {
         return;
       }
@@ -234,23 +266,25 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
       assert t == JsonToken.FIELD_NAME : String.format("Expected FIELD_NAME but got %s.", t.name());
 
       final String fieldName = parser.getText();
+
       FieldSelection childSelection = selection.getChild(fieldName);
-      if(childSelection.isNeverValid()){
+      if (childSelection.isNeverValid()) {
         consumeEntireNextValue();
         continue outside;
       }
 
-      switch(parser.nextToken()) {
+      switch (parser.nextToken()) {
       case START_ARRAY:
         writeData(map.list(fieldName));
         break;
       case START_OBJECT:
-        writeData(map.map(fieldName), childSelection);
+        if (!writeMapDataIfTyped(map, fieldName)) {
+          writeData(map.map(fieldName), childSelection, false);
+        }
         break;
       case END_OBJECT:
         break outside;
 
-      case VALUE_EMBEDDED_OBJECT:
       case VALUE_FALSE: {
         map.bit(fieldName).writeBit(0);
         atLeastOneWrite = true;
@@ -262,10 +296,6 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
         break;
       }
       case VALUE_NULL:
-        // do check value capacity only if vector is allocated.
-        if (map.getValueCapacity() > 0) {
-          map.checkValueCapacity();
-        }
         // do nothing as we don't have a type.
         break;
       case VALUE_NUMBER_FLOAT:
@@ -285,20 +315,27 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
         throw new IllegalStateException("Unexpected token " + parser.getCurrentToken());
 
       }
+
     }
     map.end();
 
   }
 
-
-  private void writeDataAllText(MapWriter map, FieldSelection selection) throws IOException {
+  private void writeDataAllText(MapWriter map, FieldSelection selection, boolean moveForward) throws IOException {
     //
     map.start();
-    outside: while(true) {
-      if (!map.ok()) {
-        return;
+    outside: while (true) {
+
+
+      JsonToken t;
+
+      if(moveForward){
+        t = parser.nextToken();
+      }else{
+        t = parser.getCurrentToken();
+        moveForward = true;
       }
-      JsonToken t = parser.nextToken();
+
       if (t == JsonToken.NOT_AVAILABLE || t == JsonToken.END_OBJECT) {
         return;
       }
@@ -307,18 +344,19 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
 
       final String fieldName = parser.getText();
       FieldSelection childSelection = selection.getChild(fieldName);
-      if(childSelection.isNeverValid()){
+      if (childSelection.isNeverValid()) {
         consumeEntireNextValue();
         continue outside;
       }
 
-
-      switch(parser.nextToken()) {
+      switch (parser.nextToken()) {
       case START_ARRAY:
         writeDataAllText(map.list(fieldName));
         break;
       case START_OBJECT:
-        writeDataAllText(map.map(fieldName), childSelection);
+        if (!writeMapDataIfTyped(map, fieldName)) {
+          writeDataAllText(map.map(fieldName), childSelection, false);
+        }
         break;
       case END_OBJECT:
         break outside;
@@ -333,14 +371,9 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
         atLeastOneWrite = true;
         break;
       case VALUE_NULL:
-        // do check value capacity only if vector is allocated.
-        if (map.getValueCapacity() > 0) {
-          map.checkValueCapacity();
-        }
         // do nothing as we don't have a type.
         break;
 
-
       default:
         throw new IllegalStateException("Unexpected token " + parser.getCurrentToken());
 
@@ -350,46 +383,65 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
 
   }
 
-
-  private void ensure(int length) {
-    workBuf = workBuf.reallocIfNeeded(length);
+  /**
+   * Will attempt to take the current value and consume it as an extended value (if extended mode is enabled).  Whether extended is enable or disabled, will consume the next token in the stream.
+   * @param writer
+   * @param fieldName
+   * @return
+   * @throws IOException
+   */
+  private boolean writeMapDataIfTyped(MapWriter writer, String fieldName) throws IOException {
+    if (extended) {
+      return mapOutput.run(writer, fieldName);
+    } else {
+      parser.nextToken();
+      return false;
+    }
   }
 
-  private int prepareVarCharHolder(String value) throws IOException {
-    byte[] b = value.getBytes(Charsets.UTF_8);
-    ensure(b.length);
-    workBuf.setBytes(0, b);
-    return b.length;
+  /**
+   * Will attempt to take the current value and consume it as an extended value (if extended mode is enabled).  Whether extended is enable or disabled, will consume the next token in the stream.
+   * @param writer
+   * @return
+   * @throws IOException
+   */
+  private boolean writeListDataIfTyped(ListWriter writer) throws IOException {
+    if (extended) {
+      return listOutput.run(writer);
+    } else {
+      parser.nextToken();
+      return false;
+    }
   }
 
   private void handleString(JsonParser parser, MapWriter writer, String fieldName) throws IOException {
-    writer.varChar(fieldName).writeVarChar(0, prepareVarCharHolder(parser.getText()), workBuf);
+    writer.varChar(fieldName).writeVarChar(0, workingBuffer.prepareVarCharHolder(parser.getText()),
+        workingBuffer.getBuf());
   }
 
   private void handleString(JsonParser parser, ListWriter writer) throws IOException {
-    writer.varChar().writeVarChar(0, prepareVarCharHolder(parser.getText()), workBuf);
+    writer.varChar().writeVarChar(0, workingBuffer.prepareVarCharHolder(parser.getText()), workingBuffer.getBuf());
   }
 
   private void writeData(ListWriter list) throws IOException {
     list.start();
     outside: while (true) {
-      if (!list.ok()) {
-        return;
-      }
 
       switch (parser.nextToken()) {
       case START_ARRAY:
         writeData(list.list());
         break;
       case START_OBJECT:
-        writeData(list.map(), FieldSelection.ALL_VALID);
+        if (!writeListDataIfTyped(list)) {
+          writeData(list.map(), FieldSelection.ALL_VALID, false);
+        }
         break;
       case END_ARRAY:
       case END_OBJECT:
         break outside;
 
       case VALUE_EMBEDDED_OBJECT:
-      case VALUE_FALSE:{
+      case VALUE_FALSE: {
         list.bit().writeBit(0);
         atLeastOneWrite = true;
         break;
@@ -400,9 +452,9 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
         break;
       }
       case VALUE_NULL:
-        throw new DrillRuntimeException("Null values are not supported in lists be default. " +
-            "Please set `store.json.all_text_mode` to true to read lists containing nulls. " +
-            "Be advised that this will treat JSON null values as string containing the word 'null'.");
+        throw new DrillRuntimeException("Drill does not currently null values in lists. "
+            + "Please set `store.json.all_text_mode` to true to read lists containing nulls. "
+            + "Be advised that this will treat JSON null values as string containing the word 'null'.");
       case VALUE_NUMBER_FLOAT:
         list.float8().writeFloat8(parser.getDoubleValue());
         atLeastOneWrite = true;
@@ -426,16 +478,15 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
   private void writeDataAllText(ListWriter list) throws IOException {
     list.start();
     outside: while (true) {
-      if (!list.ok()) {
-        return;
-      }
 
       switch (parser.nextToken()) {
       case START_ARRAY:
         writeDataAllText(list.list());
         break;
       case START_OBJECT:
-        writeDataAllText(list.map(), FieldSelection.ALL_VALID);
+        if (!writeListDataIfTyped(list)) {
+          writeDataAllText(list.map(), FieldSelection.ALL_VALID, false);
+        }
         break;
       case END_ARRAY:
       case END_OBJECT:
@@ -460,7 +511,7 @@ public class JsonReader extends BaseJsonProcessor implements JsonProcessor {
   }
 
   public DrillBuf getWorkBuf() {
-    return workBuf;
+    return workingBuffer.getBuf();
   }
 
 }


[2/6] drill git commit: DRILL-2675: Implement a subset of User Exceptions to improve how errors are reported to the user

Posted by ja...@apache.org.
DRILL-2675: Implement a subset of User Exceptions to improve how errors are reported to the user


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

Branch: refs/heads/master
Commit: 99b6d0e6d2fc6dea165e9e809db4fc63d9a70d3a
Parents: 859e6a8
Author: adeneche <ad...@gmail.com>
Authored: Thu Apr 9 09:30:33 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Apr 15 07:02:10 2015 -0700

----------------------------------------------------------------------
 .../common/exceptions/DrillRemoteException.java |  44 ++
 .../common/exceptions/DrillUserException.java   | 204 ++++++++++
 .../drill/common/exceptions/ErrorHelper.java    | 186 +++++++++
 .../common/exceptions/UserExceptionContext.java | 148 +++++++
 .../exceptions/TestDrillUserException.java      | 130 ++++++
 .../apache/drill/exec/client/DrillClient.java   |   5 +-
 .../exec/client/PrintingResultsListener.java    |   9 +-
 .../apache/drill/exec/ops/FragmentContext.java  |  17 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   2 -
 .../drill/exec/planner/sql/DrillSqlWorker.java  |  18 +-
 .../drill/exec/rpc/CoordinationQueue.java       |   3 +-
 .../drill/exec/rpc/RemoteRpcException.java      |  43 --
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |  12 +-
 .../drill/exec/rpc/user/QueryResultHandler.java |  28 +-
 .../exec/rpc/user/UserResultsListener.java      |   4 +-
 .../drill/exec/server/rest/QueryWrapper.java    |   6 +-
 .../exec/store/easy/json/JSONRecordReader.java  |  28 +-
 .../org/apache/drill/exec/work/ErrorHelper.java | 229 -----------
 .../apache/drill/exec/work/foreman/Foreman.java |  17 +-
 .../drill/exec/work/foreman/QueryManager.java   |   4 +-
 .../work/foreman/SqlUnsupportedException.java   |   4 +-
 .../work/fragment/AbstractStatusReporter.java   |  14 +-
 .../exec/work/fragment/FragmentExecutor.java    |  13 +-
 .../exec/work/fragment/StatusReporter.java      |   3 +-
 .../java/org/apache/drill/BaseTestQuery.java    |  11 +-
 .../org/apache/drill/SingleRowListener.java     |   6 +-
 .../java/org/apache/drill/TestBugFixes.java     |   6 +-
 .../apache/drill/TestDisabledFunctionality.java | 140 +++----
 .../java/org/apache/drill/TestStarQueries.java  |   6 +-
 .../java/org/apache/drill/TestUnionAll.java     |   8 +-
 .../exec/server/TestDrillbitResilience.java     |  22 +-
 .../store/parquet/ParquetResultListener.java    |   8 +-
 .../store/parquet/TestParquetPhysicalPlan.java  |   4 +-
 .../java/org/apache/drill/jdbc/DrillCursor.java |   4 +-
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   8 +-
 .../drill/exec/proto/SchemaUserBitShared.java   |   4 +-
 .../apache/drill/exec/proto/UserBitShared.java  | 408 +++++++++++++------
 .../drill/exec/proto/beans/DrillPBError.java    |  58 ++-
 protocol/src/main/protobuf/UserBitShared.proto  |  14 +-
 39 files changed, 1268 insertions(+), 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/common/src/main/java/org/apache/drill/common/exceptions/DrillRemoteException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/DrillRemoteException.java b/common/src/main/java/org/apache/drill/common/exceptions/DrillRemoteException.java
new file mode 100644
index 0000000..7b707a5
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/DrillRemoteException.java
@@ -0,0 +1,44 @@
+/**
+ * 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.common.exceptions;
+
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+
+/**
+ * Wraps a DrillPBError object so we don't need to rebuilt it multiple times when sending it to the client. It also
+ * gives access to the original exception className and message.
+ */
+public class DrillRemoteException extends DrillUserException {
+
+  private final DrillPBError error;
+
+  public DrillRemoteException(DrillPBError error) {
+    super(null, "Drill Remote Exception", null);
+    this.error = error;
+  }
+
+  @Override
+  public String getMessage() {
+    return error.getMessage(); // we don't want super class to generate the error message
+  }
+
+  @Override
+  public DrillPBError getOrCreatePBError(boolean verbose) {
+    return error;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/common/src/main/java/org/apache/drill/common/exceptions/DrillUserException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/DrillUserException.java b/common/src/main/java/org/apache/drill/common/exceptions/DrillUserException.java
new file mode 100644
index 0000000..73718ae
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/DrillUserException.java
@@ -0,0 +1,204 @@
+/**
+ * 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.common.exceptions;
+
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+
+public class DrillUserException extends DrillRuntimeException {
+
+  /**
+   * 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
+   * exception's context
+   */
+  public static class Builder extends UserExceptionContext {
+
+    private final Throwable cause;
+    private final String message;
+    private final DrillPBError.ErrorType errorType;
+
+    private final DrillUserException uex;
+
+    /**
+     * builds a system error that wrap an existing exception. If the exception is, or wraps, a DrillUserException it
+     * won't be converted to a system error.
+     *
+     * We should never need to call this, it will be done automatically before the exception is sent to the client.
+     *
+     * @param cause exception to wrap into a system error
+     */
+    Builder(Throwable cause) {
+      this(DrillPBError.ErrorType.SYSTEM, cause);
+    }
+
+    /**
+     * builds a new user exception of the specified type, with a defined error message
+     *
+     * @param errorType user exception's type
+     * @param format A format string
+     * @param args Arguments referenced by the format specifiers in the format string.
+     */
+    public Builder(DrillPBError.ErrorType errorType, String format, Object... args) {
+      this(errorType, null, format, args);
+    }
+
+    /**
+     * wraps an existing exception inside a user exception. If the exception is, or wraps, a user exception
+     * already, the builder will extract the original user exception and use it instead. If the builder creates a new
+     * user exception it will use the passed exception message, otherwise the builder won't change the message of the
+     * exception
+     *
+     * @param errorType user exception type that should be created if the passed exception isn't, or doesn't wrap a user exception
+     * @param cause exception to wrap inside a user exception
+     */
+    public Builder(DrillPBError.ErrorType errorType, Throwable cause) {
+      this(errorType, cause, null);
+    }
+
+    /**
+     * wraps an existing exception inside a user exception. If the exception is, or wraps, a user exception
+     * already, the builder will extract the original user exception and use it instead.
+     * If the builder creates a new user exception it will use the passed exception message, otherwise the builder
+     * won't change the message of the exception and will add the passed message to the context instead
+     *
+     * @param errorType user exception type that should be created if the passed exception isn't, or doesn't wrap a user exception
+     * @param cause exception to wrap inside a user exception
+     * @param format A format string
+     * @param args Arguments referenced by the format specifiers in the format string.
+     */
+    public Builder(DrillPBError.ErrorType errorType, Throwable cause, String format, Object... args) {
+      super(ErrorHelper.getExceptionContextOrNew(cause));
+      this.cause = cause;
+
+      if (format == null) {
+        this.message = cause != null ? cause.getMessage() : null;
+      } else {
+        this.message = String.format(format, args);
+      }
+
+      //TODO handle the improbable case where cause is a SYSTEM exception ?
+      uex = ErrorHelper.findWrappedUserException(cause);
+      if (uex != null) {
+        this.errorType = null;
+        if (format != null) {
+          // we won't change the exception's message, so add it to the context
+          add(this.message);
+        }
+      } else {
+        // we will create a new user exception
+        this.errorType = errorType;
+      }
+    }
+
+    public DrillUserException build() {
+
+      if (uex != null) {
+        return uex;
+      }
+
+      return new DrillUserException(this);
+    }
+  }
+
+  private final DrillPBError.ErrorType errorType;
+
+  private final UserExceptionContext context;
+
+  protected DrillUserException(DrillPBError.ErrorType errorType, String message, Throwable cause) {
+    super(message, cause);
+
+    this.errorType = errorType;
+    this.context = new UserExceptionContext();
+  }
+
+  private DrillUserException(Builder builder) {
+    super(builder.message, builder.cause);
+    this.errorType = builder.errorType;
+    this.context = builder;
+  }
+
+  public UserExceptionContext getContext() {
+    return context;
+  }
+
+  /**
+   * generates the message that will be displayed to the client without the stack trace.
+   *
+   * @return non verbose error message
+   */
+  @Override
+  public String getMessage() {
+    return generateMessage();
+  }
+
+  /**
+   *
+   * @return the error message that was passed to the builder
+   */
+  public String getOriginalMessage() {
+    return super.getMessage();
+  }
+
+  /**
+   * generates the message that will be displayed to the client. The message also contains the stack trace.
+   *
+   * @return verbose error message
+   */
+  public String getVerboseMessage() {
+    return generateMessage() + "\n\n" + ErrorHelper.buildCausesMessage(getCause());
+  }
+
+  /**
+   * returns or creates a DrillPBError object corresponding to this user exception.
+   *
+   * @param verbose should the error object contain the verbose error message ?
+   * @return protobuf error object
+   */
+  public DrillPBError getOrCreatePBError(boolean verbose) {
+    String message = verbose ? getVerboseMessage() : getMessage();
+
+    DrillPBError.Builder builder = DrillPBError.newBuilder();
+    builder.setErrorType(errorType);
+    builder.setErrorId(context.getErrorId());
+    if (context.getEndpoint() != null) {
+      builder.setEndpoint(context.getEndpoint());
+    }
+    builder.setMessage(message);
+
+    if (getCause() != null) {
+      // some unit tests use this information to make sure a specific exception was thrown in the server
+      builder.setException(ErrorHelper.getWrapper(getCause()));
+    }
+    return builder.build();
+  }
+
+  /**
+   * Generates a user error message that has the following structure:
+   * ERROR TYPE: ERROR_MESSAGE
+   * CONTEXT
+   * [ERROR_ID on DRILLBIT_IP:DRILLBIT_USER_PORT]
+   *
+   * @return generated user error message
+   */
+  private String generateMessage() {
+    return errorType + " ERROR: " + super.getMessage() + "\n" +
+      context.generateContextMessage();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java b/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java
new file mode 100644
index 0000000..1dc4eb5
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java
@@ -0,0 +1,186 @@
+/**
+ * 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.common.exceptions;
+
+import org.apache.drill.exec.proto.UserBitShared.ExceptionWrapper;
+import org.apache.drill.exec.proto.UserBitShared.StackTraceElementWrapper;
+
+import java.util.regex.Pattern;
+
+/**
+ * Utility class that handles error message generation from protobuf error objects.
+ */
+public class ErrorHelper {
+
+  private final static Pattern IGNORE= Pattern.compile("^(sun|com\\.sun|java).*");
+
+  /**
+   * Wraps the exception into a SYSTEM ERROR if it's not already a DrillUserException, or if it's not wrapping a
+   * DrillUserException
+   * @param ex exception to wrap
+   * @return user exception
+   */
+  public static DrillUserException wrap(final Throwable ex) {
+    return new DrillUserException.Builder(ex).build();
+  }
+
+  /**
+   * returns the user exception context for DrillUserExceptions(s) even if they are wrapped multiple times. If no
+   * DrillUserException is found, it will create a new one.
+   * This is useful if we want to add context to user exception before re-throwing it. For all other exception the
+   * context will be discarded.
+   *
+   * @param ex exception we are trying to get the context for
+   * @return user exception context
+   */
+  public static UserExceptionContext getExceptionContextOrNew(final Throwable ex) {
+    DrillUserException uex = findWrappedUserException(ex);
+    if (uex != null) {
+      //TODO if uex is SYSTEM exception the calling code will be able to add context information to it. Do we want this ?
+      return uex.getContext();
+    }
+
+    return new UserExceptionContext();
+  }
+
+  static String buildCausesMessage(final Throwable t) {
+
+    StringBuilder sb = new StringBuilder();
+    Throwable ex = t;
+    boolean cause = false;
+    while(ex != null){
+
+      sb.append("  ");
+
+      if(cause){
+        sb.append("Caused By ");
+      }
+
+      sb.append("(");
+      sb.append(ex.getClass().getCanonicalName());
+      sb.append(") ");
+      sb.append(ex.getMessage());
+      sb.append("\n");
+
+      for(StackTraceElement st : ex.getStackTrace()){
+        sb.append("    ");
+        sb.append(st.getClassName());
+        sb.append('.');
+        sb.append(st.getMethodName());
+        sb.append("():");
+        sb.append(st.getLineNumber());
+        sb.append("\n");
+      }
+      cause = true;
+
+      if(ex.getCause() != null && ex.getCause() != ex){
+        ex = ex.getCause();
+      } else {
+        ex = null;
+      }
+    }
+
+    return sb.toString();
+  }
+
+  static ExceptionWrapper getWrapper(Throwable ex) {
+    return getWrapperBuilder(ex).build();
+  }
+
+  private static ExceptionWrapper.Builder getWrapperBuilder(Throwable ex) {
+    return getWrapperBuilder(ex, false);
+  }
+
+  private static ExceptionWrapper.Builder getWrapperBuilder(Throwable ex, boolean includeAllStack) {
+    ExceptionWrapper.Builder ew = ExceptionWrapper.newBuilder();
+    if(ex.getMessage() != null) {
+      ew.setMessage(ex.getMessage());
+    }
+    ew.setExceptionClass(ex.getClass().getCanonicalName());
+    boolean isHidden = false;
+    StackTraceElement[] stackTrace = ex.getStackTrace();
+    for(int i = 0; i < stackTrace.length; i++){
+      StackTraceElement ele = ex.getStackTrace()[i];
+      if(include(ele, includeAllStack)){
+        if(isHidden){
+          isHidden = false;
+        }
+        ew.addStackTrace(getSTWrapper(ele));
+      }else{
+        if(!isHidden){
+          isHidden = true;
+          ew.addStackTrace(getEmptyST());
+        }
+      }
+
+    }
+
+    if(ex.getCause() != null && ex.getCause() != ex){
+      ew.setCause(getWrapper(ex.getCause()));
+    }
+    return ew;
+  }
+
+  private static boolean include(StackTraceElement ele, boolean includeAllStack) {
+    return includeAllStack || !(IGNORE.matcher(ele.getClassName()).matches());
+  }
+
+  private static StackTraceElementWrapper.Builder getSTWrapper(StackTraceElement ele) {
+    StackTraceElementWrapper.Builder w = StackTraceElementWrapper.newBuilder();
+    w.setClassName(ele.getClassName());
+    if(ele.getFileName() != null) {
+      w.setFileName(ele.getFileName());
+    }
+    w.setIsNativeMethod(ele.isNativeMethod());
+    w.setLineNumber(ele.getLineNumber());
+    w.setMethodName(ele.getMethodName());
+    return w;
+  }
+
+  private static StackTraceElementWrapper.Builder getEmptyST() {
+    StackTraceElementWrapper.Builder w = StackTraceElementWrapper.newBuilder();
+    w.setClassName("...");
+    w.setIsNativeMethod(false);
+    w.setLineNumber(0);
+    w.setMethodName("...");
+    return w;
+  }
+
+  /**
+   * searches for a DrillUserException wrapped inside the exception
+   * @param ex exception
+   * @return null if exception is null or no DrillUserException was found
+   */
+  static DrillUserException findWrappedUserException(Throwable ex) {
+    if (ex == null) {
+      return null;
+    }
+
+    Throwable cause = ex;
+    while (!(cause instanceof DrillUserException)) {
+      if (cause.getCause() != null && cause.getCause() != cause) {
+        cause = cause.getCause();
+      } else {
+        return null;
+      }
+    }
+
+    return (DrillUserException) cause;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java b/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
new file mode 100644
index 0000000..ef7d486
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
@@ -0,0 +1,148 @@
+/**
+ * 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.common.exceptions;
+
+import org.apache.drill.exec.proto.CoordinationProtos;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Holds context information about a DrillUserException. We can add structured context information that will be used
+ * to generate the error message displayed to the client. We can also specify which context information should only
+ * be displayed in verbose mode
+ */
+public class UserExceptionContext {
+
+  private final String errorId;
+  private final List<String> contextList;
+
+  private CoordinationProtos.DrillbitEndpoint endpoint;
+
+  UserExceptionContext() {
+    errorId = UUID.randomUUID().toString();
+    contextList = new ArrayList<>();
+  }
+
+  UserExceptionContext(UserExceptionContext context) {
+    this.errorId = context.errorId;
+    this.contextList = context.contextList;
+    this.endpoint = context.endpoint;
+  }
+  /**
+   * adds a context line to the bottom of the context list
+   * @param context context line
+   */
+  public UserExceptionContext add(String context) {
+    contextList.add(context);
+    return this;
+  }
+
+  public UserExceptionContext add(CoordinationProtos.DrillbitEndpoint endpoint) {
+    //TODO should we allos the endpoint to change once set ?
+    this.endpoint = endpoint;
+    return this;
+  }
+
+  /**
+   * adds an int to the bottom of the context list
+   * @param context context prefix string
+   * @param value int value
+   */
+  public UserExceptionContext add(String context, long value) {
+    add(context + ": " + value);
+    return this;
+  }
+
+  /**
+   * adds a double to the bottom of the context list
+   * @param context context prefix string
+   * @param value double value
+   */
+  public UserExceptionContext add(String context, double value) {
+    add(context + ": " + value);
+    return this;
+  }
+
+  /**
+   * adds a context line at the top of the context list
+   * @param context context line
+   */
+  public UserExceptionContext push(String context) {
+    contextList.add(0, context);
+    return this;
+  }
+
+  /**
+   * adds an int at the top of the context list
+   * @param context context prefix string
+   * @param value int value
+   */
+  public UserExceptionContext push(String context, long value) {
+    push(context + ": " + value);
+    return this;
+  }
+
+  /**
+   * adds a double at the top of the context list
+   * @param context context prefix string
+   * @param value double value
+   */
+  public UserExceptionContext push(String context, double value) {
+    push(context + ": " + value);
+    return this;
+  }
+
+  String getErrorId() {
+    return errorId;
+  }
+
+  CoordinationProtos.DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+  /**
+   * generate a context message
+   * @return string containing all context information concatenated
+   */
+  String generateContextMessage() {
+    StringBuilder sb = new StringBuilder();
+
+    for (String context : contextList) {
+      sb.append(context).append("\n");
+    }
+
+    if (errorId != null || endpoint != null) {
+      // add identification infos
+      sb.append("\n[");
+      if (errorId != null) {
+        sb.append(errorId).append(" ");
+      }
+      if(endpoint != null) {
+        sb.append("on ")
+          .append(endpoint.getAddress())
+          .append(":")
+          .append(endpoint.getUserPort());
+      }
+      sb.append("]\n");
+    }
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/common/src/test/java/org/apache/drill/common/exceptions/TestDrillUserException.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/common/exceptions/TestDrillUserException.java b/common/src/test/java/org/apache/drill/common/exceptions/TestDrillUserException.java
new file mode 100644
index 0000000..7b19d55
--- /dev/null
+++ b/common/src/test/java/org/apache/drill/common/exceptions/TestDrillUserException.java
@@ -0,0 +1,130 @@
+/**
+ * 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.common.exceptions;
+
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test various use cases around creating user exceptions
+ */
+public class TestDrillUserException {
+
+  private Exception wrap(DrillUserException uex, int numWraps) {
+    Exception ex = uex;
+    for (int i = 0; i < numWraps; i++) {
+      ex = new Exception("wrap #" + (i+1), ex);
+    }
+
+    return ex;
+  }
+
+  // make sure system exceptions are created properly
+  @Test
+  public void testBuildSystemException() {
+    try {
+      throw new DrillUserException.Builder(new RuntimeException("this is an exception")).build();
+    } catch (DrillUserException ex) {
+      DrillPBError error = ex.getOrCreatePBError(true);
+      Assert.assertEquals(ErrorType.SYSTEM, error.getErrorType());
+    }
+  }
+
+  @Test
+  public void testBuildUserExceptionWithMessage() {
+    String message = "Test message";
+
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.DATA_WRITE, message).build();
+    DrillPBError error = uex.getOrCreatePBError(false);
+
+    Assert.assertEquals(ErrorType.DATA_WRITE, error.getErrorType());
+    Assert.assertEquals(message, uex.getOriginalMessage());
+  }
+
+  @Test
+  public void testBuildUserExceptionWithCause() {
+    String message = "Test message";
+
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.DATA_WRITE, new RuntimeException(message)).build();
+    DrillPBError error = uex.getOrCreatePBError(false);
+
+    // cause message should be used
+    Assert.assertEquals(ErrorType.DATA_WRITE, error.getErrorType());
+    Assert.assertEquals(message, uex.getOriginalMessage());
+  }
+
+  @Test
+  public void testBuildUserExceptionWithCauseAndMessage() {
+    String messageA = "Test message A";
+    String messageB = "Test message B";
+
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.DATA_WRITE, new RuntimeException(messageA), messageB).build();
+    DrillPBError error = uex.getOrCreatePBError(false);
+
+    // passed message should override the cause message
+    Assert.assertEquals(ErrorType.DATA_WRITE, error.getErrorType());
+    Assert.assertFalse(error.getMessage().contains(messageA)); // messageA should not be part of the context
+    Assert.assertEquals(messageB, uex.getOriginalMessage());
+  }
+
+  @Test
+  public void testBuildUserExceptionWithUserExceptionCauseAndMessage() {
+    String messageA = "Test message A";
+    String messageB = "Test message B";
+
+    DrillUserException original = new DrillUserException.Builder(ErrorType.CONNECTION, messageA).build();
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.DATA_WRITE, wrap(original, 5), messageB).build();
+
+    //builder should return the unwrapped original user exception and not build a new one
+    Assert.assertEquals(original, uex);
+
+    DrillPBError error = uex.getOrCreatePBError(false);
+    Assert.assertEquals(messageA, uex.getOriginalMessage());
+    Assert.assertTrue(error.getMessage().contains(messageB)); // messageA should be part of the context
+  }
+
+  @Test
+  public void testBuildUserExceptionWithFormattedMessage() {
+    String format = "This is test #%d";
+
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.CONNECTION, format, 5).build();
+    DrillPBError error = uex.getOrCreatePBError(false);
+
+    Assert.assertEquals(ErrorType.CONNECTION, error.getErrorType());
+    Assert.assertEquals(String.format(format, 5), uex.getOriginalMessage());
+  }
+
+  // make sure wrapped user exceptions are retrieved properly when calling ErrorHelper.wrap()
+  @Test
+  public void testWrapUserException() {
+    DrillUserException uex = new DrillUserException.Builder(ErrorType.DATA_READ, "this is a data read exception").build();
+
+    Exception wrapped = wrap(uex, 3);
+    Assert.assertEquals(uex, ErrorHelper.wrap(wrapped));
+  }
+
+  @Test
+  public void testEdgeCases() {
+    new DrillUserException.Builder(null).build();
+    new DrillUserException.Builder(ErrorType.DATA_WRITE, null).build().getOrCreatePBError(true);
+    new DrillUserException.Builder(ErrorType.DATA_WRITE, null).build().getOrCreatePBError(true);
+    new DrillUserException.Builder(ErrorType.DATA_WRITE, new RuntimeException(), null).build().getOrCreatePBError(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 579cf7d..6555cad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -33,6 +33,7 @@ import java.util.Vector;
 
 import io.netty.channel.EventLoopGroup;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.zk.ZKClusterCoordinator;
@@ -307,9 +308,9 @@ public class DrillClient implements Closeable, ConnectionThrottle {
     }
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       // or  !client.isActive()
-      if (ex instanceof ChannelClosedException) {
+      if (ex.getCause() instanceof ChannelClosedException) {
         if (reconnect()) {
           try {
             client.submitQuery(this, query);

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
index 98948af..a11cec0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
@@ -22,6 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -29,7 +31,6 @@ import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -42,7 +43,7 @@ public class PrintingResultsListener implements UserResultsListener {
   Format format;
   int    columnWidth;
   BufferAllocator allocator;
-  volatile Exception exception;
+  volatile DrillUserException exception;
   QueryId queryId;
 
   public PrintingResultsListener(DrillConfig config, Format format, int columnWidth) {
@@ -53,7 +54,7 @@ public class PrintingResultsListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(RpcException ex) {
+  public void submissionFailed(DrillUserException ex) {
     exception = ex;
     System.out.println("Exception (no rows returned): " + ex );
     latch.countDown();
@@ -76,7 +77,7 @@ public class PrintingResultsListener implements UserResultsListener {
       try {
         loader.load(header.getDef(), data);
       } catch (SchemaChangeException e) {
-        submissionFailed(new RpcException(e));
+        submissionFailed(ErrorHelper.wrap(e));
       }
 
       switch(format) {

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/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 da2229c..8c402e8 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
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.ops;
 
 import com.google.common.base.Preconditions;
-import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 
 import java.io.IOException;
@@ -30,6 +29,8 @@ import net.hydromatic.optiq.jdbc.SimpleOptiqSchema;
 
 import org.apache.drill.common.DeferredException;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.ClassGenerator;
@@ -37,18 +38,14 @@ import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
-import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.FragmentWritableBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.ControlTunnel;
-import org.apache.drill.exec.rpc.data.DataTunnel;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.FragmentOptionManager;
@@ -150,10 +147,16 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
 
   public void fail(Throwable cause) {
     final FragmentHandle fragmentHandle = fragment.getHandle();
+
+    DrillUserException dse = ErrorHelper.wrap(cause);
+    dse.getContext().add(getIdentity());
+
+    // log the error id
     logger.error("Fragment Context received failure -- Fragment: {}:{}",
-      fragmentHandle.getMajorFragmentId(), fragmentHandle.getMinorFragmentId(), cause);
+      fragmentHandle.getMajorFragmentId(), fragmentHandle.getMinorFragmentId(), dse);
+
     setState(FragmentContextState.FAILED);
-    deferredException.addThrowable(cause);
+    deferredException.addThrowable(dse);
   }
 
   public void cancel() {

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 6b3caf4..80cbd81 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -38,8 +38,6 @@ import com.google.common.base.Preconditions;
 public class ScreenCreator implements RootCreator<Screen>{
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
 
-
-
   @Override
   public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) throws ExecutionSetupException {
     Preconditions.checkNotNull(children);

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index b98778d..7892999 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.planner.sql;
 
 import java.io.IOException;
-import java.io.Reader;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -30,6 +29,7 @@ import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.RuleSet;
 import net.hydromatic.optiq.tools.ValidationException;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
@@ -44,6 +44,7 @@ import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
@@ -56,10 +57,8 @@ import org.eigenbase.relopt.RelTraitDef;
 import org.eigenbase.relopt.hep.HepPlanner;
 import org.eigenbase.relopt.hep.HepProgramBuilder;
 import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.parser.SqlAbstractParserImpl;
 import org.eigenbase.sql.parser.SqlParseException;
 import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.sql.parser.SqlParserImplFactory;
 
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
@@ -105,8 +104,8 @@ public class DrillSqlWorker {
   private RuleSet[] getRules(QueryContext context) {
     StoragePluginRegistry storagePluginRegistry = context.getStorage();
     RuleSet drillLogicalRules = DrillRuleSets.mergedRuleSets(
-        DrillRuleSets.getDrillBasicRules(context),
-        DrillRuleSets.getDrillUserConfigurableLogicalRules(context));
+      DrillRuleSets.getDrillBasicRules(context),
+      DrillRuleSets.getDrillUserConfigurableLogicalRules(context));
     RuleSet drillPhysicalMem = DrillRuleSets.mergedRuleSets(
         DrillRuleSets.getPhysicalRules(context),
         storagePluginRegistry.getStoragePluginRuleSet());
@@ -152,14 +151,13 @@ public class DrillSqlWorker {
       handler = new DefaultSqlHandler(config, textPlan);
     }
 
-    try{
+    try {
       return handler.getPlan(sqlNode);
-    }catch(ValidationException e){
-      throw new QueryInputException("Failure validating SQL. " + e.getMessage(), e);
+    } catch(ValidationException e) {
+      String errorMessage = e.getCause() != null ? e.getCause().getMessage() : e.getMessage();
+      throw new DrillUserException.Builder(ErrorType.PARSE, e, errorMessage).build();
     } catch (IOException | RelConversionException e) {
       throw new QueryInputException("Failure handling SQL.", e);
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
index 0016d6a..ad5002e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
@@ -23,6 +23,7 @@ import io.netty.channel.ChannelFuture;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.drill.common.exceptions.DrillRemoteException;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 
 /**
@@ -150,7 +151,7 @@ public class CoordinationQueue {
     // logger.debug("Updating failed future.");
     try {
       RpcOutcome<?> rpc = removeFromMap(coordinationId);
-      rpc.setException(new RemoteRpcException(failure));
+      rpc.setException(new DrillRemoteException(failure));
     } catch(Exception ex) {
       logger.warn("Failed to remove from map.  Not a problem since we were updating on failed future.", ex);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteRpcException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteRpcException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteRpcException.java
deleted file mode 100644
index 14ea873..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteRpcException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc;
-
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.work.ErrorHelper;
-
-public class RemoteRpcException extends RpcException{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteRpcException.class);
-
-  private final DrillPBError failure;
-
-  public RemoteRpcException(DrillPBError failure) {
-    super(ErrorHelper.getErrorMessage(failure, false));
-    this.failure = failure;
-  }
-
-  @Override
-  public DrillPBError getRemoteError() {
-    return failure;
-  }
-
-  @Override
-  public boolean isRemote() {
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index b974963..79232c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -30,9 +30,10 @@ import java.io.Closeable;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.work.ErrorHelper;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.Internal.EnumLite;
@@ -193,8 +194,13 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
         try {
           handle(connection, msg.rpcType, msg.pBody, msg.dBody, sender);
         } catch(UserRpcException e){
-          DrillPBError error = ErrorHelper.logAndConvertError(e.getEndpoint(), e.getUserMessage(), e, logger);
-          OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE_FAILURE, 0, msg.coordinationId, error);
+          DrillUserException uex = ErrorHelper.wrap(e);
+          uex.getContext().add(e.getEndpoint());
+
+          logger.error("Unexpected Error while handling request message", e);
+
+          OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE_FAILURE, 0, msg.coordinationId,
+            uex.getOrCreatePBError(false));
           if (RpcConstants.EXTRA_DEBUGGING) {
             logger.debug("Adding message to outbound buffer. {}", outMessage);
           }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
index a1be83b..a03e2c0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
@@ -23,7 +23,9 @@ import io.netty.buffer.DrillBuf;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.common.exceptions.DrillRemoteException;
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
@@ -107,8 +109,7 @@ public class QueryResultHandler {
       if (isFailureResult) {
         // Failure case--pass on via submissionFailed(...).
 
-        String message = buildErrorMessage(queryResult);
-        resultsListener.submissionFailed(new RpcException(message));
+        resultsListener.submissionFailed(new DrillRemoteException(queryResult.getError(0)));
         // Note: Listener is removed in finally below.
       } else if (isTerminalResult) {
         // A successful completion/canceled case--pass on via resultArrived
@@ -116,7 +117,7 @@ public class QueryResultHandler {
         try {
           resultsListener.queryCompleted();
         } catch ( Exception e ) {
-          resultsListener.submissionFailed(new RpcException(e));
+          resultsListener.submissionFailed(ErrorHelper.wrap(e));
         }
       } else {
         logger.warn("queryState {} was ignored", queryState);
@@ -157,7 +158,7 @@ public class QueryResultHandler {
       // That releases batch if successful.
     } catch ( Exception e ) {
       batch.release();
-      resultsListener.submissionFailed(new RpcException(e));
+      resultsListener.submissionFailed(ErrorHelper.wrap(e));
     }
   }
 
@@ -189,18 +190,9 @@ public class QueryResultHandler {
     return resultsListener;
   }
 
-  protected String buildErrorMessage(QueryResult result) {
-    StringBuilder sb = new StringBuilder();
-    for (UserBitShared.DrillPBError error : result.getErrorList()) {
-      sb.append(error.getMessage());
-      sb.append("\n");
-    }
-    return sb.toString();
-  }
-
   private void failAll() {
     for (UserResultsListener l : queryIdToResultsListenersMap.values()) {
-      l.submissionFailed(new RpcException("Received result without QueryId"));
+      l.submissionFailed(ErrorHelper.wrap(new RpcException("Received result without QueryId")));
     }
   }
 
@@ -208,7 +200,7 @@ public class QueryResultHandler {
 
     private ConcurrentLinkedQueue<QueryDataBatch> results = Queues.newConcurrentLinkedQueue();
     private volatile boolean finished = false;
-    private volatile RpcException ex;
+    private volatile DrillUserException ex;
     private volatile UserResultsListener output;
     private volatile ConnectionThrottle throttle;
 
@@ -253,7 +245,7 @@ public class QueryResultHandler {
     }
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       finished = true;
       synchronized (this) {
         if (output == null) {
@@ -284,7 +276,7 @@ public class QueryResultHandler {
 
     @Override
     public void failed(RpcException ex) {
-      resultsListener.submissionFailed(ex);
+      resultsListener.submissionFailed(ErrorHelper.wrap(ex));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
index 934a094..b9f742c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.rpc.user;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.rpc.RpcException;
 
 public interface UserResultsListener {
 
@@ -33,7 +33,7 @@ public interface UserResultsListener {
    * {@link #dataArrived(QueryDataBatch, ConnectionThrottle) dataArrived()} throws an exception
    * @param ex exception describing the cause of the failure
    */
-  void submissionFailed(RpcException ex);
+  void submissionFailed(DrillUserException ex);
 
   /**
    * The query has completed (successsful completion or cancellation). The listener will not receive any other

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
index fbbf0b8..8415440 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -37,7 +38,6 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -118,7 +118,7 @@ public class QueryWrapper {
 
 
   private static class Listener implements UserResultsListener {
-    private volatile Exception exception;
+    private volatile DrillUserException exception;
     private final CountDownLatch latch = new CountDownLatch(1);
     private final BufferAllocator allocator;
     public final List<Map<String, String>> results = Lists.newArrayList();
@@ -129,7 +129,7 @@ public class QueryWrapper {
     }
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       exception = ex;
       logger.error("Query Failed", ex);
       latch.countDown();

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
index cc7cb83..91e0b21 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
@@ -22,7 +22,7 @@ import java.io.InputStream;
 import java.util.List;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ExecConstants;
@@ -30,6 +30,7 @@ import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.easy.json.reader.CountingJsonReader;
@@ -90,14 +91,27 @@ public class JSONRecordReader extends AbstractRecordReader {
     }
   }
 
-  protected void handleAndRaise(String msg, Exception e) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(msg).append(" - In ").append(hadoopPath.toUri().getPath()).append(" parser was at record: ").append(recordCount+1);
+  protected void handleAndRaise(String suffix, Exception e) throws DrillUserException {
+
+    String message = e.getMessage();
+    int columnNr = -1;
+
     if (e instanceof JsonParseException) {
-      JsonParseException ex = JsonParseException.class.cast(e);
-      sb.append(" column: ").append(ex.getLocation().getColumnNr());
+      JsonParseException ex = (JsonParseException) e;
+      message = ex.getOriginalMessage();
+      columnNr = ex.getLocation().getColumnNr();
+    }
+
+    DrillUserException.Builder builder = new DrillUserException.Builder(ErrorType.DATA_READ, e, "%s - %s", suffix, message);
+
+    // add context information
+    builder.add("Filename: " + hadoopPath.toUri().getPath());
+    builder.add("Record", recordCount + 1);
+    if (columnNr != -1) {
+      builder.add("Column", columnNr);
     }
-    throw new DrillRuntimeException(sb.toString(), e);
+
+    throw builder.build();
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java
deleted file mode 100644
index 0773d6c..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ErrorHelper.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.work;
-
-import java.util.UUID;
-import java.util.regex.Pattern;
-
-import org.apache.drill.common.exceptions.DrillException;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.proto.UserBitShared.ExceptionWrapper;
-import org.apache.drill.exec.proto.UserBitShared.StackTraceElementWrapper;
-import org.apache.drill.exec.rpc.RemoteRpcException;
-import org.slf4j.Logger;
-
-
-public class ErrorHelper {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ErrorHelper.class);
-
-  final static Pattern IGNORE= Pattern.compile("^(sun|com\\.sun|java).*");
-
-  /**
-   * Manages message conversion before returning to user.  If the exception is a remote rpc exception, will simply return user friendly message.  Otherwise, will log and return.
-   * TODO: this should really be done client side but we don't currently have any way to maintain session state on the client.
-   *
-   * @param endpoint
-   * @param message
-   * @param t
-   * @param logger
-   * @param verbose
-   * @return
-   */
-  public static DrillPBError logAndConvertMessageError(DrillbitEndpoint endpoint, String message, Throwable t, Logger logger, boolean verbose) {
-
-    DrillPBError baseError = t instanceof RemoteRpcException ? ((RemoteRpcException) t).getRemoteError() : logAndConvertError(endpoint, message, t, logger);
-    String userMessage = getErrorMessage(baseError, verbose);
-    return DrillPBError.newBuilder() //
-      .setEndpoint(baseError.getEndpoint()) //
-      .setErrorId(baseError.getErrorId()) //
-      .setMessage(userMessage) //
-      .build();
-  }
-
-  public static DrillPBError logAndConvertError(DrillbitEndpoint endpoint, String message, Throwable t, Logger logger) {
-    String id = UUID.randomUUID().toString();
-    DrillPBError.Builder builder = DrillPBError.newBuilder();
-    builder.setEndpoint(endpoint);
-    builder.setErrorId(id);
-    if(message != null){
-      builder.setMessage(message);
-    }
-    if(t == null){
-      t = new DrillException("Undefined failure occurred.");
-    }
-    builder.setException(getWrapper(t));
-
-    // record the error to the log for later reference.
-    logger.error("Error {}: {}", id, message, t);
-
-    return builder.build();
-  }
-
-  public static ExceptionWrapper getWrapper(Throwable ex){
-    return getWrapperBuilder(ex).build();
-  }
-
-  public static ExceptionWrapper.Builder getWrapperBuilder(Throwable ex){
-    return getWrapperBuilder(ex, false);
-  }
-
-  public static ExceptionWrapper.Builder getWrapperBuilder(Throwable ex, boolean includeAllStack){
-
-
-
-    ExceptionWrapper.Builder ew = ExceptionWrapper.newBuilder();
-    if(ex.getMessage() != null) {
-      ew.setMessage(ex.getMessage());
-    }
-    ew.setExceptionClass(ex.getClass().getCanonicalName());
-    boolean isHidden = false;
-    StackTraceElementWrapper[] wrappers = new StackTraceElementWrapper[ex.getStackTrace().length];
-    for(int i = 0; i < wrappers.length; i++){
-      StackTraceElement ele = ex.getStackTrace()[i];
-      if(include(ele, includeAllStack)){
-        if(isHidden){
-          isHidden = false;
-        }
-        ew.addStackTrace(getSTWrapper(ele));
-      }else{
-        if(!isHidden){
-          isHidden = true;
-          ew.addStackTrace(getEmptyST());
-        }
-      }
-
-    }
-
-    if(ex.getCause() != null && ex.getCause() != ex){
-      ew.setCause(getWrapper(ex.getCause()));
-    }
-    return ew;
-  }
-
-  private static boolean include(StackTraceElement ele, boolean includeAllStack){
-    if(includeAllStack) {
-      return true;
-    }
-    return !(IGNORE.matcher(ele.getClassName()).matches());
-  }
-
-  private static StackTraceElementWrapper.Builder getEmptyST(){
-    StackTraceElementWrapper.Builder w = StackTraceElementWrapper.newBuilder();
-    w.setClassName("...");
-    w.setIsNativeMethod(false);
-    w.setLineNumber(0);
-    w.setMethodName("...");
-    return w;
-  }
-  public static StackTraceElementWrapper.Builder getSTWrapper(StackTraceElement ele){
-    StackTraceElementWrapper.Builder w = StackTraceElementWrapper.newBuilder();
-    w.setClassName(ele.getClassName());
-    if(ele.getFileName() != null) {
-      w.setFileName(ele.getFileName());
-    }
-    w.setIsNativeMethod(ele.isNativeMethod());
-    w.setLineNumber(ele.getLineNumber());
-    w.setMethodName(ele.getMethodName());
-    return w;
-  }
-
-
-  public static String getErrorMessage(final DrillPBError error, final boolean verbose) {
-
-    String finalMessage = null;
-    ExceptionWrapper ex = error.getException();
-    StringBuilder sb = new StringBuilder();
-
-
-
-    sb //
-      .append("[ ") //
-      .append(error.getErrorId()) //
-      .append(" on ")
-      .append(error.getEndpoint().getAddress())
-      .append(":").append(error.getEndpoint().getUserPort())
-      .append(" ]\n");
-
-    boolean cause = false;
-    while(ex != null){
-
-      if(ex.hasMessage()){
-        finalMessage = ex.getMessage();
-      }
-
-      if(verbose){
-        sb.append("  ");
-
-        if(cause){
-          sb.append("Caused By ");
-        }
-
-        sb.append("(");
-        sb.append(ex.getExceptionClass());
-        sb.append(") ");
-        sb.append(ex.getMessage());
-        sb.append("\n");
-        for(int i = 0; i < ex.getStackTraceCount(); i++){
-          StackTraceElementWrapper st = ex.getStackTrace(i);
-          sb.append("    ");
-          sb.append(st.getClassName());
-          sb.append('.');
-          sb.append(st.getMethodName());
-          sb.append("():");
-          sb.append(st.getLineNumber());
-          sb.append("\n");
-        }
-        cause = true;
-      }
-
-      ex = ex.hasCause() ? ex.getCause() : null;
-
-
-    }
-
-    StringBuilder msg = new StringBuilder();
-
-    if (error.hasMessage()){
-      msg.append(error.getMessage());
-      if(finalMessage != null){
-        msg.append(", ");
-        msg.append(finalMessage);
-        msg.append(' ');
-      }
-    }else if(finalMessage != null){
-      msg.append(finalMessage);
-      msg.append(' ');
-    }else{
-      msg.append("Error ");
-    }
-
-    msg.append(sb);
-
-    return msg.toString();
-  }
-
-  public static void main(String[] args ){
-    DrillPBError e = logAndConvertError(DrillbitEndpoint.newBuilder().setAddress("host1").setControlPort(1234).build(), "RpcFailure", new Exception("Excep 1", new Exception("excep2")), logger);
-    System.out.println(getErrorMessage(e, false));
-    System.out.println("\n\n\n");
-    System.out.println(getErrorMessage(e, true));
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 23ef0d3..0df5145 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -28,9 +28,10 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Preconditions;
 
-import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.drill.common.EventProcessor;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
@@ -56,7 +57,6 @@ import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
@@ -71,7 +71,6 @@ import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.testing.ExceptionInjector;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.EndpointListener;
-import org.apache.drill.exec.work.ErrorHelper;
 import org.apache.drill.exec.work.QueryWorkUnit;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
@@ -500,7 +499,7 @@ public class Foreman implements Runnable {
    */
   private class ForemanResult implements AutoCloseable {
     private QueryState resultState = null;
-    private Exception resultException = null;
+    private DrillUserException resultException = null;
     private boolean isClosed = false;
 
     /**
@@ -533,7 +532,7 @@ public class Foreman implements Runnable {
       Preconditions.checkState(resultState == null);
 
       resultState = QueryState.FAILED;
-      resultException = exception;
+      addException(exception);
     }
 
     /**
@@ -546,7 +545,8 @@ public class Foreman implements Runnable {
       Preconditions.checkNotNull(exception);
 
       if (resultException == null) {
-        resultException = exception;
+        resultException = ErrorHelper.wrap(exception);
+        resultException.getContext().add(queryContext.getCurrentEndpoint());
       } else {
         resultException.addSuppressed(exception);
       }
@@ -614,9 +614,8 @@ public class Foreman implements Runnable {
           .setQueryId(queryId)
           .setQueryState(resultState);
       if (resultException != null) {
-        final DrillPBError error = ErrorHelper.logAndConvertError(queryContext.getCurrentEndpoint(),
-            ExceptionUtils.getRootCauseMessage(resultException), resultException, logger);
-        resultBuilder.addError(error);
+        boolean verbose = queryContext.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
+        resultBuilder.addError(resultException.getOrCreatePBError(verbose));
       }
 
       /*

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 8626d5b..dc60beb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.drill.common.exceptions.DrillRemoteException;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -39,7 +40,6 @@ import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.rpc.RemoteRpcException;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -160,7 +160,7 @@ public class QueryManager implements FragmentStatusListener, DrillbitStatusListe
       break;
 
     case FAILED:
-      stateListener.moveToState(QueryState.FAILED, new RemoteRpcException(status.getProfile().getError()));
+      stateListener.moveToState(QueryState.FAILED, new DrillRemoteException(status.getProfile().getError()));
       break;
 
     default:

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/SqlUnsupportedException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/SqlUnsupportedException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/SqlUnsupportedException.java
index 2299afa..e372283 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/SqlUnsupportedException.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/SqlUnsupportedException.java
@@ -47,10 +47,10 @@ public abstract class SqlUnsupportedException extends ForemanSetupException {
     super(errorMessage);
   }
 
-  public static void errorMessageToException(String errorMessage) throws SqlUnsupportedException {
+  public static void errorClassNameToException(String errorClassName) throws SqlUnsupportedException {
     UnsupportedOperatorCollector collector = new UnsupportedOperatorCollector();
     for(ExceptionType ex : ExceptionType.values()) {
-      if(errorMessage.startsWith(ex.toString())) {
+      if(errorClassName.endsWith(ex.toString())) {
         collector.setException(ex);
         collector.convertException();
         collector.clean();

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
index 1b0885d..8279876 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
@@ -17,13 +17,13 @@
  */
 package org.apache.drill.exec.work.fragment;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
-import org.apache.drill.exec.work.ErrorHelper;
 
 public abstract class AbstractStatusReporter implements StatusReporter{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStatusReporter.class);
@@ -37,17 +37,17 @@ public abstract class AbstractStatusReporter implements StatusReporter{
   }
 
   private  FragmentStatus.Builder getBuilder(FragmentState state){
-    return getBuilder(context, state, null, null);
+    return getBuilder(context, state, null);
   }
 
-  public static FragmentStatus.Builder getBuilder(FragmentContext context, FragmentState state, String message, Throwable t){
+  public static FragmentStatus.Builder getBuilder(FragmentContext context, FragmentState state, DrillUserException ex){
     FragmentStatus.Builder status = FragmentStatus.newBuilder();
     MinorFragmentProfile.Builder b = MinorFragmentProfile.newBuilder();
     context.getStats().addMetricsToStatus(b);
     b.setState(state);
-    if(t != null){
+    if(ex != null){
       boolean verbose = context.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
-      b.setError(ErrorHelper.logAndConvertMessageError(context.getIdentity(), message, t, logger, verbose));
+      b.setError(ex.getOrCreatePBError(verbose));
     }
     status.setHandle(context.getHandle());
     b.setMemoryUsed(context.getAllocator().getAllocatedMemory());
@@ -105,8 +105,8 @@ public abstract class AbstractStatusReporter implements StatusReporter{
   protected abstract void statusChange(FragmentHandle handle, FragmentStatus status);
 
   @Override
-  public final void fail(FragmentHandle handle, String message, Throwable excep) {
-    FragmentStatus.Builder status = getBuilder(context, FragmentState.FAILED, message, excep);
+  public final void fail(FragmentHandle handle, String message, DrillUserException excep) {
+    FragmentStatus.Builder status = getBuilder(context, FragmentState.FAILED, excep);
     fail(handle, status);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index a7e6c46..58f4256 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -21,6 +21,8 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.DeferredException;
+import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.ErrorHelper;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -83,9 +85,8 @@ public class FragmentExecutor implements Runnable {
     if(state.get() != FragmentState.RUNNING_VALUE) {
       return null;
     }
-    final FragmentStatus status =
-        AbstractStatusReporter.getBuilder(fragmentContext, FragmentState.RUNNING, null, null).build();
-    return status;
+
+    return AbstractStatusReporter.getBuilder(fragmentContext, FragmentState.RUNNING, null).build();
   }
 
   public void cancel() {
@@ -227,7 +228,11 @@ public class FragmentExecutor implements Runnable {
 
   private void internalFail(final Throwable excep) {
     state.set(FragmentState.FAILED_VALUE);
-    listener.fail(fragmentContext.getHandle(), "Failure while running fragment.", excep);
+
+    DrillUserException uex = ErrorHelper.wrap(excep);
+    uex.getContext().add(getContext().getIdentity());
+
+    listener.fail(fragmentContext.getHandle(), "Failure while running fragment.", uex);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
index 26b5d68..1699322 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.work.fragment;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 
@@ -24,6 +25,6 @@ import org.apache.drill.exec.proto.UserBitShared.FragmentState;
  * The status handler is responsible for receiving changes in fragment status and propagating them back to the foreman.
  */
 public interface StatusReporter {
-  void fail(FragmentHandle handle, String message, Throwable excep);
+  void fail(FragmentHandle handle, String message, DrillUserException excep);
   void stateChanged(FragmentHandle handle, FragmentState newState);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 0c2f0e5..6df5801 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -25,9 +25,8 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Preconditions;
-import com.google.common.io.Files;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
@@ -37,7 +36,6 @@ import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -45,9 +43,6 @@ import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
-import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.exec.util.TestUtilities;
 import org.apache.drill.exec.util.VectorUtil;
 import org.junit.AfterClass;
@@ -325,12 +320,12 @@ public class BaseTestQuery extends ExecTest {
   }
 
   private static class SilentListener implements UserResultsListener {
-    private volatile Exception exception;
+    private volatile DrillUserException exception;
     private AtomicInteger count = new AtomicInteger();
     private CountDownLatch latch = new CountDownLatch(1);
 
     @Override
-    public void submissionFailed(RpcException ex) {
+    public void submissionFailed(DrillUserException ex) {
       exception = ex;
       System.out.println("Query failed: " + ex.getMessage());
       latch.countDown();

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java b/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
index 5703bf9..715904d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
@@ -23,11 +23,11 @@ import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
@@ -49,10 +49,10 @@ public abstract class SingleRowListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(final RpcException ex) {
+  public void submissionFailed(final DrillUserException ex) {
     exception = ex;
     synchronized(errorList) {
-      errorList.add(ex.getRemoteError());
+      errorList.add(ex.getOrCreatePBError(false));
     }
     latch.countDown();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/99b6d0e6/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
index 875fb25..12b1787 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill;
 
-import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.common.exceptions.DrillUserException;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -73,13 +73,13 @@ public class TestBugFixes extends BaseTestQuery {
   }
 
 
-  @Test (expected = RpcException.class)
+  @Test (expected = DrillUserException.class)
   // Should be "Failure while parsing sql. Node [rel#26:Subset#6.LOGICAL.ANY([]).[]] could not be implemented;".
   // Drill will hit CanNotPlan, until we add code fix to transform the local LHS filter in left outer join properly.
   public void testDRILL1337_LocalLeftFilterLeftOutJoin() throws Exception {
     try {
       test("select count(*) from cp.`tpch/nation.parquet` n left outer join cp.`tpch/region.parquet` r on n.n_regionkey = r.r_regionkey and n.n_nationkey > 10;");
-    } catch (RpcException e) {
+    } catch (DrillUserException e) {
       logger.info("***** Test resulted in expected failure: " + e.getMessage());
       throw e;
     }


[3/6] drill git commit: DRILL-2788: fix NullableVV empty population logic

Posted by ja...@apache.org.
DRILL-2788: fix NullableVV empty population logic


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

Branch: refs/heads/master
Commit: 75455e809c34bbcc79da8a517e955f19f521a6a8
Parents: 99b6d0e
Author: Hanifi Gunes <hg...@maprtech.com>
Authored: Tue Apr 14 13:08:31 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed Apr 15 07:02:56 2015 -0700

----------------------------------------------------------------------
 .../src/main/codegen/templates/NullableValueVectors.java         | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/75455e80/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
index 075316e..b4b837f 100644
--- a/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/java-exec/src/main/codegen/templates/NullableValueVectors.java
@@ -476,8 +476,8 @@ public final class ${className} extends BaseDataValueVector implements <#if type
 
     <#if type.major == "VarLen">
     private void fillEmpties(int index){
-      for (int i = lastSet + 1; i < index; i++) {
-        values.getMutator().setSafe(i, new byte[]{});
+      for (int i = lastSet; i < index; i++) {
+        values.getMutator().setSafe(i+1, new byte[]{});
       }
       if (index > bits.getValueCapacity()) {
         bits.reAlloc();


[4/6] drill git commit: DRILL-2695: Add Support for large in conditions through the use of the Values operator. Update JSON reader to support reading Extended JSON. Update JSON writer to support writing extended JSON data. Update JSON reader to automatic

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
index de52b73..8309bf3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
@@ -32,11 +32,19 @@ public class JsonWriter {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonWriter.class);
 
   private final JsonFactory factory = new JsonFactory();
-  private final JsonGenerator gen;
+  private final JsonOutput gen;
 
-  public JsonWriter(OutputStream out, boolean pretty) throws IOException{
+  public JsonWriter(OutputStream out, boolean pretty, boolean useExtendedOutput) throws IOException{
     JsonGenerator writer = factory.createJsonGenerator(out);
-    gen = pretty ? writer.useDefaultPrettyPrinter() : writer;
+    if(pretty){
+      writer = writer.useDefaultPrettyPrinter();
+    }
+    if(useExtendedOutput){
+      gen = new ExtendedJsonOutput(writer);
+    }else{
+      gen = new BasicJsonOutput(writer);
+    }
+
   }
 
   public void write(FieldReader reader) throws JsonGenerationException, IOException{
@@ -50,58 +58,45 @@ public class JsonWriter {
 
     switch(m){
     case OPTIONAL:
-      if(!reader.isSet()){
-        gen.writeNull();
-        break;
-      }
-
     case REQUIRED:
 
 
       switch (mt) {
       case FLOAT4:
-        gen.writeNumber(reader.readFloat());
+        gen.writeFloat(reader);
         break;
       case FLOAT8:
-        gen.writeNumber(reader.readDouble());
+        gen.writeDouble(reader);
         break;
       case INT:
-        Integer i = reader.readInteger();
-        if(i == null){
-          gen.writeNull();
-        }else{
-          gen.writeNumber(reader.readInteger());
-        }
+        gen.writeInt(reader);
         break;
       case SMALLINT:
-        gen.writeNumber(reader.readShort());
+        gen.writeSmallInt(reader);
         break;
       case TINYINT:
-        gen.writeNumber(reader.readByte());
+        gen.writeTinyInt(reader);
         break;
       case BIGINT:
-        Long l = reader.readLong();
-        if(l == null){
-          gen.writeNull();
-        }else{
-          gen.writeNumber(reader.readLong());
-        }
-
+        gen.writeBigInt(reader);
         break;
       case BIT:
-        gen.writeBoolean(reader.readBoolean());
+        gen.writeBoolean(reader);
         break;
 
       case DATE:
+        gen.writeDate(reader);
+        break;
       case TIME:
+        gen.writeTime(reader);
+        break;
       case TIMESTAMP:
-      case TIMESTAMPTZ:
-        gen.writeString(reader.readDateTime().toString());
-
+        gen.writeTimestamp(reader);
+        break;
       case INTERVALYEAR:
       case INTERVALDAY:
       case INTERVAL:
-        gen.writeString(reader.readPeriod().toString());
+        gen.writeInterval(reader);
         break;
       case DECIMAL28DENSE:
       case DECIMAL28SPARSE:
@@ -109,7 +104,7 @@ public class JsonWriter {
       case DECIMAL38SPARSE:
       case DECIMAL9:
       case DECIMAL18:
-        gen.writeNumber(reader.readBigDecimal());
+        gen.writeDecimal(reader);
         break;
 
       case LIST:
@@ -130,17 +125,17 @@ public class JsonWriter {
         gen.writeEndObject();
         break;
       case NULL:
-        gen.writeNull();
+        gen.writeUntypedNull();
         break;
 
       case VAR16CHAR:
-        gen.writeString(reader.readString());
+        gen.writeVar16Char(reader);
         break;
       case VARBINARY:
-        gen.writeBinary(reader.readByteArray());
+        gen.writeBinary(reader);
         break;
       case VARCHAR:
-        gen.writeString(reader.readText().toString());
+        gen.writeVarChar(reader);
         break;
 
       }
@@ -151,54 +146,61 @@ public class JsonWriter {
       switch (mt) {
       case FLOAT4:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readFloat(i));
+          gen.writeFloat(i, reader);
         }
 
         break;
       case FLOAT8:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readDouble(i));
+          gen.writeDouble(i, reader);
         }
         break;
       case INT:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readInteger(i));
+          gen.writeInt(i, reader);
         }
         break;
       case SMALLINT:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readShort(i));
+          gen.writeSmallInt(i, reader);
         }
         break;
       case TINYINT:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readByte(i));
+          gen.writeTinyInt(i, reader);
         }
         break;
       case BIGINT:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeNumber(reader.readLong(i));
+          gen.writeBigInt(i, reader);
         }
         break;
       case BIT:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeBoolean(reader.readBoolean(i));
+          gen.writeBoolean(i, reader);
         }
         break;
 
       case DATE:
+        for(int i = 0; i < reader.size(); i++){
+          gen.writeDate(i, reader);
+        }
+        break;
       case TIME:
+        for(int i = 0; i < reader.size(); i++){
+          gen.writeTime(i, reader);
+        }
+        break;
       case TIMESTAMP:
-      case TIMESTAMPTZ:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeString(reader.readDateTime(i).toString());
+          gen.writeTimestamp(i, reader);
         }
-
+        break;
       case INTERVALYEAR:
       case INTERVALDAY:
       case INTERVAL:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeString(reader.readPeriod(i).toString());
+          gen.writeInterval(i, reader);
         }
         break;
       case DECIMAL28DENSE:
@@ -208,7 +210,7 @@ public class JsonWriter {
       case DECIMAL9:
       case DECIMAL18:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeNumber(reader.readBigDecimal(i));
+          gen.writeDecimal(i, reader);
         }
         break;
 
@@ -237,20 +239,22 @@ public class JsonWriter {
 
       case VAR16CHAR:
         for(int i = 0; i < reader.size(); i++){
-          gen.writeString(reader.readString(i));
+          gen.writeVar16Char(i, reader);
         }
         break;
       case VARBINARY:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeBinary(reader.readByteArray(i));
+          gen.writeBinary(i, reader);
         }
         break;
       case VARCHAR:
         for(int i = 0; i < reader.size(); i++){
-        gen.writeString(reader.readText(i).toString());
+          gen.writeVarChar(i, reader);
         }
         break;
 
+      default:
+        throw new IllegalStateException(String.format("Unable to handle type %s.", mt));
       }
       gen.writeEndArray();
       break;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/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
new file mode 100644
index 0000000..651de3d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
@@ -0,0 +1,295 @@
+/**
+ * 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.complex.fn;
+
+import java.io.IOException;
+
+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;
+import org.apache.drill.exec.expr.holders.Decimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.IntervalHolder;
+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.complex.writer.BaseWriter.ListWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
+import org.apache.drill.exec.vector.complex.writer.DateWriter;
+import org.apache.drill.exec.vector.complex.writer.IntervalWriter;
+import org.apache.drill.exec.vector.complex.writer.TimeStampWriter;
+import org.apache.drill.exec.vector.complex.writer.TimeWriter;
+import org.apache.drill.exec.vector.complex.writer.VarBinaryWriter;
+import org.joda.time.DateTime;
+import org.joda.time.Period;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.ISODateTimeFormat;
+import org.joda.time.format.ISOPeriodFormat;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
+
+abstract class VectorOutput {
+
+  final VarBinaryHolder binary = new VarBinaryHolder();
+  final TimeHolder time = new TimeHolder();
+  final DateHolder date = new DateHolder();
+  final TimeStampHolder timestamp = new TimeStampHolder();
+  final IntervalHolder interval = new IntervalHolder();
+  final BigIntHolder bigint = new BigIntHolder();
+  final Decimal38DenseHolder decimal = new Decimal38DenseHolder();
+  final VarCharHolder varchar = new VarCharHolder();
+
+  protected final WorkingBuffer work;
+  protected JsonParser parser;
+
+
+  public VectorOutput(WorkingBuffer work){
+    this.work = work;
+  }
+
+  public void setParser(JsonParser parser){
+    this.parser = parser;
+  }
+
+  protected boolean innerRun() throws IOException{
+    JsonToken t = parser.nextToken();
+    if(t != JsonToken.FIELD_NAME){
+      return false;
+    }
+
+    String possibleTypeName = parser.getText();
+    if(!possibleTypeName.isEmpty() && possibleTypeName.charAt(0) == '$'){
+      switch(possibleTypeName){
+      case ExtendedTypeName.BINARY:
+        writeBinary(checkNextToken(JsonToken.VALUE_STRING));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.DATE:
+        writeDate(checkNextToken(JsonToken.VALUE_STRING));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.TIME:
+        writeTime(checkNextToken(JsonToken.VALUE_STRING));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.TIMESTAMP:
+        writeTimestamp(checkNextToken(JsonToken.VALUE_STRING));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.INTERVAL:
+        writeInterval(checkNextToken(JsonToken.VALUE_STRING));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.INTEGER:
+        writeInteger(checkNextToken(JsonToken.VALUE_NUMBER_INT));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      case ExtendedTypeName.DECIMAL:
+        writeDecimal(checkNextToken(JsonToken.VALUE_NUMBER_FLOAT, JsonToken.VALUE_NUMBER_INT));
+        checkNextToken(JsonToken.END_OBJECT);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public boolean checkNextToken(final JsonToken expected) throws IOException{
+    return checkNextToken(expected, expected);
+  }
+
+  public boolean checkNextToken(final JsonToken expected1, final JsonToken expected2) throws IOException{
+    JsonToken t = parser.nextToken();
+    if(t == JsonToken.VALUE_NULL){
+      return true;
+    }else if(t == expected1){
+      return false;
+    }else if(t == expected2){
+      return false;
+    }else{
+      throw new JsonParseException(String.format("Failure while reading ExtendedJSON typed value. Expected a %s but "
+          + "received a token of type %s", expected1, t), parser.getCurrentLocation());
+    }
+  }
+
+  public abstract void writeBinary(boolean isNull) throws IOException;
+  public abstract void writeDate(boolean isNull) throws IOException;
+  public abstract void writeTime(boolean isNull) throws IOException;
+  public abstract void writeTimestamp(boolean isNull) throws IOException;
+  public abstract void writeInterval(boolean isNull) throws IOException;
+  public abstract void writeInteger(boolean isNull) throws IOException;
+  public abstract void writeDecimal(boolean isNull) throws IOException;
+
+  static class ListVectorOutput extends VectorOutput{
+    private ListWriter writer;
+
+    public ListVectorOutput(WorkingBuffer work) {
+      super(work);
+    }
+
+    public boolean run(ListWriter writer) throws IOException{
+      this.writer = writer;
+      return innerRun();
+    }
+
+    @Override
+    public void writeBinary(boolean isNull) throws IOException {
+      VarBinaryWriter bin = writer.varBinary();
+      if(!isNull){
+        work.prepareBinary(parser.getBinaryValue(), binary);
+        bin.write(binary);
+      }
+    }
+
+    @Override
+    public void writeDate(boolean isNull) throws IOException {
+      DateWriter dt = writer.date();
+      if(!isNull){
+        work.prepareVarCharHolder(parser.getValueAsString(), varchar);
+        dt.writeDate(StringFunctionHelpers.getDate(varchar.buffer, varchar.start, varchar.end));
+      }
+    }
+
+    @Override
+    public void writeTime(boolean isNull) throws IOException {
+      TimeWriter t = writer.time();
+      if(!isNull){
+        DateTimeFormatter f = ISODateTimeFormat.time();
+        t.writeTime((int) ((f.parseDateTime(parser.getValueAsString())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
+      }
+    }
+
+    @Override
+    public void writeTimestamp(boolean isNull) throws IOException {
+      TimeStampWriter ts = writer.timeStamp();
+      if(!isNull){
+        DateTimeFormatter f = ISODateTimeFormat.dateTime();
+        ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+      }
+    }
+
+    @Override
+    public void writeInterval(boolean isNull) throws IOException {
+      IntervalWriter intervalWriter = writer.interval();
+      if(!isNull){
+        final Period p = ISOPeriodFormat.standard().parsePeriod(parser.getValueAsString());
+        int months = DateUtility.monthsFromPeriod(p);
+        int days = p.getDays();
+        int millis = DateUtility.millisFromPeriod(p);
+        intervalWriter.writeInterval(months, days, millis);
+      }
+    }
+
+    @Override
+    public void writeInteger(boolean isNull) throws IOException {
+      BigIntWriter intWriter = writer.bigInt();
+      if(!isNull){
+        intWriter.writeBigInt(parser.getLongValue());
+      }
+    }
+
+    @Override
+    public void writeDecimal(boolean isNull) throws IOException {
+      throw new JsonParseException("Decimal Extended types not yet supported.", parser.getCurrentLocation());
+    }
+
+  }
+
+  static class MapVectorOutput extends VectorOutput {
+
+    private MapWriter writer;
+    private String fieldName;
+
+    public MapVectorOutput(WorkingBuffer work) {
+      super(work);
+    }
+
+    public boolean run(MapWriter writer, String fieldName) throws IOException{
+      this.fieldName = fieldName;
+      this.writer = writer;
+      return innerRun();
+    }
+
+    @Override
+    public void writeBinary(boolean isNull) throws IOException {
+      VarBinaryWriter bin = writer.varBinary(fieldName);
+      if(!isNull){
+        work.prepareBinary(parser.getBinaryValue(), binary);
+        bin.write(binary);
+      }
+    }
+
+    @Override
+    public void writeDate(boolean isNull) throws IOException {
+      DateWriter dt = writer.date(fieldName);
+      if(!isNull){
+        DateTimeFormatter f = ISODateTimeFormat.date();
+        DateTime date = f.parseDateTime(parser.getValueAsString());
+        dt.writeDate(date.withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+      }
+    }
+
+    @Override
+    public void writeTime(boolean isNull) throws IOException {
+      TimeWriter t = writer.time(fieldName);
+      if(!isNull){
+        DateTimeFormatter f = ISODateTimeFormat.time();
+        t.writeTime((int) ((f.parseDateTime(parser.getValueAsString())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis()));
+      }
+    }
+
+    @Override
+    public void writeTimestamp(boolean isNull) throws IOException {
+      TimeStampWriter ts = writer.timeStamp(fieldName);
+      if(!isNull){
+        DateTimeFormatter f = ISODateTimeFormat.dateTime();
+        ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+      }
+    }
+
+    @Override
+    public void writeInterval(boolean isNull) throws IOException {
+      IntervalWriter intervalWriter = writer.interval(fieldName);
+      if(!isNull){
+        final Period p = ISOPeriodFormat.standard().parsePeriod(parser.getValueAsString());
+        int months = DateUtility.monthsFromPeriod(p);
+        int days = p.getDays();
+        int millis = DateUtility.millisFromPeriod(p);
+        intervalWriter.writeInterval(months, days, millis);
+      }
+    }
+
+    @Override
+    public void writeInteger(boolean isNull) throws IOException {
+      BigIntWriter intWriter = writer.bigInt(fieldName);
+      if(!isNull){
+        intWriter.writeBigInt(parser.getLongValue());
+      }
+    }
+
+    @Override
+    public void writeDecimal(boolean isNull) throws IOException {
+      throw new IOException("Decimal Extended types not yet supported.");
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/WorkingBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/WorkingBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/WorkingBuffer.java
new file mode 100644
index 0000000..7d10d3b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/WorkingBuffer.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.vector.complex.fn;
+
+import io.netty.buffer.DrillBuf;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+
+import com.google.common.base.Charsets;
+
+class WorkingBuffer {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkingBuffer.class);
+
+  private DrillBuf workBuf;
+
+  public WorkingBuffer(DrillBuf workBuf) {
+    this.workBuf = workBuf;
+  }
+
+  private void ensure(int length) {
+    workBuf = workBuf.reallocIfNeeded(length);
+  }
+
+  public void prepareVarCharHolder(String value, VarCharHolder h) throws IOException {
+    byte[] b = value.getBytes(Charsets.UTF_8);
+    ensure(b.length);
+    workBuf.setBytes(0, b);
+    h.start = 0;
+    h.end = b.length;
+    h.buffer = workBuf;
+  }
+
+  public int prepareVarCharHolder(String value) throws IOException {
+    byte[] b = value.getBytes(Charsets.UTF_8);
+    ensure(b.length);
+    workBuf.setBytes(0, b);
+    return b.length;
+  }
+
+  public void prepareBinary(byte[] b, VarBinaryHolder h) throws IOException {
+    ensure(b.length);
+    workBuf.setBytes(0, b);
+    h.start = 0;
+    h.end = b.length;
+    h.buffer = workBuf;
+  }
+
+  public DrillBuf getBuf(){
+    return workBuf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java
index 5d85d0a..ec8c00b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/AbstractBaseWriter.java
@@ -17,20 +17,17 @@
  */
 package org.apache.drill.exec.vector.complex.impl;
 
-import org.apache.drill.exec.vector.complex.WriteState;
 import org.apache.drill.exec.vector.complex.writer.FieldWriter;
 
 
 abstract class AbstractBaseWriter implements FieldWriter{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBaseWriter.class);
 
-  final WriteState state;
   final FieldWriter parent;
   private int index;
 
   public AbstractBaseWriter(FieldWriter parent) {
     super();
-    this.state = parent == null ? new WriteState() : parent.getState();
     this.parent = parent;
   }
 
@@ -38,10 +35,6 @@ abstract class AbstractBaseWriter implements FieldWriter{
     return parent;
   }
 
-  public boolean ok(){
-    return state.isOk();
-  }
-
   public boolean isRoot(){
     return parent == null;
   }
@@ -50,24 +43,10 @@ abstract class AbstractBaseWriter implements FieldWriter{
     return index;
   }
 
-  public void resetState(){
-    state.reset();
-  }
-
   public void setPosition(int index){
     this.index = index;
   }
 
-  void inform(boolean outcome){
-    if(!outcome){
-      state.fail(this);
-    }
-  }
-
-  public WriteState getState(){
-    return state;
-  }
-
   public void end(){
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
index a110dcc..a4a35e2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
@@ -56,17 +56,12 @@ public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWri
     return container.getValueCapacity();
   }
 
-  public void checkValueCapacity(){
-    inform(container.getValueCapacity() > idx());
-  }
-
   private void check(Mode... modes){
     StateTool.check(mode, modes);
   }
 
   public void reset(){
     setPosition(0);
-    resetState();
   }
 
   public void clear(){

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedListReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedListReaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedListReaderImpl.java
index 3d0a861..767c366 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedListReaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedListReaderImpl.java
@@ -48,7 +48,7 @@ public class RepeatedListReaderImpl extends AbstractFieldReader{
 
   @Override
   public void copyAsValue(ListWriter writer) {
-    if (currentOffset == NO_VALUES || writer.ok() == false) {
+    if (currentOffset == NO_VALUES) {
       return;
     }
     RepeatedListWriter impl = (RepeatedListWriter) writer;
@@ -57,7 +57,7 @@ public class RepeatedListReaderImpl extends AbstractFieldReader{
 
   @Override
   public void copyAsField(String name, MapWriter writer) {
-    if (currentOffset == NO_VALUES || writer.ok() == false) {
+    if (currentOffset == NO_VALUES) {
       return;
     }
     RepeatedListWriter impl = (RepeatedListWriter) writer.list(name);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedMapReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedMapReaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedMapReaderImpl.java
index aa98818..71a5017 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedMapReaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/RepeatedMapReaderImpl.java
@@ -160,7 +160,7 @@ public class RepeatedMapReaderImpl extends AbstractFieldReader{
 
   @Override
   public void copyAsValue(MapWriter writer) {
-    if (currentOffset == NO_VALUES || writer.ok() == false) {
+    if (currentOffset == NO_VALUES) {
       return;
     }
     RepeatedMapWriter impl = (RepeatedMapWriter) writer;
@@ -168,7 +168,7 @@ public class RepeatedMapReaderImpl extends AbstractFieldReader{
   }
 
   public void copyAsValueSingle(MapWriter writer) {
-    if (currentOffset == NO_VALUES || writer.ok() == false) {
+    if (currentOffset == NO_VALUES) {
       return;
     }
     SingleMapWriter impl = (SingleMapWriter) writer;

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/SingleMapReaderImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/SingleMapReaderImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/SingleMapReaderImpl.java
index ec6009e..1b39775 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/SingleMapReaderImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/SingleMapReaderImpl.java
@@ -93,18 +93,14 @@ public class SingleMapReaderImpl extends AbstractFieldReader{
 
   @Override
   public void copyAsValue(MapWriter writer){
-    if (writer.ok()) {
-      SingleMapWriter impl = (SingleMapWriter) writer;
-      impl.container.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    SingleMapWriter impl = (SingleMapWriter) writer;
+    impl.container.copyFromSafe(idx(), impl.idx(), vector);
   }
 
   @Override
   public void copyAsField(String name, MapWriter writer){
-    if (writer.ok()) {
-      SingleMapWriter impl = (SingleMapWriter) writer.map(name);
-      impl.container.copyFromSafe(idx(), impl.idx(), vector);
-    }
+    SingleMapWriter impl = (SingleMapWriter) writer.map(name);
+    impl.container.copyFromSafe(idx(), impl.idx(), vector);
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
index c3c9354..6b6ab46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/VectorContainerWriter.java
@@ -48,17 +48,12 @@ public class VectorContainerWriter extends AbstractFieldWriter implements Comple
     return mapRoot.getValueCapacity();
   }
 
-  public void checkValueCapacity(){
-    inform(getValueCapacity() > idx());
-  }
-
   public MapVector getMapVector() {
     return mapVector;
   }
 
   public void reset() {
     setPosition(0);
-    resetState();
   }
 
   public void clear() {

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestLargeInClause.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestLargeInClause.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestLargeInClause.java
new file mode 100644
index 0000000..1e29214
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestLargeInClause.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.filter;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestLargeInClause extends BaseTestQuery {
+
+  private static String getInIntList(int size){
+    StringBuffer sb = new StringBuffer();
+    for(int i =0; i < size; i++){
+      if(i != 0){
+        sb.append(", ");
+      }
+      sb.append(i);
+    }
+    return sb.toString();
+  }
+
+  private static String getInDateList(int size){
+    StringBuffer sb = new StringBuffer();
+    for(int i =0; i < size; i++){
+      if(i != 0){
+        sb.append(", ");
+      }
+      sb.append("DATE '1961-08-26'");
+    }
+    return sb.toString();
+  }
+
+  @Test
+  public void queryWith300InConditions() throws Exception {
+    test("select * from cp.`employee.json` where id in (" + getInIntList(300) + ")");
+  }
+
+  @Test
+  public void queryWith50000InConditions() throws Exception {
+    test("select * from cp.`employee.json` where id in (" + getInIntList(50000) + ")");
+  }
+
+  @Test
+  public void queryWith50000DateInConditions() throws Exception {
+    test("select * from cp.`employee.json` where cast(birth_date as date) in (" + getInDateList(500) + ")");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java
index c4bfcce..8b09e80 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/TestJsonRecordReader.java
@@ -34,6 +34,11 @@ public class TestJsonRecordReader extends BaseTestQuery{
   }
 
   @Test
+  public void testContainingArray() throws Exception {
+    test("select * from dfs.`${WORKING_PATH}/src/test/resources/store/json/listdoc.json`");
+  }
+
+  @Test
   public void testComplexMultipleTimes() throws Exception{
     for(int i =0 ; i < 5; i++){
     test("select * from cp.`join/merge_join.json`");

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
new file mode 100644
index 0000000..9d0af41
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.vector.complex.writer;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.util.TestTools;
+import org.junit.Test;
+
+public class TestExtendedTypes extends BaseTestQuery {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExtendedTypes.class);
+
+  @Test
+  public void checkReadWriteExtended() throws Exception {
+
+    final String originalFile = "${WORKING_PATH}/src/test/resources/vector/complex/extended.json".replaceAll(
+        Pattern.quote("${WORKING_PATH}"),
+        Matcher.quoteReplacement(TestTools.getWorkingPath()));
+
+    final String newTable = "TestExtendedTypes/newjson";
+    testNoResult("ALTER SESSION SET `store.format` = 'json'");
+
+    // create table
+    test("create table dfs_test.tmp.`%s` as select * from dfs.`%s`", newTable, originalFile);
+
+    // check query of table.
+    test("select * from dfs_test.tmp.`%s`", newTable);
+
+    // check that original file and new file match.
+    final byte[] originalData = Files.readAllBytes(Paths.get(originalFile));
+    final byte[] newData = Files.readAllBytes(Paths.get(BaseTestQuery.getDfsTestTmpSchemaLocation() + '/' + newTable
+        + "/0_0_0.json"));
+    assertEquals(new String(originalData), new String(newData));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
index 098c7de..6e2a2b5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
@@ -184,7 +184,6 @@ public class TestRepeated {
 
       map.end();
     }
-    assert writer.ok();
 
     {
       writer.setPosition(1);
@@ -247,7 +246,7 @@ public class TestRepeated {
 
 
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    JsonWriter jsonWriter = new JsonWriter(stream, true);
+    JsonWriter jsonWriter = new JsonWriter(stream, true, true);
     FieldReader reader = v.getChild("col", MapVector.class).getReader();
     reader.setPosition(0);
     jsonWriter.write(reader);

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/resources/store/json/listdoc.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/json/listdoc.json b/exec/java-exec/src/test/resources/store/json/listdoc.json
new file mode 100644
index 0000000..3b35e7a
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/json/listdoc.json
@@ -0,0 +1,4 @@
+[
+  {a: 4, b:6},
+  {a: 5, b:7}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/314e5a2a/exec/java-exec/src/test/resources/vector/complex/extended.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/extended.json b/exec/java-exec/src/test/resources/vector/complex/extended.json
new file mode 100644
index 0000000..16f3528
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/extended.json
@@ -0,0 +1,41 @@
+{
+  "bin" : {
+    "$binary" : "ZHJpbGw="
+  },
+  "drill_date" : {
+    "$dateDay" : "1997-07-16"
+  },
+  "drill_timestamp" : {
+    "$date" : "2009-02-23T08:00:00.000Z"
+  },
+  "time" : {
+    "$time" : "19:20:30.450Z"
+  },
+  "interval" : {
+    "$interval" : "PT26.400S"
+  },
+  "integer" : {
+    "$numberLong" : 4
+  },
+  "inner" : {
+    "bin" : {
+      "$binary" : "ZHJpbGw="
+    },
+    "drill_date" : {
+      "$dateDay" : "1997-07-16"
+    },
+    "drill_timestamp" : {
+      "$date" : "2009-02-23T08:00:00.000Z"
+    },
+    "time" : {
+      "$time" : "19:20:30.450Z"
+    },
+    "interval" : {
+      "$interval" : "PT26.400S"
+    },
+    "integer" : {
+      "$numberLong" : 4
+    }
+  },
+  "other1" : 3.3
+}
\ No newline at end of file