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

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

Repository: drill
Updated Branches:
  refs/heads/master 64e3ec52b -> 238399de5


http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 12b1787..c627ff2 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.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -73,13 +73,13 @@ public class TestBugFixes extends BaseTestQuery {
   }
 
 
-  @Test (expected = DrillUserException.class)
+  @Test (expected = UserException.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 (DrillUserException e) {
+    } catch (UserException e) {
       logger.info("***** Test resulted in expected failure: " + e.getMessage());
       throw e;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 111c3c1..e049943 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.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 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,27 +26,27 @@ import org.junit.Test;
 public class TestDisabledFunctionality extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExampleQueries.class);
 
-  @Test(expected = DrillUserException.class)  // see DRILL-2054
+  @Test(expected = UserException.class)  // see DRILL-2054
   public void testBooleanORExpression() throws Exception {
         test("select (1 = 1) || (1 > 0) from cp.`tpch/nation.parquet` ");
     }
 
-  @Test(expected = DrillUserException.class)  // see DRILL-2054
+  @Test(expected = UserException.class)  // see DRILL-2054
   public void testBooleanORSelectClause() throws Exception {
     test("select true || true from cp.`tpch/nation.parquet` ");
   }
 
-  @Test(expected = DrillUserException.class)  // see DRILL-2054
+  @Test(expected = UserException.class)  // see DRILL-2054
   public void testBooleanORWhereClause() throws Exception {
     test("select * from cp.`tpch/nation.parquet` where (true || true) ");
   }
 
-  @Test(expected = DrillUserException.class)  // see DRILL-2054
+  @Test(expected = UserException.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 {
+  private static void throwAsUnsupportedException(UserException ex) throws Exception {
     SqlUnsupportedException.errorClassNameToException(ex.getOrCreatePBError(false).getException().getExceptionClass());
     throw ex;
   }
@@ -58,7 +58,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
            "where n_nationkey = " +
            "(select r_regionkey from cp.`tpch/region.parquet` " +
            "where r_regionkey = 1)");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -70,7 +70,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
            "where n_nationkey = " +
            "(select r_regionkey from cp.`tpch/region.parquet` " +
            "where r_regionkey = 1)");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -79,7 +79,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -88,7 +88,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -97,7 +97,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -106,7 +106,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -115,7 +115,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -124,7 +124,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -133,7 +133,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -142,7 +142,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -151,7 +151,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastTINYINT() throws Exception {
     try {
       test("select cast(n_name as tinyint) from cp.`tpch/nation.parquet`;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -160,7 +160,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastSMALLINT() throws Exception {
     try {
       test("select cast(n_name as smallint) from cp.`tpch/nation.parquet`;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -169,7 +169,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCastREAL() throws Exception {
     try {
       test("select cast(n_name as real) from cp.`tpch/nation.parquet`;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -178,7 +178,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
   public void testDisabledCardinality() throws Exception {
     try {
       test("select cardinality(employee_id) from cp.`employee.json`;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -188,7 +188,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
     try {
       test("select a.*, b.user_port " +
           "from cp.`employee.json` a, sys.drillbits b;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -199,7 +199,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -210,7 +210,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -221,7 +221,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -232,7 +232,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -245,7 +245,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
           "where b.n_nationkey = " +
           "(select r_regionkey from cp.`tpch/region.parquet` " +
           "where r_regionkey = 1)");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -256,7 +256,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -267,7 +267,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -278,7 +278,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -289,7 +289,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }
@@ -301,7 +301,7 @@ public class TestDisabledFunctionality extends BaseTestQuery{
           "OVER (PARTITION BY position_id order by position_id) " +
           "FROM cp.`employee.json` " +
           "order by employee_id;");
-    } catch(DrillUserException ex) {
+    } catch(UserException ex) {
       throwAsUnsupportedException(ex);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 3dfaf10..365acd9 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,7 +17,7 @@
  */
 package org.apache.drill;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.TestTools;
 import org.junit.Test;
@@ -263,11 +263,11 @@ public class TestStarQueries extends BaseTestQuery{
     test("select *, first_name, *, last_name from cp.`employee.json`;");
   }
 
-  @Test(expected = DrillUserException.class)  // Should get "At line 1, column 8: Column 'n_nationkey' is ambiguous"
+  @Test(expected = UserException.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 (DrillUserException e) {
+    } catch (UserException e) {
       logger.info("***** Test resulted in expected failure: " + e.getMessage());
       throw e;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 3e00423..7749a2a 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,7 +17,7 @@
  */
 package org.apache.drill;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
@@ -227,7 +227,7 @@ 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(DrillUserException ex) {
+    } catch(UserException ex) {
       SqlUnsupportedException.errorClassNameToException(ex.getOrCreatePBError(false).getException().getExceptionClass());
       throw ex;
     } finally {
@@ -356,7 +356,7 @@ public class TestUnionAll extends BaseTestQuery{
         .build().run();
   }
 
-  @Test(expected = DrillUserException.class) // see DRILL-2590
+  @Test(expected = UserException.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/238399de/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 69c274c..7aee6d3 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,7 +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.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
@@ -312,7 +312,7 @@ public class TestDrillbitResilience extends ExecTest {
    * @param desc the expected exception site description
    */
   private static void assertInjected(
-      final DrillUserException caught, final Class<? extends Throwable> exceptionClass, final String desc) {
+      final UserException caught, final Class<? extends Throwable> exceptionClass, final String desc) {
     ExceptionWrapper cause = caught.getOrCreatePBError(false).getException();
     assertEquals(exceptionClass.getName(), cause.getExceptionClass());
     assertEquals(desc, cause.getMessage());
@@ -339,7 +339,7 @@ public class TestDrillbitResilience extends ExecTest {
     try {
       QueryTestUtil.test(drillClient, "select * from sys.drillbits");
       fail();
-    } catch(DrillUserException dre) {
+    } catch(UserException dre) {
       assertInjected(dre, ForemanException.class, desc);
     }
   }
@@ -365,7 +365,7 @@ public class TestDrillbitResilience extends ExecTest {
     try {
       QueryTestUtil.test(drillClient, "select * from sys.drillbits");
       fail();
-    } catch(DrillUserException dre) {
+    } catch(UserException dre) {
       assertInjected(dre, ForemanException.class, exceptionDesc);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 3bcfbdf..3a794a9 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,8 +23,7 @@ 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.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -62,7 +61,7 @@ public class ParquetResultListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(DrillUserException ex) {
+  public void submissionFailed(UserException ex) {
     logger.error("Submission failed.", ex);
     future.setException(ex);
   }
@@ -170,7 +169,7 @@ public class ParquetResultListener implements UserResultsListener {
         }
         assertEquals("Record count incorrect for column: " + s, totalRecords, (long) valuesChecked.get(s));
       } catch (AssertionError e) {
-        submissionFailed(ErrorHelper.wrap(e));
+        submissionFailed(UserException.systemError(e).build());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 cde2e95..cfe52c2 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,7 +23,7 @@ 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.common.exceptions.UserException;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -90,7 +90,7 @@ public class TestParquetPhysicalPlan extends ExecTest {
     private CountDownLatch latch = new CountDownLatch(1);
 
     @Override
-    public void submissionFailed(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       logger.error("submission failed", ex);
       latch.countDown();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 a9753bc..d2302fb 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,7 +29,7 @@ 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.common.exceptions.UserException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
@@ -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 DrillUserException ex;
+    private volatile UserException 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(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       this.ex = ex;
       completed = true;
       close();

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 9a095aa..acd8624 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
@@ -1953,84 +1953,196 @@ public final class UserBitShared {
         implements com.google.protobuf.ProtocolMessageEnum {
       /**
        * <code>CONNECTION = 0;</code>
+       *
+       * <pre>
+       * equivalent to SQLClientInfoException
+       * - handshake version error
+       * - invalid schema
+       * </pre>
        */
       CONNECTION(0, 0),
       /**
        * <code>DATA_READ = 1;</code>
+       *
+       * <pre>
+       * equivalent to SQLRecoverableException
+       * - corrupt files: can't be read. FS read error
+       * - parsing error due to incomplete or incorrectly written records
+       * </pre>
        */
       DATA_READ(1, 1),
       /**
        * <code>DATA_WRITE = 2;</code>
+       *
+       * <pre>
+       * equivalent to SQLDataException
+       * - data type unsupported by format
+       * </pre>
        */
       DATA_WRITE(2, 2),
       /**
        * <code>FUNCTION = 3;</code>
+       *
+       * <pre>
+       * equivalent to SQLDataException
+       * - Casting errors
+       * - function not found for incoming types after implicit casting
+       * - Flatten misuse
+       * </pre>
        */
       FUNCTION(3, 3),
       /**
        * <code>PARSE = 4;</code>
+       *
+       * <pre>
+       * equivalent to SQLSyntaxErrorException
+       * - typos
+       * - missing table
+       * - SQL keyword misuse
+       * - function names/resolution
+       * </pre>
        */
       PARSE(4, 4),
       /**
        * <code>PERMISSION = 5;</code>
+       *
+       * <pre>
+       * equivalent to SQLInvalidAuthorizationSpecException
+       * </pre>
        */
       PERMISSION(5, 5),
       /**
        * <code>PLAN = 6;</code>
+       *
+       * <pre>
+       * equivalent to SQLNonTransientException
+       * </pre>
        */
       PLAN(6, 6),
       /**
        * <code>RESOURCE = 7;</code>
+       *
+       * <pre>
+       * equivalent to SQLRecoverableException or SQLTransientException
+       * - Recoverable: memory, disk
+       * - Transient: network
+       * </pre>
        */
       RESOURCE(7, 7),
       /**
        * <code>SYSTEM = 8;</code>
+       *
+       * <pre>
+       * equivalent to SQLNonTransientException.
+       * </pre>
        */
       SYSTEM(8, 8),
       /**
        * <code>UNSUPPORTED_OPERATION = 9;</code>
+       *
+       * <pre>
+       * equivalent to SQLFeatureNotSupportedException
+       * - type change
+       * - schema change
+       * </pre>
        */
       UNSUPPORTED_OPERATION(9, 9),
       ;
 
       /**
        * <code>CONNECTION = 0;</code>
+       *
+       * <pre>
+       * equivalent to SQLClientInfoException
+       * - handshake version error
+       * - invalid schema
+       * </pre>
        */
       public static final int CONNECTION_VALUE = 0;
       /**
        * <code>DATA_READ = 1;</code>
+       *
+       * <pre>
+       * equivalent to SQLRecoverableException
+       * - corrupt files: can't be read. FS read error
+       * - parsing error due to incomplete or incorrectly written records
+       * </pre>
        */
       public static final int DATA_READ_VALUE = 1;
       /**
        * <code>DATA_WRITE = 2;</code>
+       *
+       * <pre>
+       * equivalent to SQLDataException
+       * - data type unsupported by format
+       * </pre>
        */
       public static final int DATA_WRITE_VALUE = 2;
       /**
        * <code>FUNCTION = 3;</code>
+       *
+       * <pre>
+       * equivalent to SQLDataException
+       * - Casting errors
+       * - function not found for incoming types after implicit casting
+       * - Flatten misuse
+       * </pre>
        */
       public static final int FUNCTION_VALUE = 3;
       /**
        * <code>PARSE = 4;</code>
+       *
+       * <pre>
+       * equivalent to SQLSyntaxErrorException
+       * - typos
+       * - missing table
+       * - SQL keyword misuse
+       * - function names/resolution
+       * </pre>
        */
       public static final int PARSE_VALUE = 4;
       /**
        * <code>PERMISSION = 5;</code>
+       *
+       * <pre>
+       * equivalent to SQLInvalidAuthorizationSpecException
+       * </pre>
        */
       public static final int PERMISSION_VALUE = 5;
       /**
        * <code>PLAN = 6;</code>
+       *
+       * <pre>
+       * equivalent to SQLNonTransientException
+       * </pre>
        */
       public static final int PLAN_VALUE = 6;
       /**
        * <code>RESOURCE = 7;</code>
+       *
+       * <pre>
+       * equivalent to SQLRecoverableException or SQLTransientException
+       * - Recoverable: memory, disk
+       * - Transient: network
+       * </pre>
        */
       public static final int RESOURCE_VALUE = 7;
       /**
        * <code>SYSTEM = 8;</code>
+       *
+       * <pre>
+       * equivalent to SQLNonTransientException.
+       * </pre>
        */
       public static final int SYSTEM_VALUE = 8;
       /**
        * <code>UNSUPPORTED_OPERATION = 9;</code>
+       *
+       * <pre>
+       * equivalent to SQLFeatureNotSupportedException
+       * - type change
+       * - schema change
+       * </pre>
        */
       public static final int UNSUPPORTED_OPERATION_VALUE = 9;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 10c2790..7383bd2 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -31,15 +31,51 @@ message QueryId {
 
 message DrillPBError{
   enum ErrorType {
+    /* equivalent to SQLClientInfoException
+     * - handshake version error
+     * - invalid schema
+     */
     CONNECTION = 0;
+    /* equivalent to SQLRecoverableException
+     * - corrupt files: can't be read. FS read error
+     * - parsing error due to incomplete or incorrectly written records
+     */
     DATA_READ = 1;
+    /* equivalent to SQLDataException
+     * - data type unsupported by format
+     */
     DATA_WRITE = 2;
+    /* equivalent to SQLDataException
+     * - Casting errors
+     * - function not found for incoming types after implicit casting
+     * - Flatten misuse
+     */
     FUNCTION = 3;
+    /* equivalent to SQLSyntaxErrorException
+     * - typos
+     * - missing table
+     * - SQL keyword misuse
+     * - function names/resolution
+     */
     PARSE = 4;
+    /* equivalent to SQLInvalidAuthorizationSpecException
+     */
     PERMISSION = 5;
+    /* equivalent to SQLNonTransientException
+     */
     PLAN = 6;
+    /* equivalent to SQLRecoverableException or SQLTransientException
+     * - Recoverable: memory, disk
+     * - Transient: network
+     */
     RESOURCE = 7;
+    /* equivalent to SQLNonTransientException.
+     */
     SYSTEM = 8;
+    /* equivalent to SQLFeatureNotSupportedException
+     * - type change
+     * - schema change
+     */
     UNSUPPORTED_OPERATION = 9;
   }
   optional string error_id = 1; // for debug tracing purposes


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

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

Added missing changes from committed patch


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

Branch: refs/heads/master
Commit: 238399de50b460106114d9003f3c9fc4b447f9ea
Parents: 64e3ec5
Author: adeneche <ad...@gmail.com>
Authored: Tue Apr 14 13:46:39 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Thu Apr 16 22:08:23 2015 -0700

----------------------------------------------------------------------
 .../common/exceptions/DrillRemoteException.java |  44 --
 .../common/exceptions/DrillUserException.java   | 204 -------
 .../drill/common/exceptions/ErrorHelper.java    |  37 +-
 .../drill/common/exceptions/UserException.java  | 550 +++++++++++++++++++
 .../common/exceptions/UserExceptionContext.java | 120 ++--
 .../common/exceptions/UserRemoteException.java  |  44 ++
 .../exceptions/TestDrillUserException.java      | 130 -----
 .../common/exceptions/TestUserException.java    | 125 +++++
 .../apache/drill/exec/client/DrillClient.java   |   4 +-
 .../exec/client/PrintingResultsListener.java    |   9 +-
 .../apache/drill/exec/ops/FragmentContext.java  |   6 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   2 +
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   9 +-
 .../drill/exec/rpc/CoordinationQueue.java       |   4 +-
 .../java/org/apache/drill/exec/rpc/RpcBus.java  |   6 +-
 .../drill/exec/rpc/user/QueryResultHandler.java |  19 +-
 .../exec/rpc/user/UserResultsListener.java      |   4 +-
 .../drill/exec/server/rest/QueryWrapper.java    |   6 +-
 .../exec/store/easy/json/JSONRecordReader.java  |  22 +-
 .../apache/drill/exec/work/foreman/Foreman.java |  13 +-
 .../drill/exec/work/foreman/QueryManager.java   |   4 +-
 .../work/fragment/AbstractStatusReporter.java   |   6 +-
 .../exec/work/fragment/FragmentExecutor.java    |   7 +-
 .../exec/work/fragment/StatusReporter.java      |   4 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   6 +-
 .../org/apache/drill/SingleRowListener.java     |   4 +-
 .../java/org/apache/drill/TestBugFixes.java     |   6 +-
 .../apache/drill/TestDisabledFunctionality.java |  62 +--
 .../java/org/apache/drill/TestStarQueries.java  |   6 +-
 .../java/org/apache/drill/TestUnionAll.java     |   6 +-
 .../exec/server/TestDrillbitResilience.java     |   8 +-
 .../store/parquet/ParquetResultListener.java    |   7 +-
 .../store/parquet/TestParquetPhysicalPlan.java  |   4 +-
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   6 +-
 .../apache/drill/exec/proto/UserBitShared.java  | 112 ++++
 protocol/src/main/protobuf/UserBitShared.proto  |  36 ++
 36 files changed, 1048 insertions(+), 594 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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
deleted file mode 100644
index 7b707a5..0000000
--- a/common/src/main/java/org/apache/drill/common/exceptions/DrillRemoteException.java
+++ /dev/null
@@ -1,44 +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.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/238399de/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
deleted file mode 100644
index 73718ae..0000000
--- a/common/src/main/java/org/apache/drill/common/exceptions/DrillUserException.java
+++ /dev/null
@@ -1,204 +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.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/238399de/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
index 1dc4eb5..4da4ee8 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/ErrorHelper.java
@@ -25,39 +25,10 @@ import java.util.regex.Pattern;
 /**
  * Utility class that handles error message generation from protobuf error objects.
  */
-public class ErrorHelper {
+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();
@@ -166,13 +137,13 @@ public class ErrorHelper {
    * @param ex exception
    * @return null if exception is null or no DrillUserException was found
    */
-  static DrillUserException findWrappedUserException(Throwable ex) {
+  static UserException findWrappedUserException(Throwable ex) {
     if (ex == null) {
       return null;
     }
 
     Throwable cause = ex;
-    while (!(cause instanceof DrillUserException)) {
+    while (!(cause instanceof UserException)) {
       if (cause.getCause() != null && cause.getCause() != cause) {
         cause = cause.getCause();
       } else {
@@ -180,7 +151,7 @@ public class ErrorHelper {
       }
     }
 
-    return (DrillUserException) cause;
+    return (UserException) cause;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
new file mode 100644
index 0000000..e995346
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
@@ -0,0 +1,550 @@
+/**
+ * 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 org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+
+/**
+ * Base class for all user exception. The goal is to separate out common error condititions where we can give users
+ * useful feedback.
+ * <p>Throwing a user exception will guarantee it's message will be displayed to the user, along with any context
+ * information added to the exception at various levels while being sent to the client.
+ * <p>A specific class of user exceptions are system exception. They represent system level errors that don't display
+ * any specific error message to the user apart from "A system error has occurend" along with informations to retrieve
+ * the details of the exception from the logs.
+ * <p>although system exception should only display a generic message to the user, for now they will display the root
+ * error message, until all user errors are properly sent from the server side.
+ * <p>Any thrown exception that is not wrapped inside a user exception will automatically be converted to a system
+ * exception before being sent to the client.
+ *
+ * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType
+ */
+public class UserException extends DrillRuntimeException {
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#SYSTEM
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   *
+   * @deprecated this method should never need to be used explicitely, unless you are passing the exception to the
+   *             Rpc layer or UserResultListener.submitFailed()
+   */
+  @Deprecated
+  public static Builder systemError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.SYSTEM, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#CONNECTION
+   * @return user exception builder
+   */
+  public static Builder connectionError() {
+    return connectionError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a connection error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#CONNECTION
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder connectionError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.CONNECTION, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#DATA_READ
+   * @return user exception builder
+   */
+  public static Builder dataReadError() {
+    return dataReadError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a data read error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#DATA_READ
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder dataReadError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.DATA_READ, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#DATA_WRITE
+   * @return user exception builder
+   */
+  public static Builder dataWriteError() {
+    return dataWriteError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a data write error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#DATA_WRITE
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder dataWriteError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.DATA_WRITE, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#FUNCTION
+   * @return user exception builder
+   */
+  public static Builder functionError() {
+    return functionError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a function error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#FUNCTION
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder functionError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.FUNCTION, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PARSE
+   * @return user exception builder
+   */
+  public static Builder parseError() {
+    return parseError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PARSE
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder parseError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.PARSE, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PERMISSION
+   * @return user exception builder
+   */
+  public static Builder permissionError() {
+    return permissionError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PERMISSION
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder permissionError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.PERMISSION, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PLAN
+   * @return user exception builder
+   */
+  public static Builder planError() {
+    return planError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#PLAN
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder planError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.PLAN, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#RESOURCE
+   * @return user exception builder
+   */
+  public static Builder resourceError() {
+    return resourceError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#RESOURCE
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder resourceError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.RESOURCE, cause);
+  }
+
+  /**
+   * creates a new user exception builder .
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#UNSUPPORTED_OPERATION
+   * @return user exception builder
+   */
+  public static Builder unsupportedError() {
+    return unsupportedError(null);
+  }
+
+  /**
+   * wraps the passed exception inside a system error.
+   * <p>the cause message will be used unless {@link Builder#message(String, Object...)} is called.
+   * <p>if the wrapped exception is, or wraps, a user exception it will be returned by {@link Builder#build()} instead
+   * of creating a new exception. Any added context will be added to the user exception as well.
+   *
+   * @see org.apache.drill.exec.proto.UserBitShared.DrillPBError.ErrorType#UNSUPPORTED_OPERATION
+   *
+   * @param cause exception we want the user exception to wrap. If cause is, or wrap, a user exception it will be
+   *              returned by the builder instead of creating a new user exception
+   * @return user exception builder
+   */
+  public static Builder unsupportedError(Throwable cause) {
+    return new Builder(DrillPBError.ErrorType.UNSUPPORTED_OPERATION, cause);
+  }
+
+  /**
+   * Builder class for DrillUserException. You can wrap an existing exception, in this case it will first check if
+   * this exception is, or wraps, a DrillUserException. If it does then the builder will use the user exception as it is
+   * (it will ignore the message passed to the constructor) and will add any additional context information to the
+   * exception's context
+   */
+  public static class Builder {
+
+    private final Throwable cause;
+    private final DrillPBError.ErrorType errorType;
+    private final UserException uex;
+    private final UserExceptionContext context;
+
+    private String message;
+
+    /**
+     * wraps an existing exception inside a user 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. Can be null
+     */
+    private Builder(DrillPBError.ErrorType errorType, Throwable cause) {
+      this.cause = cause;
+
+      //TODO handle the improbable case where cause is a SYSTEM exception ?
+      uex = ErrorHelper.findWrappedUserException(cause);
+      if (uex != null) {
+        this.errorType = null;
+        this.context = uex.context;
+      } else {
+        // we will create a new user exception
+        this.errorType = errorType;
+        this.context = new UserExceptionContext();
+        this.message = cause != null ? cause.getMessage() : null;
+      }
+    }
+
+    /**
+     * sets or replaces the error message.
+     * <p>This will be ignored if this builder is wrapping a user exception
+     *
+     * @see String#format(String, Object...)
+     *
+     * @param format format string
+     * @param args Arguments referenced by the format specifiers in the format string
+     * @return this builder
+     */
+    public Builder message(String format, Object... args) {
+      // we can't replace the message of a user exception
+      if (uex == null && format != null) {
+        this.message = String.format(format, args);
+      }
+      return this;
+    }
+
+    /**
+     * add DrillbitEndpoint identity to the context.
+     * <p>if the context already has a drillbitEndpoint identity, the new identity will be ignored
+     *
+     * @param endpoint drillbit endpoint identity
+     */
+    public Builder addIdentity(CoordinationProtos.DrillbitEndpoint endpoint) {
+      context.add(endpoint);
+      return this;
+    }
+
+    /**
+     * add a string line to the bottom of the context
+     * @param value string line
+     * @return this builder
+     */
+    public Builder addContext(String value) {
+      context.add(value);
+      return this;
+    }
+
+    /**
+     * add a string value to the bottom of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder addContext(String name, String value) {
+      context.add(name, value);
+      return this;
+    }
+
+    /**
+     * add a long value to the bottom of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder addContext(String name, long value) {
+      context.add(name, value);
+      return this;
+    }
+
+    /**
+     * add a double value to the bottom of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder addContext(String name, double value) {
+      context.add(name, value);
+      return this;
+    }
+
+    /**
+     * pushes a string value to the top of the context
+     *
+     * @param value context value
+     * @return this builder
+     */
+    public Builder pushContext(String value) {
+      context.push(value);
+      return this;
+    }
+
+    /**
+     * pushes a string value to the top of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder pushContext(String name, String value) {
+      context.push(name, value);
+      return this;
+    }
+
+    /**
+     * pushes a long value to the top of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder pushContext(String name, long value) {
+      context.push(name, value);
+      return this;
+    }
+
+    /**
+     * pushes a double value to the top of the context
+     *
+     * @param name context name
+     * @param value context value
+     * @return this builder
+     */
+    public Builder pushContext(String name, double value) {
+      context.push(name, value);
+      return this;
+    }
+
+    /**
+     * builds a user exception or returns the wrapped one.
+     *
+     * @return user exception
+     */
+    public UserException build() {
+
+      if (uex != null) {
+        return uex;
+      }
+
+      return new UserException(this);
+    }
+  }
+
+  private final DrillPBError.ErrorType errorType;
+
+  private final UserExceptionContext context;
+
+  protected UserException(DrillPBError.ErrorType errorType, String message, Throwable cause) {
+    super(message, cause);
+
+    this.errorType = errorType;
+    this.context = new UserExceptionContext();
+  }
+
+  private UserException(Builder builder) {
+    super(builder.message, builder.cause);
+    this.errorType = builder.errorType;
+    this.context = builder.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\n" +
+      context.generateContextMessage();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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
index ef7d486..1054c7f 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
@@ -24,11 +24,10 @@ 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
+ * Holds context information about a DrillUserException. We can add structured context information that will added
+ * to the error message displayed to the client.
  */
-public class UserExceptionContext {
+class UserExceptionContext {
 
   private final String errorId;
   private final List<String> contextList;
@@ -40,73 +39,86 @@ public class UserExceptionContext {
     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
+   * adds a string to the bottom of the context list
+   * @param context context string
    */
-  public UserExceptionContext add(String context) {
+  void 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;
+  /**
+   * add DrillbitEndpoint identity to the context.
+   * <p>if the context already has a drillbitEndpoint identity, the new identity will be ignored
+   *
+   * @param endpoint drillbit endpoint identity
+   */
+  void add(CoordinationProtos.DrillbitEndpoint endpoint) {
+    if (this.endpoint == null) {
+      this.endpoint = endpoint;
+    }
+  }
+
+  /**
+   * adds a sring value to the bottom of the context list
+   * @param context context name
+   * @param value context value
+   */
+  void add(String context, String value) {
+    add(context + " " + value);
   }
 
   /**
-   * adds an int to the bottom of the context list
-   * @param context context prefix string
-   * @param value int value
+   * adds a long value to the bottom of the context list
+   * @param context context name
+   * @param value context value
    */
-  public UserExceptionContext add(String context, long value) {
-    add(context + ": " + value);
-    return this;
+  void add(String context, long value) {
+    add(context + " " + value);
   }
 
   /**
    * adds a double to the bottom of the context list
-   * @param context context prefix string
-   * @param value double value
+   * @param context context name
+   * @param value context value
    */
-  public UserExceptionContext add(String context, double value) {
-    add(context + ": " + value);
-    return this;
+  void add(String context, double value) {
+    add(context + " " + value);
   }
 
   /**
-   * adds a context line at the top of the context list
-   * @param context context line
+   * pushes a string to the top of the context list
+   * @param context context string
    */
-  public UserExceptionContext push(String context) {
+  void 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
+   * pushes a string value to the top of the context list
+   * @param context context name
+   * @param value context value
+   */
+  void push(String context, String value) {
+    push(context + " " + value);
+  }
+
+  /**
+   * pushes a long value to the top of the context list
+   * @param context context name
+   * @param value context value
    */
-  public UserExceptionContext push(String context, long value) {
-    push(context + ": " + value);
-    return this;
+  void push(String context, long value) {
+    push(context + " " + value);
   }
 
   /**
    * adds a double at the top of the context list
-   * @param context context prefix string
-   * @param value double value
+   * @param context context name
+   * @param value context value
    */
-  public UserExceptionContext push(String context, double value) {
-    push(context + ": " + value);
-    return this;
+  void push(String context, double value) {
+    push(context + " " + value);
   }
 
   String getErrorId() {
@@ -128,20 +140,16 @@ public class UserExceptionContext {
       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");
+    // add identification infos
+    sb.append("\n[");
+    sb.append(errorId).append(" ");
+    if(endpoint != null) {
+      sb.append("on ")
+        .append(endpoint.getAddress())
+        .append(":")
+        .append(endpoint.getUserPort());
     }
+    sb.append("]");
 
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/common/src/main/java/org/apache/drill/common/exceptions/UserRemoteException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserRemoteException.java b/common/src/main/java/org/apache/drill/common/exceptions/UserRemoteException.java
new file mode 100644
index 0000000..1b3fa42
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserRemoteException.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 UserRemoteException extends UserException {
+
+  private final DrillPBError error;
+
+  public UserRemoteException(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/238399de/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
deleted file mode 100644
index 7b19d55..0000000
--- a/common/src/test/java/org/apache/drill/common/exceptions/TestDrillUserException.java
+++ /dev/null
@@ -1,130 +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.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/238399de/common/src/test/java/org/apache/drill/common/exceptions/TestUserException.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/common/exceptions/TestUserException.java b/common/src/test/java/org/apache/drill/common/exceptions/TestUserException.java
new file mode 100644
index 0000000..a145f95
--- /dev/null
+++ b/common/src/test/java/org/apache/drill/common/exceptions/TestUserException.java
@@ -0,0 +1,125 @@
+/**
+ * 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 when creating user exceptions
+ */
+public class TestUserException {
+
+  private Exception wrap(UserException 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() {
+    String message = "This is an exception";
+    UserException uex = UserException.systemError(new RuntimeException(message)).build();
+
+    Assert.assertEquals(message, uex.getOriginalMessage());
+
+    DrillPBError error = uex.getOrCreatePBError(true);
+
+    Assert.assertEquals(ErrorType.SYSTEM, error.getErrorType());
+  }
+
+  @Test
+  public void testBuildUserExceptionWithMessage() {
+    String message = "Test message";
+
+    UserException uex = UserException.dataWriteError().message(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";
+
+    UserException uex = UserException.dataWriteError(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";
+
+    UserException uex = UserException.dataWriteError(new RuntimeException(messageA)).message(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";
+
+    UserException original = UserException.connectionError().message(messageA).build();
+    UserException uex = UserException.dataWriteError(wrap(original, 5)).message(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.assertFalse(error.getMessage().contains(messageB)); // messageB should not be part of the context
+  }
+
+  @Test
+  public void testBuildUserExceptionWithFormattedMessage() {
+    String format = "This is test #%d";
+
+    UserException uex = UserException.connectionError().message(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() {
+    UserException uex = UserException.dataReadError().message("this is a data read exception").build();
+
+    Exception wrapped = wrap(uex, 3);
+    Assert.assertEquals(uex, UserException.systemError(wrapped).build());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 6555cad..336a149 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,7 +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.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.zk.ZKClusterCoordinator;
@@ -308,7 +308,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
     }
 
     @Override
-    public void submissionFailed(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       // or  !client.isActive()
       if (ex.getCause() instanceof ChannelClosedException) {
         if (reconnect()) {

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 a11cec0..2bf35b1 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,8 +22,7 @@ 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.common.exceptions.UserException;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -43,7 +42,7 @@ public class PrintingResultsListener implements UserResultsListener {
   Format format;
   int    columnWidth;
   BufferAllocator allocator;
-  volatile DrillUserException exception;
+  volatile UserException exception;
   QueryId queryId;
 
   public PrintingResultsListener(DrillConfig config, Format format, int columnWidth) {
@@ -54,7 +53,7 @@ public class PrintingResultsListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(DrillUserException ex) {
+  public void submissionFailed(UserException ex) {
     exception = ex;
     System.out.println("Exception (no rows returned): " + ex );
     latch.countDown();
@@ -77,7 +76,7 @@ public class PrintingResultsListener implements UserResultsListener {
       try {
         loader.load(header.getDef(), data);
       } catch (SchemaChangeException e) {
-        submissionFailed(ErrorHelper.wrap(e));
+        submissionFailed(UserException.systemError(e).build());
       }
 
       switch(format) {

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 8c402e8..44ca78a 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
@@ -29,8 +29,7 @@ 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.UserException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.ClassGenerator;
@@ -148,8 +147,7 @@ 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());
+    UserException dse = UserException.systemError(cause).addIdentity(getIdentity()).build();
 
     // log the error id
     logger.error("Fragment Context received failure -- Fragment: {}:{}",

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 80cbd81..6b3caf4 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,6 +38,8 @@ 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/238399de/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 7892999..9ca64d8 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
@@ -29,7 +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.common.exceptions.UserException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
@@ -44,7 +44,6 @@ 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;
@@ -104,8 +103,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());
@@ -155,7 +154,7 @@ public class DrillSqlWorker {
       return handler.getPlan(sqlNode);
     } catch(ValidationException e) {
       String errorMessage = e.getCause() != null ? e.getCause().getMessage() : e.getMessage();
-      throw new DrillUserException.Builder(ErrorType.PARSE, e, errorMessage).build();
+      throw UserException.parseError(e).message(errorMessage).build();
     } catch (IOException | RelConversionException e) {
       throw new QueryInputException("Failure handling SQL.", e);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 ad5002e..1bb65d3 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,7 +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.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 
 /**
@@ -151,7 +151,7 @@ public class CoordinationQueue {
     // logger.debug("Updating failed future.");
     try {
       RpcOutcome<?> rpc = removeFromMap(coordinationId);
-      rpc.setException(new DrillRemoteException(failure));
+      rpc.setException(new UserRemoteException(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/238399de/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 79232c9..b165b53 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,8 +30,7 @@ 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.common.exceptions.UserException;
 import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 
@@ -194,8 +193,7 @@ public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> imp
         try {
           handle(connection, msg.rpcType, msg.pBody, msg.dBody, sender);
         } catch(UserRpcException e){
-          DrillUserException uex = ErrorHelper.wrap(e);
-          uex.getContext().add(e.getEndpoint());
+          UserException uex = UserException.systemError(e).addIdentity(e.getEndpoint()).build();
 
           logger.error("Unexpected Error while handling request message", e);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 a03e2c0..3c807d5 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,9 +23,8 @@ import io.netty.buffer.DrillBuf;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 
-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.common.exceptions.UserRemoteException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
@@ -109,7 +108,7 @@ public class QueryResultHandler {
       if (isFailureResult) {
         // Failure case--pass on via submissionFailed(...).
 
-        resultsListener.submissionFailed(new DrillRemoteException(queryResult.getError(0)));
+        resultsListener.submissionFailed(new UserRemoteException(queryResult.getError(0)));
         // Note: Listener is removed in finally below.
       } else if (isTerminalResult) {
         // A successful completion/canceled case--pass on via resultArrived
@@ -117,7 +116,7 @@ public class QueryResultHandler {
         try {
           resultsListener.queryCompleted();
         } catch ( Exception e ) {
-          resultsListener.submissionFailed(ErrorHelper.wrap(e));
+          resultsListener.submissionFailed(UserException.systemError(e).build());
         }
       } else {
         logger.warn("queryState {} was ignored", queryState);
@@ -158,7 +157,7 @@ public class QueryResultHandler {
       // That releases batch if successful.
     } catch ( Exception e ) {
       batch.release();
-      resultsListener.submissionFailed(ErrorHelper.wrap(e));
+      resultsListener.submissionFailed(UserException.systemError(e).build());
     }
   }
 
@@ -192,7 +191,7 @@ public class QueryResultHandler {
 
   private void failAll() {
     for (UserResultsListener l : queryIdToResultsListenersMap.values()) {
-      l.submissionFailed(ErrorHelper.wrap(new RpcException("Received result without QueryId")));
+      l.submissionFailed(UserException.systemError(new RpcException("Received result without QueryId")).build());
     }
   }
 
@@ -200,7 +199,7 @@ public class QueryResultHandler {
 
     private ConcurrentLinkedQueue<QueryDataBatch> results = Queues.newConcurrentLinkedQueue();
     private volatile boolean finished = false;
-    private volatile DrillUserException ex;
+    private volatile UserException ex;
     private volatile UserResultsListener output;
     private volatile ConnectionThrottle throttle;
 
@@ -245,7 +244,7 @@ public class QueryResultHandler {
     }
 
     @Override
-    public void submissionFailed(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       finished = true;
       synchronized (this) {
         if (output == null) {
@@ -276,7 +275,7 @@ public class QueryResultHandler {
 
     @Override
     public void failed(RpcException ex) {
-      resultsListener.submissionFailed(ErrorHelper.wrap(ex));
+      resultsListener.submissionFailed(UserException.systemError(ex).build());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 b9f742c..f928476 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,7 +17,7 @@
  */
 package org.apache.drill.exec.rpc.user;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 
 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(DrillUserException ex);
+  void submissionFailed(UserException 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/238399de/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 8415440..62f5bdb 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,7 +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.common.exceptions.UserException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -118,7 +118,7 @@ public class QueryWrapper {
 
 
   private static class Listener implements UserResultsListener {
-    private volatile DrillUserException exception;
+    private volatile UserException 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(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       exception = ex;
       logger.error("Query Failed", ex);
       latch.countDown();

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 4c44dbd..b41de31 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,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ExecConstants;
@@ -29,7 +30,6 @@ 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.JsonProcessor.ReadState;
@@ -148,7 +148,7 @@ public class JSONRecordReader extends AbstractRecordReader {
     }
   }
 
-  protected void handleAndRaise(final String suffix, final Exception e) throws DrillUserException {
+  protected void handleAndRaise(String suffix, Exception e) throws UserException {
 
     String message = e.getMessage();
     int columnNr = -1;
@@ -159,16 +159,12 @@ public class JSONRecordReader extends AbstractRecordReader {
       columnNr = ex.getLocation().getColumnNr();
     }
 
-    final 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 builder.build();
+    throw UserException.dataReadError(e)
+      .message("%s - %s", suffix, message)
+      .addContext("Filename", hadoopPath.toUri().getPath())
+      .addContext("Record", recordCount + 1)
+      .addContext("Column", columnNr)
+      .build();
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 0df5145..f824b53 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
@@ -30,8 +30,7 @@ import com.google.common.base.Preconditions;
 
 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.UserException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
@@ -499,7 +498,7 @@ public class Foreman implements Runnable {
    */
   private class ForemanResult implements AutoCloseable {
     private QueryState resultState = null;
-    private DrillUserException resultException = null;
+    private Exception resultException = null;
     private boolean isClosed = false;
 
     /**
@@ -532,7 +531,7 @@ public class Foreman implements Runnable {
       Preconditions.checkState(resultState == null);
 
       resultState = QueryState.FAILED;
-      addException(exception);
+      resultException = exception;
     }
 
     /**
@@ -545,8 +544,7 @@ public class Foreman implements Runnable {
       Preconditions.checkNotNull(exception);
 
       if (resultException == null) {
-        resultException = ErrorHelper.wrap(exception);
-        resultException.getContext().add(queryContext.getCurrentEndpoint());
+        resultException = exception;
       } else {
         resultException.addSuppressed(exception);
       }
@@ -615,7 +613,8 @@ public class Foreman implements Runnable {
           .setQueryState(resultState);
       if (resultException != null) {
         boolean verbose = queryContext.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
-        resultBuilder.addError(resultException.getOrCreatePBError(verbose));
+        UserException uex = UserException.systemError(resultException).addIdentity(queryContext.getCurrentEndpoint()).build();
+        resultBuilder.addError(uex.getOrCreatePBError(verbose));
       }
 
       /*

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 dc60beb..31b1f2b 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,7 +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.UserRemoteException;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -160,7 +160,7 @@ public class QueryManager implements FragmentStatusListener, DrillbitStatusListe
       break;
 
     case FAILED:
-      stateListener.moveToState(QueryState.FAILED, new DrillRemoteException(status.getProfile().getError()));
+      stateListener.moveToState(QueryState.FAILED, new UserRemoteException(status.getProfile().getError()));
       break;
 
     default:

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 8279876..4ff28f3 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,7 +17,7 @@
  */
 package org.apache.drill.exec.work.fragment;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -40,7 +40,7 @@ public abstract class AbstractStatusReporter implements StatusReporter{
     return getBuilder(context, state, null);
   }
 
-  public static FragmentStatus.Builder getBuilder(FragmentContext context, FragmentState state, DrillUserException ex){
+  public static FragmentStatus.Builder getBuilder(FragmentContext context, FragmentState state, UserException ex){
     FragmentStatus.Builder status = FragmentStatus.newBuilder();
     MinorFragmentProfile.Builder b = MinorFragmentProfile.newBuilder();
     context.getStats().addMetricsToStatus(b);
@@ -105,7 +105,7 @@ public abstract class AbstractStatusReporter implements StatusReporter{
   protected abstract void statusChange(FragmentHandle handle, FragmentStatus status);
 
   @Override
-  public final void fail(FragmentHandle handle, String message, DrillUserException excep) {
+  public final void fail(FragmentHandle handle, String message, UserException excep) {
     FragmentStatus.Builder status = getBuilder(context, FragmentState.FAILED, excep);
     fail(handle, status);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 58f4256..a4a97c9 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,8 +21,7 @@ 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.common.exceptions.UserException;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
@@ -229,9 +228,7 @@ public class FragmentExecutor implements Runnable {
   private void internalFail(final Throwable excep) {
     state.set(FragmentState.FAILED_VALUE);
 
-    DrillUserException uex = ErrorHelper.wrap(excep);
-    uex.getContext().add(getContext().getIdentity());
-
+    UserException uex = UserException.systemError(excep).addIdentity(getContext().getIdentity()).build();
     listener.fail(fragmentContext.getHandle(), "Failure while running fragment.", uex);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 1699322..2bba345 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,7 +17,7 @@
  */
 package org.apache.drill.exec.work.fragment;
 
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 
@@ -25,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, DrillUserException excep);
+  void fail(FragmentHandle handle, String message, UserException excep);
   void stateChanged(FragmentHandle handle, FragmentState newState);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 6df5801..3931039 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
@@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Preconditions;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillUserException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
@@ -320,12 +320,12 @@ public class BaseTestQuery extends ExecTest {
   }
 
   private static class SilentListener implements UserResultsListener {
-    private volatile DrillUserException exception;
+    private volatile UserException exception;
     private AtomicInteger count = new AtomicInteger();
     private CountDownLatch latch = new CountDownLatch(1);
 
     @Override
-    public void submissionFailed(DrillUserException ex) {
+    public void submissionFailed(UserException ex) {
       exception = ex;
       System.out.println("Query failed: " + ex.getMessage());
       latch.countDown();

http://git-wip-us.apache.org/repos/asf/drill/blob/238399de/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 715904d..99aa9fc 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,7 +23,7 @@ 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.common.exceptions.UserException;
 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;
@@ -49,7 +49,7 @@ public abstract class SingleRowListener implements UserResultsListener {
   }
 
   @Override
-  public void submissionFailed(final DrillUserException ex) {
+  public void submissionFailed(final UserException ex) {
     exception = ex;
     synchronized(errorList) {
       errorList.add(ex.getOrCreatePBError(false));