You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2015/05/06 02:27:35 UTC

[1/5] drill git commit: DRILL-2598: Throw validation error if CREATE VIEW/CTAS contains duplicate columns in definition

Repository: drill
Updated Branches:
  refs/heads/master ac823fe8b -> d43324f89


DRILL-2598: Throw validation error if CREATE VIEW/CTAS contains duplicate columns in definition


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

Branch: refs/heads/master
Commit: ed02612a1b4eb419315791f87ab889e3765c1ceb
Parents: 703314b
Author: vkorukanti <ve...@gmail.com>
Authored: Mon May 4 22:26:41 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Tue May 5 11:03:58 2015 -0700

----------------------------------------------------------------------
 .../planner/sql/handlers/SqlHandlerUtil.java    |  38 +++++-
 .../org/apache/drill/exec/sql/TestCTAS.java     |  95 +++++++++++++
 .../apache/drill/exec/sql/TestViewSupport.java  | 132 +++++++++++++++++--
 3 files changed, 250 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/ed02612a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
index 50af972..7ae5e0d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.planner.sql.handlers;
 
+import com.google.common.collect.Sets;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.TypedSqlNode;
 import org.apache.calcite.tools.Planner;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.drill.common.exceptions.DrillException;
@@ -33,6 +35,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.exec.store.ischema.Records;
 
+import java.util.HashSet;
 import java.util.List;
 
 public class SqlHandlerUtil {
@@ -55,12 +58,23 @@ public class SqlHandlerUtil {
   public static RelNode resolveNewTableRel(boolean isNewTableView, Planner planner, List<String> tableFieldNames,
       SqlNode newTableQueryDef) throws ValidationException, RelConversionException {
 
-    SqlNode validatedQuery = planner.validate(newTableQueryDef);
-    RelNode validatedQueryRelNode = planner.convert(validatedQuery);
 
-    if (tableFieldNames.size() > 0) {
-      final RelDataType queryRowType = validatedQueryRelNode.getRowType();
+    TypedSqlNode validatedSqlNodeWithType = planner.validateAndGetType(newTableQueryDef);
+
+    // Get the row type of view definition query.
+    // Reason for getting the row type from validated SqlNode than RelNode is because SqlNode -> RelNode involves
+    // renaming duplicate fields which is not desired when creating a view or table.
+    // For ex: SELECT region_id, region_id FROM cp.`region.json` LIMIT 1 returns
+    //  +------------+------------+
+    //  | region_id  | region_id0 |
+    //  +------------+------------+
+    //  | 0          | 0          |
+    //  +------------+------------+
+    // which is not desired when creating new views or tables.
+    final RelDataType queryRowType = validatedSqlNodeWithType.getType();
+    final RelNode validatedQueryRelNode = planner.convert(validatedSqlNodeWithType.getSqlNode());
 
+    if (tableFieldNames.size() > 0) {
       // Field count should match.
       if (tableFieldNames.size() != queryRowType.getFieldCount()) {
         final String tblType = isNewTableView ? "view" : "table";
@@ -78,6 +92,9 @@ public class SqlHandlerUtil {
         }
       }
 
+      // validate the given field names to make sure there are no duplicates
+      ensureNoDuplicateColumnNames(tableFieldNames);
+
       // CTAS statement has table field list (ex. below), add a project rel to rename the query fields.
       // Ex. CREATE TABLE tblname(col1, medianOfCol2, avgOfCol3) AS
       //        SELECT col1, median(col2), avg(col3) FROM sourcetbl GROUP BY col1 ;
@@ -86,9 +103,22 @@ public class SqlHandlerUtil {
       return DrillRelOptUtil.createRename(validatedQueryRelNode, tableFieldNames);
     }
 
+    // As the column names of the view are derived from SELECT query, make sure the query has no duplicate column names
+    ensureNoDuplicateColumnNames(queryRowType.getFieldNames());
+
     return validatedQueryRelNode;
   }
 
+  private static void ensureNoDuplicateColumnNames(List<String> fieldNames) throws ValidationException {
+    final HashSet<String> fieldHashSet = Sets.newHashSetWithExpectedSize(fieldNames.size());
+    for(String field : fieldNames) {
+      if (fieldHashSet.contains(field.toLowerCase())) {
+        throw new ValidationException(String.format("Duplicate column name [%s]", field));
+      }
+      fieldHashSet.add(field.toLowerCase());
+    }
+  }
+
   public static Table getTableFromSchema(AbstractSchema drillSchema, String tblName) throws DrillException {
     try {
       return drillSchema.getTable(tblName);

http://git-wip-us.apache.org/repos/asf/drill/blob/ed02612a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
new file mode 100644
index 0000000..5fff956
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.sql;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestCTAS extends BaseTestQuery {
+  @Test // DRILL-2589
+  public void withDuplicateColumnsInDef1() throws Exception {
+    ctasErrorTestHelper("CREATE TABLE %s.%s AS SELECT region_id, region_id FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "region_id")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void withDuplicateColumnsInDef2() throws Exception {
+    ctasErrorTestHelper("CREATE TABLE %s.%s AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "sales_city")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void withDuplicateColumnsInDef3() throws Exception {
+    ctasErrorTestHelper(
+        "CREATE TABLE %s.%s(regionid, regionid) " +
+            "AS SELECT region_id, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "regionid")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void withDuplicateColumnsInDef4() throws Exception {
+    ctasErrorTestHelper(
+        "CREATE TABLE %s.%s(regionid, salescity, salescity) " +
+            "AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "salescity")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void withDuplicateColumnsInDef5() throws Exception {
+    ctasErrorTestHelper(
+        "CREATE TABLE %s.%s(regionid, salescity, SalesCity) " +
+            "AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "SalesCity")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void whenInEqualColumnCountInTableDefVsInTableQuery() throws Exception {
+    ctasErrorTestHelper(
+        "CREATE TABLE %s.%s(regionid, salescity) " +
+            "AS SELECT region_id, sales_city, sales_region FROM cp.`region.json`",
+        "Error: table's field list and the table's query field list have different counts."
+    );
+  }
+
+  @Test // DRILL-2589
+  public void whenTableQueryColumnHasStarAndTableFiledListIsSpecified() throws Exception {
+    ctasErrorTestHelper(
+        "CREATE TABLE %s.%s(regionid, salescity) " +
+            "AS SELECT region_id, * FROM cp.`region.json`",
+        "Error: table's query field list has a '*', which is invalid when table's field list is specified."
+    );
+  }
+
+  private static void ctasErrorTestHelper(final String ctasSql, final String errorMsg)
+      throws Exception {
+    final String createTableSql = String.format(ctasSql, "dfs_test.tmp", "testTableName");
+
+    testBuilder()
+        .sqlQuery(createTableSql)
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, errorMsg)
+        .go();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/ed02612a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index 2ae6991..5c2dc90 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -193,10 +193,10 @@ public class TestViewSupport extends TestBaseViewSupport {
         "(regionid, salescity)",
         "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id` DESC",
         "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "regionid" },
+        new String[]{"regionid"},
         ImmutableList.of(
-            new Object[] { 109L },
-            new Object[] { 108L }
+            new Object[]{109L},
+            new Object[]{108L}
         )
     );
   }
@@ -209,10 +209,10 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         "SELECT region_id FROM cp.`region.json` UNION SELECT employee_id FROM cp.`employee.json`",
         "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
-        new String[] { "regionid" },
+        new String[]{"regionid"},
         ImmutableList.of(
-            new Object[] { 110L },
-            new Object[] { 108L }
+            new Object[]{110L},
+            new Object[]{108L}
         )
     );
   }
@@ -254,9 +254,9 @@ public class TestViewSupport extends TestBaseViewSupport {
         null,
         viewDef,
         "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
-        new String[] { "n_nationkey", "n_name", "n_regionkey", "n_comment" },
+        new String[]{"n_nationkey", "n_name", "n_regionkey", "n_comment"},
         ImmutableList.of(
-            new Object[] { 0, "ALGERIA", 0, " haggle. carefully final deposits detect slyly agai" }
+            new Object[]{0, "ALGERIA", 0, " haggle. carefully final deposits detect slyly agai"}
         )
     );
   }
@@ -296,8 +296,8 @@ public class TestViewSupport extends TestBaseViewSupport {
 
       // Make sure the new view created returns the data expected.
       queryViewHelper(String.format("SELECT * FROM %s.`%s` LIMIT 1", TEMP_SCHEMA, viewName),
-          new String[] { "sales_state_province" },
-          ImmutableList.of(new Object[] { "None" })
+          new String[]{"sales_state_province" },
+          ImmutableList.of(new Object[]{"None"})
       );
     } finally {
       dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
@@ -434,7 +434,7 @@ public class TestViewSupport extends TestBaseViewSupport {
       createViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA, null, "SELECT region_id, sales_city FROM `region.json`");
 
       final String[] baselineColumns = new String[] { "region_id", "sales_city" };
-      final List<Object[]> baselineValues = ImmutableList.of(new Object[] { 109L, "Santa Fe"});
+      final List<Object[]> baselineValues = ImmutableList.of(new Object[]{109L, "Santa Fe"});
 
       // Query the view
       queryViewHelper(
@@ -480,4 +480,114 @@ public class TestViewSupport extends TestBaseViewSupport {
       dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
     }
   }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef1() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s AS SELECT region_id, region_id FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "region_id")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef2() throws Exception {
+    createViewErrorTestHelper("CREATE VIEW %s.%s AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "sales_city")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef3() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s(regionid, regionid) AS SELECT region_id, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "regionid")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef4() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s(regionid, salescity, salescity) " +
+            "AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "salescity")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef5() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s(regionid, salescity, SalesCity) " +
+            "AS SELECT region_id, sales_city, sales_city FROM cp.`region.json`",
+        String.format("Error: Duplicate column name [%s]", "SalesCity")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithDuplicateColumnsInDef6() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s " +
+            "AS SELECT t1.region_id, t2.region_id FROM cp.`region.json` t1 JOIN cp.`region.json` t2 " +
+            "ON t1.region_id = t2.region_id LIMIT 1",
+        String.format("Error: Duplicate column name [%s]", "region_id")
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithUniqueColsInFieldListDuplicateColsInQuery1() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        "(regionid1, regionid2)",
+        "SELECT region_id, region_id FROM cp.`region.json` LIMIT 1",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME",
+        new String[]{"regionid1", "regionid2"},
+        ImmutableList.of(
+            new Object[]{0L, 0L}
+        )
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWithUniqueColsInFieldListDuplicateColsInQuery2() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        "(regionid1, regionid2)",
+        "SELECT t1.region_id, t2.region_id FROM cp.`region.json` t1 JOIN cp.`region.json` t2 " +
+            "ON t1.region_id = t2.region_id LIMIT 1",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME",
+        new String[]{"regionid1", "regionid2"},
+        ImmutableList.of(
+            new Object[]{0L, 0L}
+        )
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWhenInEqualColumnCountInViewDefVsInViewQuery() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s(regionid, salescity) " +
+            "AS SELECT region_id, sales_city, sales_region FROM cp.`region.json`",
+        "Error: view's field list and the view's query field list have different counts."
+    );
+  }
+
+  @Test // DRILL-2589
+  public void createViewWhenViewQueryColumnHasStarAndViewFiledListIsSpecified() throws Exception {
+    createViewErrorTestHelper(
+        "CREATE VIEW %s.%s(regionid, salescity) " +
+            "AS SELECT region_id, * FROM cp.`region.json`",
+        "Error: view's query field list has a '*', which is invalid when view's field list is specified."
+    );
+  }
+
+  private static void createViewErrorTestHelper(final String viewSql, final String errorMsg)
+      throws Exception {
+    final String createViewSql = String.format(viewSql, TEMP_SCHEMA, "duplicateColumnsInViewDef");
+
+    testBuilder()
+        .sqlQuery(createViewSql)
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, errorMsg)
+        .go();
+  }
 }


[2/5] drill git commit: DRILL-2902: Add support for context functions: user (synonyms session_user and system_user) and current_schema

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java
index 5e7562e..bbf7860 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitControl.java
@@ -535,15 +535,14 @@ public final class SchemaBitControl
                     output.writeInt64(12, message.getMemInitial(), false);
                 if(message.hasMemMax())
                     output.writeInt64(13, message.getMemMax(), false);
-                if(message.hasQueryStartTime())
-                    output.writeInt64(14, message.getQueryStartTime(), false);
                 if(message.hasCredentials())
-                    output.writeObject(15, message.getCredentials(), org.apache.drill.exec.proto.SchemaUserBitShared.UserCredentials.WRITE, false);
+                    output.writeObject(14, message.getCredentials(), org.apache.drill.exec.proto.SchemaUserBitShared.UserCredentials.WRITE, false);
 
-                if(message.hasTimeZone())
-                    output.writeInt32(16, message.getTimeZone(), false);
                 if(message.hasOptionsJson())
-                    output.writeString(17, message.getOptionsJson(), false);
+                    output.writeString(15, message.getOptionsJson(), false);
+                if(message.hasContext())
+                    output.writeObject(16, message.getContext(), org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.WRITE, false);
+
             }
             public boolean isInitialized(org.apache.drill.exec.proto.BitControl.PlanFragment message)
             {
@@ -620,18 +619,16 @@ public final class SchemaBitControl
                             builder.setMemMax(input.readInt64());
                             break;
                         case 14:
-                            builder.setQueryStartTime(input.readInt64());
-                            break;
-                        case 15:
                             builder.setCredentials(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.UserCredentials.MERGE));
 
                             break;
-                        case 16:
-                            builder.setTimeZone(input.readInt32());
-                            break;
-                        case 17:
+                        case 15:
                             builder.setOptionsJson(input.readString());
                             break;
+                        case 16:
+                            builder.setContext(input.mergeObject(org.apache.drill.exec.proto.BitControl.QueryContextInformation.newBuilder(), org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.MERGE));
+
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -683,10 +680,9 @@ public final class SchemaBitControl
                 case 11: return "foreman";
                 case 12: return "memInitial";
                 case 13: return "memMax";
-                case 14: return "queryStartTime";
-                case 15: return "credentials";
-                case 16: return "timeZone";
-                case 17: return "optionsJson";
+                case 14: return "credentials";
+                case 15: return "optionsJson";
+                case 16: return "context";
                 default: return null;
             }
         }
@@ -709,10 +705,134 @@ public final class SchemaBitControl
             fieldMap.put("foreman", 11);
             fieldMap.put("memInitial", 12);
             fieldMap.put("memMax", 13);
-            fieldMap.put("queryStartTime", 14);
-            fieldMap.put("credentials", 15);
-            fieldMap.put("timeZone", 16);
-            fieldMap.put("optionsJson", 17);
+            fieldMap.put("credentials", 14);
+            fieldMap.put("optionsJson", 15);
+            fieldMap.put("context", 16);
+        }
+    }
+
+    public static final class QueryContextInformation
+    {
+        public static final org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.MessageSchema WRITE =
+            new org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.MessageSchema();
+        public static final org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.BuilderSchema MERGE =
+            new org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.BuilderSchema();
+        
+        public static class MessageSchema implements com.dyuproject.protostuff.Schema<org.apache.drill.exec.proto.BitControl.QueryContextInformation>
+        {
+            public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.exec.proto.BitControl.QueryContextInformation message) throws java.io.IOException
+            {
+                if(message.hasQueryStartTime())
+                    output.writeInt64(1, message.getQueryStartTime(), false);
+                if(message.hasTimeZone())
+                    output.writeInt32(2, message.getTimeZone(), false);
+                if(message.hasDefaultSchemaName())
+                    output.writeString(3, message.getDefaultSchemaName(), false);
+            }
+            public boolean isInitialized(org.apache.drill.exec.proto.BitControl.QueryContextInformation message)
+            {
+                return message.isInitialized();
+            }
+            public java.lang.String getFieldName(int number)
+            {
+                return org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.getFieldName(number);
+            }
+            public int getFieldNumber(java.lang.String name)
+            {
+                return org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.getFieldNumber(name);
+            }
+            public java.lang.Class<org.apache.drill.exec.proto.BitControl.QueryContextInformation> typeClass()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.class;
+            }
+            public java.lang.String messageName()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.class.getSimpleName();
+            }
+            public java.lang.String messageFullName()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.class.getName();
+            }
+            //unused
+            public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.exec.proto.BitControl.QueryContextInformation message) throws java.io.IOException {}
+            public org.apache.drill.exec.proto.BitControl.QueryContextInformation newMessage() { return null; }
+        }
+        public static class BuilderSchema implements com.dyuproject.protostuff.Schema<org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder>
+        {
+            public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder builder) throws java.io.IOException
+            {
+                for(int number = input.readFieldNumber(this);; number = input.readFieldNumber(this))
+                {
+                    switch(number)
+                    {
+                        case 0:
+                            return;
+                        case 1:
+                            builder.setQueryStartTime(input.readInt64());
+                            break;
+                        case 2:
+                            builder.setTimeZone(input.readInt32());
+                            break;
+                        case 3:
+                            builder.setDefaultSchemaName(input.readString());
+                            break;
+                        default:
+                            input.handleUnknownField(number, this);
+                    }
+                }
+            }
+            public boolean isInitialized(org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder builder)
+            {
+                return builder.isInitialized();
+            }
+            public org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder newMessage()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.newBuilder();
+            }
+            public java.lang.String getFieldName(int number)
+            {
+                return org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.getFieldName(number);
+            }
+            public int getFieldNumber(java.lang.String name)
+            {
+                return org.apache.drill.exec.proto.SchemaBitControl.QueryContextInformation.getFieldNumber(name);
+            }
+            public java.lang.Class<org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder> typeClass()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder.class;
+            }
+            public java.lang.String messageName()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.class.getSimpleName();
+            }
+            public java.lang.String messageFullName()
+            {
+                return org.apache.drill.exec.proto.BitControl.QueryContextInformation.class.getName();
+            }
+            //unused
+            public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder builder) throws java.io.IOException {}
+        }
+        public static java.lang.String getFieldName(int number)
+        {
+            switch(number)
+            {
+                case 1: return "queryStartTime";
+                case 2: return "timeZone";
+                case 3: return "defaultSchemaName";
+                default: return null;
+            }
+        }
+        public static int getFieldNumber(java.lang.String name)
+        {
+            java.lang.Integer number = fieldMap.get(name);
+            return number == null ? 0 : number.intValue();
+        }
+        private static final java.util.HashMap<java.lang.String,java.lang.Integer> fieldMap = new java.util.HashMap<java.lang.String,java.lang.Integer>();
+        static
+        {
+            fieldMap.put("queryStartTime", 1);
+            fieldMap.put("timeZone", 2);
+            fieldMap.put("defaultSchemaName", 3);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java
index f6fbce1..08c2073 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/PlanFragment.java
@@ -60,10 +60,9 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
     private DrillbitEndpoint foreman;
     private long memInitial = DEFAULT_MEM_INITIAL;
     private long memMax = DEFAULT_MEM_MAX;
-    private long queryStartTime;
     private UserCredentials credentials;
-    private int timeZone;
     private String optionsJson;
+    private QueryContextInformation context;
 
     public PlanFragment()
     {
@@ -215,19 +214,6 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
         return this;
     }
 
-    // queryStartTime
-
-    public long getQueryStartTime()
-    {
-        return queryStartTime;
-    }
-
-    public PlanFragment setQueryStartTime(long queryStartTime)
-    {
-        this.queryStartTime = queryStartTime;
-        return this;
-    }
-
     // credentials
 
     public UserCredentials getCredentials()
@@ -241,29 +227,29 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
         return this;
     }
 
-    // timeZone
+    // optionsJson
 
-    public int getTimeZone()
+    public String getOptionsJson()
     {
-        return timeZone;
+        return optionsJson;
     }
 
-    public PlanFragment setTimeZone(int timeZone)
+    public PlanFragment setOptionsJson(String optionsJson)
     {
-        this.timeZone = timeZone;
+        this.optionsJson = optionsJson;
         return this;
     }
 
-    // optionsJson
+    // context
 
-    public String getOptionsJson()
+    public QueryContextInformation getContext()
     {
-        return optionsJson;
+        return context;
     }
 
-    public PlanFragment setOptionsJson(String optionsJson)
+    public PlanFragment setContext(QueryContextInformation context)
     {
-        this.optionsJson = optionsJson;
+        this.context = context;
         return this;
     }
 
@@ -358,18 +344,16 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
                     message.memMax = input.readInt64();
                     break;
                 case 14:
-                    message.queryStartTime = input.readInt64();
-                    break;
-                case 15:
                     message.credentials = input.mergeObject(message.credentials, UserCredentials.getSchema());
                     break;
 
-                case 16:
-                    message.timeZone = input.readInt32();
-                    break;
-                case 17:
+                case 15:
                     message.optionsJson = input.readString();
                     break;
+                case 16:
+                    message.context = input.mergeObject(message.context, QueryContextInformation.getSchema());
+                    break;
+
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -415,18 +399,16 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
         if(message.memMax != DEFAULT_MEM_MAX)
             output.writeInt64(13, message.memMax, false);
 
-        if(message.queryStartTime != 0)
-            output.writeInt64(14, message.queryStartTime, false);
-
         if(message.credentials != null)
-             output.writeObject(15, message.credentials, UserCredentials.getSchema(), false);
+             output.writeObject(14, message.credentials, UserCredentials.getSchema(), false);
 
 
-        if(message.timeZone != 0)
-            output.writeInt32(16, message.timeZone, false);
-
         if(message.optionsJson != null)
-            output.writeString(17, message.optionsJson, false);
+            output.writeString(15, message.optionsJson, false);
+
+        if(message.context != null)
+             output.writeObject(16, message.context, QueryContextInformation.getSchema(), false);
+
     }
 
     public String getFieldName(int number)
@@ -444,10 +426,9 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
             case 11: return "foreman";
             case 12: return "memInitial";
             case 13: return "memMax";
-            case 14: return "queryStartTime";
-            case 15: return "credentials";
-            case 16: return "timeZone";
-            case 17: return "optionsJson";
+            case 14: return "credentials";
+            case 15: return "optionsJson";
+            case 16: return "context";
             default: return null;
         }
     }
@@ -472,10 +453,9 @@ public final class PlanFragment implements Externalizable, Message<PlanFragment>
         __fieldMap.put("foreman", 11);
         __fieldMap.put("memInitial", 12);
         __fieldMap.put("memMax", 13);
-        __fieldMap.put("queryStartTime", 14);
-        __fieldMap.put("credentials", 15);
-        __fieldMap.put("timeZone", 16);
-        __fieldMap.put("optionsJson", 17);
+        __fieldMap.put("credentials", 14);
+        __fieldMap.put("optionsJson", 15);
+        __fieldMap.put("context", 16);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java
new file mode 100644
index 0000000..440968b
--- /dev/null
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryContextInformation.java
@@ -0,0 +1,207 @@
+/**
+ * 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.
+ */
+// Generated by http://code.google.com/p/protostuff/ ... DO NOT EDIT!
+// Generated from protobuf
+
+package org.apache.drill.exec.proto.beans;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+import com.dyuproject.protostuff.GraphIOUtil;
+import com.dyuproject.protostuff.Input;
+import com.dyuproject.protostuff.Message;
+import com.dyuproject.protostuff.Output;
+import com.dyuproject.protostuff.Schema;
+
+public final class QueryContextInformation implements Externalizable, Message<QueryContextInformation>, Schema<QueryContextInformation>
+{
+
+    public static Schema<QueryContextInformation> getSchema()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    public static QueryContextInformation getDefaultInstance()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    static final QueryContextInformation DEFAULT_INSTANCE = new QueryContextInformation();
+
+    
+    private long queryStartTime;
+    private int timeZone;
+    private String defaultSchemaName;
+
+    public QueryContextInformation()
+    {
+        
+    }
+
+    // getters and setters
+
+    // queryStartTime
+
+    public long getQueryStartTime()
+    {
+        return queryStartTime;
+    }
+
+    public QueryContextInformation setQueryStartTime(long queryStartTime)
+    {
+        this.queryStartTime = queryStartTime;
+        return this;
+    }
+
+    // timeZone
+
+    public int getTimeZone()
+    {
+        return timeZone;
+    }
+
+    public QueryContextInformation setTimeZone(int timeZone)
+    {
+        this.timeZone = timeZone;
+        return this;
+    }
+
+    // defaultSchemaName
+
+    public String getDefaultSchemaName()
+    {
+        return defaultSchemaName;
+    }
+
+    public QueryContextInformation setDefaultSchemaName(String defaultSchemaName)
+    {
+        this.defaultSchemaName = defaultSchemaName;
+        return this;
+    }
+
+    // java serialization
+
+    public void readExternal(ObjectInput in) throws IOException
+    {
+        GraphIOUtil.mergeDelimitedFrom(in, this, this);
+    }
+
+    public void writeExternal(ObjectOutput out) throws IOException
+    {
+        GraphIOUtil.writeDelimitedTo(out, this, this);
+    }
+
+    // message method
+
+    public Schema<QueryContextInformation> cachedSchema()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    // schema methods
+
+    public QueryContextInformation newMessage()
+    {
+        return new QueryContextInformation();
+    }
+
+    public Class<QueryContextInformation> typeClass()
+    {
+        return QueryContextInformation.class;
+    }
+
+    public String messageName()
+    {
+        return QueryContextInformation.class.getSimpleName();
+    }
+
+    public String messageFullName()
+    {
+        return QueryContextInformation.class.getName();
+    }
+
+    public boolean isInitialized(QueryContextInformation message)
+    {
+        return true;
+    }
+
+    public void mergeFrom(Input input, QueryContextInformation message) throws IOException
+    {
+        for(int number = input.readFieldNumber(this);; number = input.readFieldNumber(this))
+        {
+            switch(number)
+            {
+                case 0:
+                    return;
+                case 1:
+                    message.queryStartTime = input.readInt64();
+                    break;
+                case 2:
+                    message.timeZone = input.readInt32();
+                    break;
+                case 3:
+                    message.defaultSchemaName = input.readString();
+                    break;
+                default:
+                    input.handleUnknownField(number, this);
+            }   
+        }
+    }
+
+
+    public void writeTo(Output output, QueryContextInformation message) throws IOException
+    {
+        if(message.queryStartTime != 0)
+            output.writeInt64(1, message.queryStartTime, false);
+
+        if(message.timeZone != 0)
+            output.writeInt32(2, message.timeZone, false);
+
+        if(message.defaultSchemaName != null)
+            output.writeString(3, message.defaultSchemaName, false);
+    }
+
+    public String getFieldName(int number)
+    {
+        switch(number)
+        {
+            case 1: return "queryStartTime";
+            case 2: return "timeZone";
+            case 3: return "defaultSchemaName";
+            default: return null;
+        }
+    }
+
+    public int getFieldNumber(String name)
+    {
+        final Integer number = __fieldMap.get(name);
+        return number == null ? 0 : number.intValue();
+    }
+
+    private static final java.util.HashMap<String,Integer> __fieldMap = new java.util.HashMap<String,Integer>();
+    static
+    {
+        __fieldMap.put("queryStartTime", 1);
+        __fieldMap.put("timeZone", 2);
+        __fieldMap.put("defaultSchemaName", 3);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/protocol/src/main/protobuf/BitControl.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
index 0424725..93bc33c 100644
--- a/protocol/src/main/protobuf/BitControl.proto
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -59,15 +59,20 @@ message PlanFragment {
   optional float disk_cost = 6;
   optional float memory_cost = 7;
   optional string fragment_json = 8;
-  optional DrillbitEndpoint assignment = 10;
   optional bool leaf_fragment = 9;
+  optional DrillbitEndpoint assignment = 10;
   optional DrillbitEndpoint foreman = 11;
   optional int64 mem_initial = 12 [default = 20000000]; // 20 megs
   optional int64 mem_max = 13 [default = 2000000000]; // 20 gigs
-  optional int64 query_start_time = 14; // start time of query in milliseconds
-  optional exec.shared.UserCredentials credentials = 15;
-  optional int32 time_zone = 16;
-  optional string options_json = 17;
+  optional exec.shared.UserCredentials credentials = 14;
+  optional string options_json = 15;
+  optional QueryContextInformation context = 16;
+}
+
+message QueryContextInformation {
+  optional int64 query_start_time = 1;      // start time of query in milliseconds
+  optional int32 time_zone = 2;             // timezone of the Drillbit where user is connected
+  optional string default_schema_name = 3;  // default schema in current session when the query is submitted
 }
 
 message WorkQueueStatus{


[5/5] drill git commit: DRILL-2423: Show proper message when trying to drop an unknown view.

Posted by ve...@apache.org.
DRILL-2423: Show proper message when trying to drop an unknown view.


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

Branch: refs/heads/master
Commit: d43324f895172dcdbb5070faef37470d6bc107f5
Parents: ed02612
Author: vkorukanti <ve...@gmail.com>
Authored: Sat Apr 11 00:11:02 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Tue May 5 16:28:28 2015 -0700

----------------------------------------------------------------------
 .../exec/planner/sql/handlers/ViewHandler.java  | 27 ++++++++++----
 .../drill/exec/sql/TestBaseViewSupport.java     |  2 +-
 .../apache/drill/exec/sql/TestViewSupport.java  | 39 ++++++++++++++++++++
 3 files changed, 59 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d43324f8/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
index 2428b45..c59c3a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.tools.Planner;
@@ -134,6 +135,7 @@ public abstract class ViewHandler extends AbstractSqlHandler {
     @Override
     public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException, ForemanSetupException {
       SqlDropView dropView = unwrap(sqlNode, SqlDropView.class);
+      final String viewToDrop = dropView.getName();
 
       try {
         SchemaPlus schema = findSchema(context.getRootSchema(), context.getNewDefaultSchema(), dropView.getSchemaPath());
@@ -141,20 +143,29 @@ public abstract class ViewHandler extends AbstractSqlHandler {
 
         String schemaPath = drillSchema.getFullSchemaName();
         if (!drillSchema.isMutable()) {
-          return DirectPlan.createDirectPlan(context, false, String.format("Schema '%s' is not a mutable schema. " +
-              "Views don't exist in this schema", schemaPath));
+          return DirectPlan.createDirectPlan(context, false, String.format("Schema [%s] is immutable.", schemaPath));
         }
 
-        if (drillSchema instanceof WorkspaceSchema) {
-          ((WorkspaceSchema) drillSchema).dropView(dropView.getName());;
-        } else {
-          return DirectPlan.createDirectPlan(context, false, "Schema provided was not a workspace schema.");
+        if (!(drillSchema instanceof WorkspaceSchema)) {
+          return DirectPlan.createDirectPlan(context, false,
+              String.format("Schema [%s] doesn't support creating/dropping views.", schemaPath));
         }
 
+        final Table existingTable = SqlHandlerUtil.getTableFromSchema(drillSchema, viewToDrop);
+        if (existingTable != null && existingTable.getJdbcTableType() != Schema.TableType.VIEW) {
+          return DirectPlan.createDirectPlan(context, false,
+              String.format("[%s] is not a VIEW in schema [%s]", viewToDrop, schemaPath));
+        } else if (existingTable == null) {
+          return DirectPlan.createDirectPlan(context, false,
+              String.format("Unknown view [%s] in schema [%s].", viewToDrop, schemaPath));
+        }
+
+        ((WorkspaceSchema) drillSchema).dropView(viewToDrop);
+
         return DirectPlan.createDirectPlan(context, true,
-            String.format("View '%s' deleted successfully from '%s' schema", dropView.getName(), schemaPath));
+            String.format("View [%s] deleted successfully from schema [%s].", viewToDrop, schemaPath));
       } catch(Exception e) {
-        logger.debug("Failed to delete view {}", dropView.getName(), e);
+        logger.debug("Failed to delete view {}", viewToDrop, e);
         return DirectPlan.createDirectPlan(context, false, String.format("Error: %s", e.getMessage()));
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/d43324f8/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
index bfe113b..a8f5bbb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
@@ -100,7 +100,7 @@ public class TestBaseViewSupport extends BaseTestQuery {
         .sqlQuery(String.format("DROP VIEW %s", viewFullName))
         .unOrdered()
         .baselineColumns("ok", "summary")
-        .baselineValues(true, String.format("View '%s' deleted successfully from '%s' schema", viewName, finalSchema))
+        .baselineValues(true, String.format("View [%s] deleted successfully from schema [%s].", viewName, finalSchema))
         .go();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d43324f8/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index 5c2dc90..0fc1f32 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -590,4 +590,43 @@ public class TestViewSupport extends TestBaseViewSupport {
         .baselineValues(false, errorMsg)
         .go();
   }
+
+  @Test // DRILL-2423
+  public void showProperMsgWhenDroppingNonExistentView() throws Exception{
+    testBuilder()
+        .sqlQuery("DROP VIEW dfs_test.tmp.nonExistentView")
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, "Unknown view [nonExistentView] in schema [dfs_test.tmp].")
+        .go();
+  }
+
+  @Test // DRILL-2423
+  public void showProperMsgWhenTryingToDropAViewInImmutableSchema() throws Exception{
+    testBuilder()
+        .sqlQuery("DROP VIEW cp.nonExistentView")
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(false, "Schema [cp.default] is immutable.")
+        .go();
+  }
+
+  @Test // DRILL-2423
+  public void showProperMsgWhenTryingToDropANonViewTable() throws Exception{
+    final String testTableName = "testTableShowErrorMsg";
+    try {
+      test(String.format("CREATE TABLE %s.%s AS SELECT c_custkey, c_nationkey from cp.`tpch/customer.parquet`",
+          TEMP_SCHEMA, testTableName));
+
+      testBuilder()
+          .sqlQuery(String.format("DROP VIEW %s.%s", TEMP_SCHEMA, testTableName))
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(false, "[testTableShowErrorMsg] is not a VIEW in schema [dfs_test.tmp]")
+          .go();
+    } finally {
+      File tblPath = new File(getDfsTestTmpSchemaLocation(), testTableName);
+      FileUtils.deleteQuietly(tblPath);
+    }
+  }
 }


[3/5] drill git commit: DRILL-2902: Add support for context functions: user (synonyms session_user and system_user) and current_schema

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
index 813d961..470e976 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -3056,6 +3056,16 @@ public final class BitControl {
     com.google.protobuf.ByteString
         getFragmentJsonBytes();
 
+    // optional bool leaf_fragment = 9;
+    /**
+     * <code>optional bool leaf_fragment = 9;</code>
+     */
+    boolean hasLeafFragment();
+    /**
+     * <code>optional bool leaf_fragment = 9;</code>
+     */
+    boolean getLeafFragment();
+
     // optional .exec.DrillbitEndpoint assignment = 10;
     /**
      * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
@@ -3070,16 +3080,6 @@ public final class BitControl {
      */
     org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getAssignmentOrBuilder();
 
-    // optional bool leaf_fragment = 9;
-    /**
-     * <code>optional bool leaf_fragment = 9;</code>
-     */
-    boolean hasLeafFragment();
-    /**
-     * <code>optional bool leaf_fragment = 9;</code>
-     */
-    boolean getLeafFragment();
-
     // optional .exec.DrillbitEndpoint foreman = 11;
     /**
      * <code>optional .exec.DrillbitEndpoint foreman = 11;</code>
@@ -3130,62 +3130,48 @@ public final class BitControl {
      */
     long getMemMax();
 
-    // optional int64 query_start_time = 14;
-    /**
-     * <code>optional int64 query_start_time = 14;</code>
-     *
-     * <pre>
-     * start time of query in milliseconds
-     * </pre>
-     */
-    boolean hasQueryStartTime();
-    /**
-     * <code>optional int64 query_start_time = 14;</code>
-     *
-     * <pre>
-     * start time of query in milliseconds
-     * </pre>
-     */
-    long getQueryStartTime();
-
-    // optional .exec.shared.UserCredentials credentials = 15;
+    // optional .exec.shared.UserCredentials credentials = 14;
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     boolean hasCredentials();
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials();
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder();
 
-    // optional int32 time_zone = 16;
+    // optional string options_json = 15;
     /**
-     * <code>optional int32 time_zone = 16;</code>
+     * <code>optional string options_json = 15;</code>
      */
-    boolean hasTimeZone();
+    boolean hasOptionsJson();
     /**
-     * <code>optional int32 time_zone = 16;</code>
+     * <code>optional string options_json = 15;</code>
      */
-    int getTimeZone();
+    java.lang.String getOptionsJson();
+    /**
+     * <code>optional string options_json = 15;</code>
+     */
+    com.google.protobuf.ByteString
+        getOptionsJsonBytes();
 
-    // optional string options_json = 17;
+    // optional .exec.bit.control.QueryContextInformation context = 16;
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
      */
-    boolean hasOptionsJson();
+    boolean hasContext();
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
      */
-    java.lang.String getOptionsJson();
+    org.apache.drill.exec.proto.BitControl.QueryContextInformation getContext();
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
      */
-    com.google.protobuf.ByteString
-        getOptionsJsonBytes();
+    org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder getContextOrBuilder();
   }
   /**
    * Protobuf type {@code exec.bit.control.PlanFragment}
@@ -3277,13 +3263,13 @@ public final class BitControl {
               break;
             }
             case 72: {
-              bitField0_ |= 0x00000080;
+              bitField0_ |= 0x00000040;
               leafFragment_ = input.readBool();
               break;
             }
             case 82: {
               org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000040) == 0x00000040)) {
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
                 subBuilder = assignment_.toBuilder();
               }
               assignment_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
@@ -3291,7 +3277,7 @@ public final class BitControl {
                 subBuilder.mergeFrom(assignment_);
                 assignment_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00000040;
+              bitField0_ |= 0x00000080;
               break;
             }
             case 90: {
@@ -3317,14 +3303,9 @@ public final class BitControl {
               memMax_ = input.readInt64();
               break;
             }
-            case 112: {
-              bitField0_ |= 0x00000800;
-              queryStartTime_ = input.readInt64();
-              break;
-            }
-            case 122: {
+            case 114: {
               org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder subBuilder = null;
-              if (((bitField0_ & 0x00001000) == 0x00001000)) {
+              if (((bitField0_ & 0x00000800) == 0x00000800)) {
                 subBuilder = credentials_.toBuilder();
               }
               credentials_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.UserCredentials.PARSER, extensionRegistry);
@@ -3332,17 +3313,25 @@ public final class BitControl {
                 subBuilder.mergeFrom(credentials_);
                 credentials_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00001000;
+              bitField0_ |= 0x00000800;
               break;
             }
-            case 128: {
-              bitField0_ |= 0x00002000;
-              timeZone_ = input.readInt32();
+            case 122: {
+              bitField0_ |= 0x00001000;
+              optionsJson_ = input.readBytes();
               break;
             }
-            case 138: {
-              bitField0_ |= 0x00004000;
-              optionsJson_ = input.readBytes();
+            case 130: {
+              org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00002000) == 0x00002000)) {
+                subBuilder = context_.toBuilder();
+              }
+              context_ = input.readMessage(org.apache.drill.exec.proto.BitControl.QueryContextInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(context_);
+                context_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00002000;
               break;
             }
           }
@@ -3514,6 +3503,22 @@ public final class BitControl {
       }
     }
 
+    // optional bool leaf_fragment = 9;
+    public static final int LEAF_FRAGMENT_FIELD_NUMBER = 9;
+    private boolean leafFragment_;
+    /**
+     * <code>optional bool leaf_fragment = 9;</code>
+     */
+    public boolean hasLeafFragment() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional bool leaf_fragment = 9;</code>
+     */
+    public boolean getLeafFragment() {
+      return leafFragment_;
+    }
+
     // optional .exec.DrillbitEndpoint assignment = 10;
     public static final int ASSIGNMENT_FIELD_NUMBER = 10;
     private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_;
@@ -3521,7 +3526,7 @@ public final class BitControl {
      * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
      */
     public boolean hasAssignment() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+      return ((bitField0_ & 0x00000080) == 0x00000080);
     }
     /**
      * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
@@ -3536,22 +3541,6 @@ public final class BitControl {
       return assignment_;
     }
 
-    // optional bool leaf_fragment = 9;
-    public static final int LEAF_FRAGMENT_FIELD_NUMBER = 9;
-    private boolean leafFragment_;
-    /**
-     * <code>optional bool leaf_fragment = 9;</code>
-     */
-    public boolean hasLeafFragment() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional bool leaf_fragment = 9;</code>
-     */
-    public boolean getLeafFragment() {
-      return leafFragment_;
-    }
-
     // optional .exec.DrillbitEndpoint foreman = 11;
     public static final int FOREMAN_FIELD_NUMBER = 11;
     private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_;
@@ -3622,79 +3611,39 @@ public final class BitControl {
       return memMax_;
     }
 
-    // optional int64 query_start_time = 14;
-    public static final int QUERY_START_TIME_FIELD_NUMBER = 14;
-    private long queryStartTime_;
-    /**
-     * <code>optional int64 query_start_time = 14;</code>
-     *
-     * <pre>
-     * start time of query in milliseconds
-     * </pre>
-     */
-    public boolean hasQueryStartTime() {
-      return ((bitField0_ & 0x00000800) == 0x00000800);
-    }
-    /**
-     * <code>optional int64 query_start_time = 14;</code>
-     *
-     * <pre>
-     * start time of query in milliseconds
-     * </pre>
-     */
-    public long getQueryStartTime() {
-      return queryStartTime_;
-    }
-
-    // optional .exec.shared.UserCredentials credentials = 15;
-    public static final int CREDENTIALS_FIELD_NUMBER = 15;
+    // optional .exec.shared.UserCredentials credentials = 14;
+    public static final int CREDENTIALS_FIELD_NUMBER = 14;
     private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_;
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     public boolean hasCredentials() {
-      return ((bitField0_ & 0x00001000) == 0x00001000);
+      return ((bitField0_ & 0x00000800) == 0x00000800);
     }
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
       return credentials_;
     }
     /**
-     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
       return credentials_;
     }
 
-    // optional int32 time_zone = 16;
-    public static final int TIME_ZONE_FIELD_NUMBER = 16;
-    private int timeZone_;
-    /**
-     * <code>optional int32 time_zone = 16;</code>
-     */
-    public boolean hasTimeZone() {
-      return ((bitField0_ & 0x00002000) == 0x00002000);
-    }
-    /**
-     * <code>optional int32 time_zone = 16;</code>
-     */
-    public int getTimeZone() {
-      return timeZone_;
-    }
-
-    // optional string options_json = 17;
-    public static final int OPTIONS_JSON_FIELD_NUMBER = 17;
+    // optional string options_json = 15;
+    public static final int OPTIONS_JSON_FIELD_NUMBER = 15;
     private java.lang.Object optionsJson_;
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional string options_json = 15;</code>
      */
     public boolean hasOptionsJson() {
-      return ((bitField0_ & 0x00004000) == 0x00004000);
+      return ((bitField0_ & 0x00001000) == 0x00001000);
     }
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional string options_json = 15;</code>
      */
     public java.lang.String getOptionsJson() {
       java.lang.Object ref = optionsJson_;
@@ -3711,7 +3660,7 @@ public final class BitControl {
       }
     }
     /**
-     * <code>optional string options_json = 17;</code>
+     * <code>optional string options_json = 15;</code>
      */
     public com.google.protobuf.ByteString
         getOptionsJsonBytes() {
@@ -3727,6 +3676,28 @@ public final class BitControl {
       }
     }
 
+    // optional .exec.bit.control.QueryContextInformation context = 16;
+    public static final int CONTEXT_FIELD_NUMBER = 16;
+    private org.apache.drill.exec.proto.BitControl.QueryContextInformation context_;
+    /**
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+     */
+    public boolean hasContext() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+     */
+    public org.apache.drill.exec.proto.BitControl.QueryContextInformation getContext() {
+      return context_;
+    }
+    /**
+     * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+     */
+    public org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder getContextOrBuilder() {
+      return context_;
+    }
+
     private void initFields() {
       handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
       networkCost_ = 0F;
@@ -3734,15 +3705,14 @@ public final class BitControl {
       diskCost_ = 0F;
       memoryCost_ = 0F;
       fragmentJson_ = "";
-      assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
       leafFragment_ = false;
+      assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
       foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
       memInitial_ = 20000000L;
       memMax_ = 2000000000L;
-      queryStartTime_ = 0L;
       credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
-      timeZone_ = 0;
       optionsJson_ = "";
+      context_ = org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3774,10 +3744,10 @@ public final class BitControl {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeBytes(8, getFragmentJsonBytes());
       }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
         output.writeBool(9, leafFragment_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
         output.writeMessage(10, assignment_);
       }
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
@@ -3790,16 +3760,13 @@ public final class BitControl {
         output.writeInt64(13, memMax_);
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
-        output.writeInt64(14, queryStartTime_);
+        output.writeMessage(14, credentials_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
-        output.writeMessage(15, credentials_);
+        output.writeBytes(15, getOptionsJsonBytes());
       }
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
-        output.writeInt32(16, timeZone_);
-      }
-      if (((bitField0_ & 0x00004000) == 0x00004000)) {
-        output.writeBytes(17, getOptionsJsonBytes());
+        output.writeMessage(16, context_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -3834,11 +3801,11 @@ public final class BitControl {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(8, getFragmentJsonBytes());
       }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(9, leafFragment_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(10, assignment_);
       }
@@ -3856,19 +3823,15 @@ public final class BitControl {
       }
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(14, queryStartTime_);
+          .computeMessageSize(14, credentials_);
       }
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(15, credentials_);
+          .computeBytesSize(15, getOptionsJsonBytes());
       }
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(16, timeZone_);
-      }
-      if (((bitField0_ & 0x00004000) == 0x00004000)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(17, getOptionsJsonBytes());
+          .computeMessageSize(16, context_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -3982,6 +3945,7 @@ public final class BitControl {
           getAssignmentFieldBuilder();
           getForemanFieldBuilder();
           getCredentialsFieldBuilder();
+          getContextFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4006,13 +3970,13 @@ public final class BitControl {
         bitField0_ = (bitField0_ & ~0x00000010);
         fragmentJson_ = "";
         bitField0_ = (bitField0_ & ~0x00000020);
+        leafFragment_ = false;
+        bitField0_ = (bitField0_ & ~0x00000040);
         if (assignmentBuilder_ == null) {
           assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
         } else {
           assignmentBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
-        leafFragment_ = false;
         bitField0_ = (bitField0_ & ~0x00000080);
         if (foremanBuilder_ == null) {
           foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
@@ -4024,18 +3988,20 @@ public final class BitControl {
         bitField0_ = (bitField0_ & ~0x00000200);
         memMax_ = 2000000000L;
         bitField0_ = (bitField0_ & ~0x00000400);
-        queryStartTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000800);
         if (credentialsBuilder_ == null) {
           credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
         } else {
           credentialsBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000800);
+        optionsJson_ = "";
         bitField0_ = (bitField0_ & ~0x00001000);
-        timeZone_ = 0;
+        if (contextBuilder_ == null) {
+          context_ = org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance();
+        } else {
+          contextBuilder_.clear();
+        }
         bitField0_ = (bitField0_ & ~0x00002000);
-        optionsJson_ = "";
-        bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
 
@@ -4095,15 +4061,15 @@ public final class BitControl {
         if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000040;
         }
+        result.leafFragment_ = leafFragment_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
         if (assignmentBuilder_ == null) {
           result.assignment_ = assignment_;
         } else {
           result.assignment_ = assignmentBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        result.leafFragment_ = leafFragment_;
         if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
           to_bitField0_ |= 0x00000100;
         }
@@ -4123,23 +4089,23 @@ public final class BitControl {
         if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
           to_bitField0_ |= 0x00000800;
         }
-        result.queryStartTime_ = queryStartTime_;
-        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
-          to_bitField0_ |= 0x00001000;
-        }
         if (credentialsBuilder_ == null) {
           result.credentials_ = credentials_;
         } else {
           result.credentials_ = credentialsBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+          to_bitField0_ |= 0x00001000;
+        }
+        result.optionsJson_ = optionsJson_;
         if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
           to_bitField0_ |= 0x00002000;
         }
-        result.timeZone_ = timeZone_;
-        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
-          to_bitField0_ |= 0x00004000;
+        if (contextBuilder_ == null) {
+          result.context_ = context_;
+        } else {
+          result.context_ = contextBuilder_.build();
         }
-        result.optionsJson_ = optionsJson_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4176,12 +4142,12 @@ public final class BitControl {
           fragmentJson_ = other.fragmentJson_;
           onChanged();
         }
-        if (other.hasAssignment()) {
-          mergeAssignment(other.getAssignment());
-        }
         if (other.hasLeafFragment()) {
           setLeafFragment(other.getLeafFragment());
         }
+        if (other.hasAssignment()) {
+          mergeAssignment(other.getAssignment());
+        }
         if (other.hasForeman()) {
           mergeForeman(other.getForeman());
         }
@@ -4191,20 +4157,17 @@ public final class BitControl {
         if (other.hasMemMax()) {
           setMemMax(other.getMemMax());
         }
-        if (other.hasQueryStartTime()) {
-          setQueryStartTime(other.getQueryStartTime());
-        }
         if (other.hasCredentials()) {
           mergeCredentials(other.getCredentials());
         }
-        if (other.hasTimeZone()) {
-          setTimeZone(other.getTimeZone());
-        }
         if (other.hasOptionsJson()) {
-          bitField0_ |= 0x00004000;
+          bitField0_ |= 0x00001000;
           optionsJson_ = other.optionsJson_;
           onChanged();
         }
+        if (other.hasContext()) {
+          mergeContext(other.getContext());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4555,6 +4518,39 @@ public final class BitControl {
         return this;
       }
 
+      // optional bool leaf_fragment = 9;
+      private boolean leafFragment_ ;
+      /**
+       * <code>optional bool leaf_fragment = 9;</code>
+       */
+      public boolean hasLeafFragment() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bool leaf_fragment = 9;</code>
+       */
+      public boolean getLeafFragment() {
+        return leafFragment_;
+      }
+      /**
+       * <code>optional bool leaf_fragment = 9;</code>
+       */
+      public Builder setLeafFragment(boolean value) {
+        bitField0_ |= 0x00000040;
+        leafFragment_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool leaf_fragment = 9;</code>
+       */
+      public Builder clearLeafFragment() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        leafFragment_ = false;
+        onChanged();
+        return this;
+      }
+
       // optional .exec.DrillbitEndpoint assignment = 10;
       private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint assignment_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
@@ -4563,7 +4559,7 @@ public final class BitControl {
        * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
        */
       public boolean hasAssignment() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
+        return ((bitField0_ & 0x00000080) == 0x00000080);
       }
       /**
        * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
@@ -4588,7 +4584,7 @@ public final class BitControl {
         } else {
           assignmentBuilder_.setMessage(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
@@ -4602,7 +4598,7 @@ public final class BitControl {
         } else {
           assignmentBuilder_.setMessage(builderForValue.build());
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
@@ -4610,7 +4606,7 @@ public final class BitControl {
        */
       public Builder mergeAssignment(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
         if (assignmentBuilder_ == null) {
-          if (((bitField0_ & 0x00000040) == 0x00000040) &&
+          if (((bitField0_ & 0x00000080) == 0x00000080) &&
               assignment_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
             assignment_ =
               org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(assignment_).mergeFrom(value).buildPartial();
@@ -4621,7 +4617,7 @@ public final class BitControl {
         } else {
           assignmentBuilder_.mergeFrom(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         return this;
       }
       /**
@@ -4634,14 +4630,14 @@ public final class BitControl {
         } else {
           assignmentBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
+        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
       /**
        * <code>optional .exec.DrillbitEndpoint assignment = 10;</code>
        */
       public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getAssignmentBuilder() {
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000080;
         onChanged();
         return getAssignmentFieldBuilder().getBuilder();
       }
@@ -4672,39 +4668,6 @@ public final class BitControl {
         return assignmentBuilder_;
       }
 
-      // optional bool leaf_fragment = 9;
-      private boolean leafFragment_ ;
-      /**
-       * <code>optional bool leaf_fragment = 9;</code>
-       */
-      public boolean hasLeafFragment() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional bool leaf_fragment = 9;</code>
-       */
-      public boolean getLeafFragment() {
-        return leafFragment_;
-      }
-      /**
-       * <code>optional bool leaf_fragment = 9;</code>
-       */
-      public Builder setLeafFragment(boolean value) {
-        bitField0_ |= 0x00000080;
-        leafFragment_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool leaf_fragment = 9;</code>
-       */
-      public Builder clearLeafFragment() {
-        bitField0_ = (bitField0_ & ~0x00000080);
-        leafFragment_ = false;
-        onChanged();
-        return this;
-      }
-
       // optional .exec.DrillbitEndpoint foreman = 11;
       private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
@@ -4871,69 +4834,926 @@ public final class BitControl {
         return this;
       }
 
-      // optional int64 mem_max = 13 [default = 2000000000];
-      private long memMax_ = 2000000000L;
-      /**
-       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
-       *
-       * <pre>
-       * 20 gigs
-       * </pre>
-       */
-      public boolean hasMemMax() {
-        return ((bitField0_ & 0x00000400) == 0x00000400);
-      }
-      /**
-       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
-       *
-       * <pre>
-       * 20 gigs
-       * </pre>
-       */
-      public long getMemMax() {
-        return memMax_;
+      // optional int64 mem_max = 13 [default = 2000000000];
+      private long memMax_ = 2000000000L;
+      /**
+       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
+       *
+       * <pre>
+       * 20 gigs
+       * </pre>
+       */
+      public boolean hasMemMax() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
+       *
+       * <pre>
+       * 20 gigs
+       * </pre>
+       */
+      public long getMemMax() {
+        return memMax_;
+      }
+      /**
+       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
+       *
+       * <pre>
+       * 20 gigs
+       * </pre>
+       */
+      public Builder setMemMax(long value) {
+        bitField0_ |= 0x00000400;
+        memMax_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
+       *
+       * <pre>
+       * 20 gigs
+       * </pre>
+       */
+      public Builder clearMemMax() {
+        bitField0_ = (bitField0_ & ~0x00000400);
+        memMax_ = 2000000000L;
+        onChanged();
+        return this;
+      }
+
+      // optional .exec.shared.UserCredentials credentials = 14;
+      private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> credentialsBuilder_;
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public boolean hasCredentials() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
+        if (credentialsBuilder_ == null) {
+          return credentials_;
+        } else {
+          return credentialsBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public Builder setCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          credentials_ = value;
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public Builder setCredentials(
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder builderForValue) {
+        if (credentialsBuilder_ == null) {
+          credentials_ = builderForValue.build();
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public Builder mergeCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (((bitField0_ & 0x00000800) == 0x00000800) &&
+              credentials_ != org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance()) {
+            credentials_ =
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder(credentials_).mergeFrom(value).buildPartial();
+          } else {
+            credentials_ = value;
+          }
+          onChanged();
+        } else {
+          credentialsBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000800;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public Builder clearCredentials() {
+        if (credentialsBuilder_ == null) {
+          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+          onChanged();
+        } else {
+          credentialsBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000800);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder getCredentialsBuilder() {
+        bitField0_ |= 0x00000800;
+        onChanged();
+        return getCredentialsFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
+        if (credentialsBuilder_ != null) {
+          return credentialsBuilder_.getMessageOrBuilder();
+        } else {
+          return credentials_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 14;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> 
+          getCredentialsFieldBuilder() {
+        if (credentialsBuilder_ == null) {
+          credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder>(
+                  credentials_,
+                  getParentForChildren(),
+                  isClean());
+          credentials_ = null;
+        }
+        return credentialsBuilder_;
+      }
+
+      // optional string options_json = 15;
+      private java.lang.Object optionsJson_ = "";
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public boolean hasOptionsJson() {
+        return ((bitField0_ & 0x00001000) == 0x00001000);
+      }
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public java.lang.String getOptionsJson() {
+        java.lang.Object ref = optionsJson_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          optionsJson_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public com.google.protobuf.ByteString
+          getOptionsJsonBytes() {
+        java.lang.Object ref = optionsJson_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          optionsJson_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public Builder setOptionsJson(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        optionsJson_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public Builder clearOptionsJson() {
+        bitField0_ = (bitField0_ & ~0x00001000);
+        optionsJson_ = getDefaultInstance().getOptionsJson();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string options_json = 15;</code>
+       */
+      public Builder setOptionsJsonBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        optionsJson_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional .exec.bit.control.QueryContextInformation context = 16;
+      private org.apache.drill.exec.proto.BitControl.QueryContextInformation context_ = org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.BitControl.QueryContextInformation, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder, org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder> contextBuilder_;
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public boolean hasContext() {
+        return ((bitField0_ & 0x00002000) == 0x00002000);
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformation getContext() {
+        if (contextBuilder_ == null) {
+          return context_;
+        } else {
+          return contextBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public Builder setContext(org.apache.drill.exec.proto.BitControl.QueryContextInformation value) {
+        if (contextBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          context_ = value;
+          onChanged();
+        } else {
+          contextBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00002000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public Builder setContext(
+          org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder builderForValue) {
+        if (contextBuilder_ == null) {
+          context_ = builderForValue.build();
+          onChanged();
+        } else {
+          contextBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00002000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public Builder mergeContext(org.apache.drill.exec.proto.BitControl.QueryContextInformation value) {
+        if (contextBuilder_ == null) {
+          if (((bitField0_ & 0x00002000) == 0x00002000) &&
+              context_ != org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance()) {
+            context_ =
+              org.apache.drill.exec.proto.BitControl.QueryContextInformation.newBuilder(context_).mergeFrom(value).buildPartial();
+          } else {
+            context_ = value;
+          }
+          onChanged();
+        } else {
+          contextBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00002000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public Builder clearContext() {
+        if (contextBuilder_ == null) {
+          context_ = org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          contextBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00002000);
+        return this;
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder getContextBuilder() {
+        bitField0_ |= 0x00002000;
+        onChanged();
+        return getContextFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder getContextOrBuilder() {
+        if (contextBuilder_ != null) {
+          return contextBuilder_.getMessageOrBuilder();
+        } else {
+          return context_;
+        }
+      }
+      /**
+       * <code>optional .exec.bit.control.QueryContextInformation context = 16;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.BitControl.QueryContextInformation, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder, org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder> 
+          getContextFieldBuilder() {
+        if (contextBuilder_ == null) {
+          contextBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.BitControl.QueryContextInformation, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder, org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder>(
+                  context_,
+                  getParentForChildren(),
+                  isClean());
+          context_ = null;
+        }
+        return contextBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:exec.bit.control.PlanFragment)
+    }
+
+    static {
+      defaultInstance = new PlanFragment(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:exec.bit.control.PlanFragment)
+  }
+
+  public interface QueryContextInformationOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional int64 query_start_time = 1;
+    /**
+     * <code>optional int64 query_start_time = 1;</code>
+     *
+     * <pre>
+     * start time of query in milliseconds
+     * </pre>
+     */
+    boolean hasQueryStartTime();
+    /**
+     * <code>optional int64 query_start_time = 1;</code>
+     *
+     * <pre>
+     * start time of query in milliseconds
+     * </pre>
+     */
+    long getQueryStartTime();
+
+    // optional int32 time_zone = 2;
+    /**
+     * <code>optional int32 time_zone = 2;</code>
+     *
+     * <pre>
+     * timezone of the Drillbit where user is connected
+     * </pre>
+     */
+    boolean hasTimeZone();
+    /**
+     * <code>optional int32 time_zone = 2;</code>
+     *
+     * <pre>
+     * timezone of the Drillbit where user is connected
+     * </pre>
+     */
+    int getTimeZone();
+
+    // optional string default_schema_name = 3;
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    boolean hasDefaultSchemaName();
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    java.lang.String getDefaultSchemaName();
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getDefaultSchemaNameBytes();
+  }
+  /**
+   * Protobuf type {@code exec.bit.control.QueryContextInformation}
+   */
+  public static final class QueryContextInformation extends
+      com.google.protobuf.GeneratedMessage
+      implements QueryContextInformationOrBuilder {
+    // Use QueryContextInformation.newBuilder() to construct.
+    private QueryContextInformation(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private QueryContextInformation(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final QueryContextInformation defaultInstance;
+    public static QueryContextInformation getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public QueryContextInformation getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private QueryContextInformation(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              queryStartTime_ = input.readInt64();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              timeZone_ = input.readInt32();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              defaultSchemaName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_QueryContextInformation_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_QueryContextInformation_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.drill.exec.proto.BitControl.QueryContextInformation.class, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<QueryContextInformation> PARSER =
+        new com.google.protobuf.AbstractParser<QueryContextInformation>() {
+      public QueryContextInformation parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new QueryContextInformation(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<QueryContextInformation> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional int64 query_start_time = 1;
+    public static final int QUERY_START_TIME_FIELD_NUMBER = 1;
+    private long queryStartTime_;
+    /**
+     * <code>optional int64 query_start_time = 1;</code>
+     *
+     * <pre>
+     * start time of query in milliseconds
+     * </pre>
+     */
+    public boolean hasQueryStartTime() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int64 query_start_time = 1;</code>
+     *
+     * <pre>
+     * start time of query in milliseconds
+     * </pre>
+     */
+    public long getQueryStartTime() {
+      return queryStartTime_;
+    }
+
+    // optional int32 time_zone = 2;
+    public static final int TIME_ZONE_FIELD_NUMBER = 2;
+    private int timeZone_;
+    /**
+     * <code>optional int32 time_zone = 2;</code>
+     *
+     * <pre>
+     * timezone of the Drillbit where user is connected
+     * </pre>
+     */
+    public boolean hasTimeZone() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int32 time_zone = 2;</code>
+     *
+     * <pre>
+     * timezone of the Drillbit where user is connected
+     * </pre>
+     */
+    public int getTimeZone() {
+      return timeZone_;
+    }
+
+    // optional string default_schema_name = 3;
+    public static final int DEFAULT_SCHEMA_NAME_FIELD_NUMBER = 3;
+    private java.lang.Object defaultSchemaName_;
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    public boolean hasDefaultSchemaName() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    public java.lang.String getDefaultSchemaName() {
+      java.lang.Object ref = defaultSchemaName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          defaultSchemaName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string default_schema_name = 3;</code>
+     *
+     * <pre>
+     * default schema in current session when the query is submitted
+     * </pre>
+     */
+    public com.google.protobuf.ByteString
+        getDefaultSchemaNameBytes() {
+      java.lang.Object ref = defaultSchemaName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        defaultSchemaName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      queryStartTime_ = 0L;
+      timeZone_ = 0;
+      defaultSchemaName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt64(1, queryStartTime_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(2, timeZone_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, getDefaultSchemaNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(1, queryStartTime_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, timeZone_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, getDefaultSchemaNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.BitControl.QueryContextInformation parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.drill.exec.proto.BitControl.QueryContextInformation prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code exec.bit.control.QueryContextInformation}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.drill.exec.proto.BitControl.QueryContextInformationOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_QueryContextInformation_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_QueryContextInformation_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.drill.exec.proto.BitControl.QueryContextInformation.class, org.apache.drill.exec.proto.BitControl.QueryContextInformation.Builder.class);
+      }
+
+      // Construct using org.apache.drill.exec.proto.BitControl.QueryContextInformation.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        queryStartTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        timeZone_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        defaultSchemaName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.drill.exec.proto.BitControl.internal_static_exec_bit_control_QueryContextInformation_descriptor;
+      }
+
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformation getDefaultInstanceForType() {
+        return org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance();
+      }
+
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformation build() {
+        org.apache.drill.exec.proto.BitControl.QueryContextInformation result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.drill.exec.proto.BitControl.QueryContextInformation buildPartial() {
+        org.apache.drill.exec.proto.BitControl.QueryContextInformation result = new org.apache.drill.exec.proto.BitControl.QueryContextInformation(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.queryStartTime_ = queryStartTime_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.timeZone_ = timeZone_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.defaultSchemaName_ = defaultSchemaName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.drill.exec.proto.BitControl.QueryContextInformation) {
+          return mergeFrom((org.apache.drill.exec.proto.BitControl.QueryContextInformation)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
       }
-      /**
-       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
-       *
-       * <pre>
-       * 20 gigs
-       * </pre>
-       */
-      public Builder setMemMax(long value) {
-        bitField0_ |= 0x00000400;
-        memMax_ = value;
-        onChanged();
+
+      public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.QueryContextInformation other) {
+        if (other == org.apache.drill.exec.proto.BitControl.QueryContextInformation.getDefaultInstance()) return this;
+        if (other.hasQueryStartTime()) {
+          setQueryStartTime(other.getQueryStartTime());
+        }
+        if (other.hasTimeZone()) {
+          setTimeZone(other.getTimeZone());
+        }
+        if (other.hasDefaultSchemaName()) {
+          bitField0_ |= 0x00000004;
+          defaultSchemaName_ = other.defaultSchemaName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
-      /**
-       * <code>optional int64 mem_max = 13 [default = 2000000000];</code>
-       *
-       * <pre>
-       * 20 gigs
-       * </pre>
-       */
-      public Builder clearMemMax() {
-        bitField0_ = (bitField0_ & ~0x00000400);
-        memMax_ = 2000000000L;
-        onChanged();
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.drill.exec.proto.BitControl.QueryContextInformation parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.drill.exec.proto.BitControl.QueryContextInformation) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
         return this;
       }
+      private int bitField0_;
 
-      // optional int64 query_start_time = 14;
+      // optional int64 query_start_time = 1;
       private long queryStartTime_ ;
       /**
-       * <code>optional int64 query_start_time = 14;</code>
+       * <code>optional int64 query_start_time = 1;</code>
        *
        * <pre>
        * start time of query in milliseconds
        * </pre>
        */
       public boolean hasQueryStartTime() {
-        return ((bitField0_ & 0x00000800) == 0x00000800);
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional int64 query_start_time = 14;</code>
+       * <code>optional int64 query_start_time = 1;</code>
        *
        * <pre>
        * start time of query in milliseconds
@@ -4943,265 +5763,188 @@ public final class BitControl {
         return queryStartTime_;
       }
       /**
-       * <code>optional int64 query_start_time = 14;</code>
+       * <code>optional int64 query_start_time = 1;</code>
        *
        * <pre>
        * start time of query in milliseconds
        * </pre>
        */
       public Builder setQueryStartTime(long value) {
-        bitField0_ |= 0x00000800;
+        bitField0_ |= 0x00000001;
         queryStartTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int64 query_start_time = 14;</code>
+       * <code>optional int64 query_start_time = 1;</code>
        *
        * <pre>
        * start time of query in milliseconds
        * </pre>
        */
       public Builder clearQueryStartTime() {
-        bitField0_ = (bitField0_ & ~0x00000800);
+        bitField0_ = (bitField0_ & ~0x00000001);
         queryStartTime_ = 0L;
         onChanged();
         return this;
       }
 
-      // optional .exec.shared.UserCredentials credentials = 15;
-      private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> credentialsBuilder_;
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public boolean hasCredentials() {
-        return ((bitField0_ & 0x00001000) == 0x00001000);
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
-        if (credentialsBuilder_ == null) {
-          return credentials_;
-        } else {
-          return credentialsBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public Builder setCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
-        if (credentialsBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          credentials_ = value;
-          onChanged();
-        } else {
-          credentialsBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00001000;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public Builder setCredentials(
-          org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder builderForValue) {
-        if (credentialsBuilder_ == null) {
-          credentials_ = builderForValue.build();
-          onChanged();
-        } else {
-          credentialsBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00001000;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public Builder mergeCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
-        if (credentialsBuilder_ == null) {
-          if (((bitField0_ & 0x00001000) == 0x00001000) &&
-              credentials_ != org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance()) {
-            credentials_ =
-              org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder(credentials_).mergeFrom(value).buildPartial();
-          } else {
-            credentials_ = value;
-          }
-          onChanged();
-        } else {
-          credentialsBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00001000;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public Builder clearCredentials() {
-        if (credentialsBuilder_ == null) {
-          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
-          onChanged();
-        } else {
-          credentialsBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00001000);
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder getCredentialsBuilder() {
-        bitField0_ |= 0x00001000;
-        onChanged();
-        return getCredentialsFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
-        if (credentialsBuilder_ != null) {
-          return credentialsBuilder_.getMessageOrBuilder();
-        } else {
-          return credentials_;
-        }
-      }
-      /**
-       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> 
-          getCredentialsFieldBuilder() {
-        if (credentialsBuilder_ == null) {
-          credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder>(
-                  credentials_,
-                  getParentForChildren(),
-                  isClean());
-          credentials_ = null;
-        }
-        return credentialsBuilder_;
-      }
-
-      // optional int32 time_zone = 16;
+      // optional int32 time_zone = 2;
       private int timeZone_ ;
       /**
-       * <code>optional int32 time_zone = 16;</code>
+       * <code>optional int32 time_zone = 2;</code>
+       *
+       * <pre>
+       * timezone of the Drillbit where user is connected
+       * </pre>
        */
       public boolean hasTimeZone() {
-        return ((bitField0_ & 0x00002000) == 0x00002000);
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional int32 time_zone = 16;</code>
+       * <code>optional int32 time_zone = 2;</code>
+       *
+       * <pre>
+       * timezone of the Drillbit where user is connected
+       * </pre>
        */
       public int getTimeZone() {
         return timeZone_;
       }
       /**
-       * <code>optional int32 time_zone = 16;</code>
+       * <code>optional int32 time_zone = 2;</code>
+       *
+       * <pre>
+       * timezone of the Drillbit where user is connected
+       * </pre>
        */
       public Builder setTimeZone(int value) {
-        bitField0_ |= 0x00002000;
+        bitField0_ |= 0x00000002;
         timeZone_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 time_zone = 16;</code>
+       * <code>optional int32 time_zone = 2;</code>
+       *
+       * <pre>
+       * timezone of the Drillbit where user is connected
+       * </pre>
        */
       public Builder clearTimeZone() {
-        bitField0_ = (bitField0_ & ~0x00002000);
+        bitField0_ = (bitField0_ & ~0x00000002);
         timeZone_ = 0;
         onChanged();
         return this;
       }
 
-      // optional string options_json = 17;
-      private java.lang.Object optionsJson_ = "";
+      // optional string default_schema_name = 3;
+      private java.lang.Object defaultSchemaName_ = "";
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
-      public boolean hasOptionsJson() {
-        return ((bitField0_ & 0x00004000) == 0x00004000);
+      public boolean hasDefaultSchemaName() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
-      public java.lang.String getOptionsJson() {
-        java.lang.Object ref = optionsJson_;
+      public java.lang.String getDefaultSchemaName() {
+        java.lang.Object ref = defaultSchemaName_;
         if (!(ref instanceof java.lang.String)) {
           java.lang.String s = ((com.google.protobuf.ByteString) ref)
               .toStringUtf8();
-          optionsJson_ = s;
+          defaultSchemaName_ = s;
           return s;
         } else {
           return (java.lang.String) ref;
         }
       }
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
       public com.google.protobuf.ByteString
-          getOptionsJsonBytes() {
-        java.lang.Object ref = optionsJson_;
+          getDefaultSchemaNameBytes() {
+        java.lang.Object ref = defaultSchemaName_;
         if (ref instanceof String) {
           com.google.protobuf.ByteString b = 
               com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
-          optionsJson_ = b;
+          defaultSchemaName_ = b;
           return b;
         } else {
           return (com.google.protobuf.ByteString) ref;
         }
       }
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
-      public Builder setOptionsJson(
+      public Builder setDefaultSchemaName(
           java.lang.String value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00004000;
-        optionsJson_ = value;
+  bitField0_ |= 0x00000004;
+        defaultSchemaName_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
-      public Builder clearOptionsJson() {
-        bitField0_ = (bitField0_ & ~0x00004000);
-        optionsJson_ = getDefaultInstance().getOptionsJson();
+      public Builder clearDefaultSchemaName() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        defaultSchemaName_ = getDefaultInstance().getDefaultSchemaName();
         onChanged();
         return this;
       }
       /**
-       * <code>optional string options_json = 17;</code>
+       * <code>optional string default_schema_name = 3;</code>
+       *
+       * <pre>
+       * default schema in current session when the query is submitted
+       * </pre>
        */
-      public Builder setOptionsJsonBytes(
+      public Builder setDefaultSchemaNameBytes(
           com.google.protobuf.ByteString value) {
         if (value == null) {
     throw new NullPointerException();
   }
-  bitField0_ |= 0x00004000;
-        optionsJson_ = value;
+  bitField0_ |= 0x00000004;
+        defaultSchemaName_ = value;
         onChanged();
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:exec.bit.control.PlanFragment)
+      // @@protoc_insertion_point(builder_scope:exec.bit.control.QueryContextInformation)
     }
 
     static {
-      defaultInstance = new PlanFragment(true);
+      defaultInstance = new QueryContextInformation(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:exec.bit.control.PlanFragment)
+    // @@protoc_insertion_point(class_scope:exec.bit.control.QueryContextInformation)
   }
 
   public interface WorkQueueStatusOrBuilder
@@ -6598,6 +7341,11 @@ public final class BitControl {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_exec_bit_control_PlanFragment_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_exec_bit_control_QueryContextInformation_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_exec_bit_control_QueryContextInformation_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_exec_bit_control_WorkQueueStatus_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -6628,32 +7376,35 @@ public final class BitControl {
       "red.MinorFragmentProfile\022(\n\006handle\030\002 \001(\013",
       "2\030.exec.bit.FragmentHandle\"G\n\023Initialize" +
       "Fragments\0220\n\010fragment\030\001 \003(\0132\036.exec.bit.c" +
-      "ontrol.PlanFragment\"\275\003\n\014PlanFragment\022(\n\006" +
+      "ontrol.PlanFragment\"\314\003\n\014PlanFragment\022(\n\006" +
       "handle\030\001 \001(\0132\030.exec.bit.FragmentHandle\022\024" +
       "\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030\005 \001(\002\022\021" +
       "\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030\007 \001(\002\022\025" +
-      "\n\rfragment_json\030\010 \001(\t\022*\n\nassignment\030\n \001(" +
-      "\0132\026.exec.DrillbitEndpoint\022\025\n\rleaf_fragme" +
-      "nt\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.exec.Drillbi" +
+      "\n\rfragment_json\030\010 \001(\t\022\025\n\rleaf_fragment\030\t" +
+      " \001(\010\022*\n\nassignment\030\n \001(\0132\026.exec.Drillbit" +
+      "Endpoint\022\'\n\007foreman\030\013 \001(\0132\026.exec.Drillbi" +
       "tEndpoint\022\035\n\013mem_initial\030\014 \001(\003:\01020000000",
-      "\022\033\n\007mem_max\030\r \001(\003:\n2000000000\022\030\n\020query_s" +
-      "tart_time\030\016 \001(\003\0221\n\013credentials\030\017 \001(\0132\034.e" +
-      "xec.shared.UserCredentials\022\021\n\ttime_zone\030" +
-      "\020 \001(\005\022\024\n\014options_json\030\021 \001(\t\"f\n\017WorkQueue" +
-      "Status\022(\n\010endpoint\030\001 \001(\0132\026.exec.Drillbit" +
-      "Endpoint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report" +
-      "_time\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010recei" +
-      "ver\030\001 \001(\0132\030.exec.bit.FragmentHandle\022(\n\006s" +
-      "ender\030\002 \001(\0132\030.exec.bit.FragmentHandle*\271\002" +
-      "\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOO",
-      "DBYE\020\002\022\034\n\030REQ_INIATILIZE_FRAGMENTS\020\003\022\027\n\023" +
-      "REQ_CANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIVER_FI" +
-      "NISHED\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ" +
-      "_BIT_STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020R" +
-      "EQ_QUERY_CANCEL\020\017\022\030\n\024RESP_FRAGMENT_HANDL" +
-      "E\020\013\022\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BI" +
-      "T_STATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016B+\n\033org" +
-      ".apache.drill.exec.protoB\nBitControlH\001"
+      "\022\033\n\007mem_max\030\r \001(\003:\n2000000000\0221\n\013credent" +
+      "ials\030\016 \001(\0132\034.exec.shared.UserCredentials" +
+      "\022\024\n\014options_json\030\017 \001(\t\022:\n\007context\030\020 \001(\0132" +
+      ").exec.bit.control.QueryContextInformati" +
+      "on\"c\n\027QueryContextInformation\022\030\n\020query_s" +
+      "tart_time\030\001 \001(\003\022\021\n\ttime_zone\030\002 \001(\005\022\033\n\023de" +
+      "fault_schema_name\030\003 \001(\t\"f\n\017WorkQueueStat" +
+      "us\022(\n\010endpoint\030\001 \001(\0132\026.exec.DrillbitEndp" +
+      "oint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report_tim" +
+      "e\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010receiver\030",
+      "\001 \001(\0132\030.exec.bit.FragmentHandle\022(\n\006sende" +
+      "r\030\002 \001(\0132\030.exec.bit.FragmentHandle*\271\002\n\007Rp" +
+      "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
+      "\020\002\022\034\n\030REQ_INIATILIZE_FRAGMENTS\020\003\022\027\n\023REQ_" +
+      "CANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISH" +
+      "ED\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT" +
+      "_STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_Q" +
+      "UERY_CANCEL\020\017\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022" +
+      "\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_ST" +
+      "ATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016B+\n\033org.apa",
+      "che.drill.exec.protoB\nBitControlH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -6689,15 +7440,21 @@ public final class BitControl {
           internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_PlanFragment_descriptor,
-              new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "Assignment", "LeafFragment", "Foreman", "MemInitial", "MemMax", "QueryStartTime", "Credentials", "TimeZone", "OptionsJson", });
-          internal_static_exec_bit_control_WorkQueueStatus_descriptor =
+              new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "LeafFragment", "Assignment", "Foreman", "MemInitial", "MemMax", "Credentials", "OptionsJson", "Context", });
+          internal_static_exec_bit_control_QueryContextInformation_descriptor =
             getDescriptor().getMessageTypes().get(5);
+          internal_static_exec_bit_control_QueryContextInformation_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_exec_bit_control_QueryContextInformation_descriptor,
+              new java.lang.String[] { "QueryStartTime", "TimeZone", "DefaultSchemaName", });
+          internal_static_exec_bit_control_WorkQueueStatus_descriptor =
+            getDescriptor().getMessageTypes().get(6);
           internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_WorkQueueStatus_descriptor,
               new java.lang.String[] { "Endpoint", "QueueLength", "ReportTime", });
           internal_static_exec_bit_control_FinishedReceiver_descriptor =
-            getDescriptor().getMessageTypes().get(6);
+            getDescriptor().getMessageTypes().get(7);
           internal_static_exec_bit_control_FinishedReceiver_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_FinishedReceiver_descriptor,


[4/5] drill git commit: DRILL-2902: Add support for context functions: user (synonyms session_user and system_user) and current_schema

Posted by ve...@apache.org.
DRILL-2902: Add support for context functions: user (synonyms session_user and system_user) and current_schema


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

Branch: refs/heads/master
Commit: 703314baf3cf1ddd4756b838449421271703ca37
Parents: ac823fe
Author: vkorukanti <ve...@gmail.com>
Authored: Tue Apr 28 17:57:00 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Tue May 5 11:03:58 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/client/DrillClient.java   |    4 +-
 .../exec/expr/fn/impl/ContextFunctions.java     |   80 +
 .../exec/expr/fn/impl/DateTypeFunctions.java    |   34 +-
 .../drill/exec/ops/ContextInformation.java      |   66 +
 .../apache/drill/exec/ops/FragmentContext.java  |    9 +-
 .../org/apache/drill/exec/ops/QueryContext.java |   18 +-
 .../drill/exec/ops/QueryDateTimeInfo.java       |   36 -
 .../org/apache/drill/exec/ops/UdfUtilities.java |   16 +-
 .../planner/fragment/SimpleParallelizer.java    |   16 +-
 .../apache/drill/exec/rpc/user/UserSession.java |    4 +
 .../org/apache/drill/exec/util/Utilities.java   |   19 +
 .../apache/drill/exec/work/foreman/Foreman.java |    2 +-
 .../exec/fn/impl/TestContextFunctions.java      |   93 +
 .../fn/interp/ExpressionInterpreterTest.java    |    8 +-
 .../exec/physical/impl/TestLocalExchange.java   |   13 +-
 .../partitionsender/TestPartitionSender.java    |   11 +-
 .../drill/exec/pop/TestFragmentChecker.java     |   12 +-
 .../security/TestCustomUserAuthenticator.java   |    2 +-
 .../exec/testing/TestExceptionInjection.java    |    3 +
 .../drill/exec/testing/TestPauseInjection.java  |    2 +
 .../org/apache/drill/exec/proto/BitControl.java | 1655 +++++++++++++-----
 .../drill/exec/proto/SchemaBitControl.java      |  162 +-
 .../drill/exec/proto/beans/PlanFragment.java    |   76 +-
 .../proto/beans/QueryContextInformation.java    |  207 +++
 protocol/src/main/protobuf/BitControl.proto     |   15 +-
 25 files changed, 1930 insertions(+), 633 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/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 4576eb4..3fda9c1 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
@@ -20,6 +20,8 @@ package org.apache.drill.exec.client;
 import static com.google.common.base.Preconditions.checkState;
 import static org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
 import static org.apache.drill.exec.proto.UserProtos.RunQuery.newBuilder;
+
+import com.google.common.base.Strings;
 import io.netty.buffer.DrillBuf;
 
 import java.io.Closeable;
@@ -301,7 +303,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
 
     if (props != null) {
       for (Property property: props.getPropertiesList()) {
-        if (property.getKey().equalsIgnoreCase("user")) {
+        if (property.getKey().equalsIgnoreCase("user") && !Strings.isNullOrEmpty(property.getValue())) {
           userName = property.getValue();
           break;
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
new file mode 100644
index 0000000..9d4696d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ContextFunctions.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.ops.ContextInformation;
+
+import javax.inject.Inject;
+
+@SuppressWarnings("unused")
+public class ContextFunctions {
+
+  /**
+   * Implement "user", "session_user" or "system_user" function. Returns the username of the user connected to Drillbit.
+   */
+  @FunctionTemplate(names = {"user", "session_user", "system_user"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class User implements DrillSimpleFunc {
+    @Output VarCharHolder out;
+    @Inject ContextInformation contextInfo;
+    @Inject DrillBuf buffer;
+    @Workspace int queryUserBytesLength;
+
+    public void setup() {
+      final byte[] queryUserNameBytes = contextInfo.getQueryUser().getBytes();
+      buffer = buffer.reallocIfNeeded(queryUserNameBytes.length);
+      queryUserBytesLength = queryUserNameBytes.length;
+      buffer.setBytes(0, queryUserNameBytes);
+    }
+
+    public void eval() {
+      out.start = 0;
+      out.end = queryUserBytesLength;
+      out.buffer = buffer;
+    }
+  }
+
+  /**
+   * Implement "current_schema" function. Returns the default schema in current session.
+   */
+  @FunctionTemplate(name = "current_schema", scope = FunctionTemplate.FunctionScope.SIMPLE)
+  public static class CurrentSchema implements DrillSimpleFunc {
+    @Output VarCharHolder out;
+    @Inject ContextInformation contextInfo;
+    @Inject DrillBuf buffer;
+    @Workspace int currentSchemaBytesLength;
+
+    public void setup() {
+      final byte[] currentSchemaBytes = contextInfo.getCurrentDefaultSchema().getBytes();
+      buffer = buffer.reallocIfNeeded(currentSchemaBytes.length);
+      currentSchemaBytesLength= currentSchemaBytes.length;
+      buffer.setBytes(0, currentSchemaBytes);
+    }
+
+    public void eval() {
+      out.start = 0;
+      out.end = currentSchemaBytesLength;
+      out.buffer = buffer;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
index 9c932d6..41ce328 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.expr.holders.IntervalYearHolder;
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
+import org.apache.drill.exec.ops.ContextInformation;
 
 public class DateTypeFunctions {
 
@@ -181,13 +181,13 @@ public class DateTypeFunctions {
     public static class CurrentDate implements DrillSimpleFunc {
         @Workspace long queryStartDate;
         @Output DateHolder out;
-        @Inject QueryDateTimeInfo dateTime;
+        @Inject ContextInformation contextInfo;
 
         public void setup() {
 
-            int timeZoneIndex = dateTime.getRootFragmentTimeZone();
+            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(dateTime.getQueryStartTime(), timeZone);
+            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
             queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).
                 withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
         }
@@ -220,11 +220,11 @@ public class DateTypeFunctions {
     public static class LocalTimeStamp implements DrillSimpleFunc {
         @Workspace long queryStartDate;
         @Output TimeStampHolder out;
-        @Inject QueryDateTimeInfo dateTime;
+        @Inject ContextInformation contextInfo;
 
         public void setup() {
 
-            org.joda.time.DateTime now = (new org.joda.time.DateTime(dateTime.getQueryStartTime())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC);
+            org.joda.time.DateTime now = (new org.joda.time.DateTime(contextInfo.getQueryStartTime())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC);
             queryStartDate = now.getMillis();
         }
 
@@ -237,13 +237,13 @@ public class DateTypeFunctions {
     public static class CurrentTime implements DrillSimpleFunc {
         @Workspace int queryStartTime;
         @Output TimeHolder out;
-        @Inject QueryDateTimeInfo dateTime;
+        @Inject ContextInformation contextInfo;
 
         public void setup() {
 
-            int timeZoneIndex = dateTime.getRootFragmentTimeZone();
+            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(dateTime.getQueryStartTime(), timeZone);
+            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
             queryStartTime= (int) ((now.getHourOfDay() * org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis) +
                                    (now.getMinuteOfHour() * org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis) +
                                    (now.getSecondOfMinute() * org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis) +
@@ -331,12 +331,12 @@ public class DateTypeFunctions {
         @Param TimeStampHolder right;
         @Workspace long queryStartDate;
         @Output IntervalHolder out;
-        @Inject QueryDateTimeInfo dateTime;
+        @Inject ContextInformation contextInfo;
 
         public void setup() {
-            int timeZoneIndex = dateTime.getRootFragmentTimeZone();
+            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(dateTime.getQueryStartTime(), timeZone);
+            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
             queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
         }
 
@@ -374,12 +374,12 @@ public class DateTypeFunctions {
         @Param DateHolder right;
         @Workspace long queryStartDate;
         @Output IntervalHolder out;
-        @Inject QueryDateTimeInfo dateTime;
+        @Inject ContextInformation contextInfo;
 
         public void setup() {
-            int timeZoneIndex = dateTime.getRootFragmentTimeZone();
+            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
             org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(dateTime.getQueryStartTime(), timeZone);
+            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
             queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
         }
 
@@ -411,11 +411,11 @@ public class DateTypeFunctions {
     public static class UnixTimeStamp implements DrillSimpleFunc {
       @Output BigIntHolder out;
       @Workspace long queryStartDate;
-      @Inject QueryDateTimeInfo dateTime;
+      @Inject ContextInformation contextInfo;
 
       @Override
       public void setup() {
-         queryStartDate = dateTime.getQueryStartTime();
+         queryStartDate = contextInfo.getQueryStartTime();
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
new file mode 100644
index 0000000..7054f36
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ ******************************************************************************/
+package org.apache.drill.exec.ops;
+
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+
+/**
+ * Provides query context information (such as query start time, query user, default schema etc.) for UDFs.
+ */
+public class ContextInformation {
+  private final String queryUser;
+  private final String currentDefaultSchema;
+  private final long queryStartTime;
+  private final int rootFragmentTimeZone;
+
+  public ContextInformation(final UserCredentials userCredentials, final QueryContextInformation queryContextInfo) {
+    this.queryUser = userCredentials.getUserName();
+    this.currentDefaultSchema = queryContextInfo.getDefaultSchemaName();
+    this.queryStartTime = queryContextInfo.getQueryStartTime();
+    this.rootFragmentTimeZone = queryContextInfo.getTimeZone();
+  }
+
+  /**
+   * @return userName of the user who issued the current query.
+   */
+  public String getQueryUser() {
+    return queryUser;
+  }
+
+  /**
+   * @return Get the current default schema in user session at the time of this particular query submission.
+   */
+  public String getCurrentDefaultSchema() {
+    return currentDefaultSchema;
+  }
+
+  /**
+   * @return Query start time in milliseconds
+   */
+  public long getQueryStartTime() {
+    return queryStartTime;
+  }
+
+  /**
+   * @return Time zone.
+   */
+  public int getRootFragmentTimeZone() {
+    return rootFragmentTimeZone;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/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 09a7568..b108924 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
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.jdbc.SimpleCalciteSchema;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -76,7 +75,7 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
   private final FunctionImplementationRegistry funcRegistry;
   private final BufferAllocator allocator;
   private final PlanFragment fragment;
-  private final QueryDateTimeInfo queryDateTimeInfo;
+  private final ContextInformation contextInformation;
   private IncomingBuffers buffers;
   private final OptionManager fragmentOptions;
   private final BufferManager bufferManager;
@@ -126,7 +125,7 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     this.accountingUserConnection = new AccountingUserConnection(connection, sendingAccountor, statusHandler);
     this.fragment = fragment;
     this.funcRegistry = funcRegistry;
-    queryDateTimeInfo = new QueryDateTimeInfo(fragment.getQueryStartTime(), fragment.getTimeZone());
+    contextInformation = new ContextInformation(fragment.getCredentials(), fragment.getContext());
 
     logger.debug("Getting initial memory allocation of {}", fragment.getMemInitial());
     logger.debug("Fragment max allocation: {}", fragment.getMemMax());
@@ -234,8 +233,8 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
   }
 
   @Override
-  public QueryDateTimeInfo getQueryDateTimeInfo(){
-    return this.queryDateTimeInfo;
+  public ContextInformation getContextInformation() {
+    return contextInformation;
   }
 
   public DrillbitEndpoint getForemanEndpoint() {

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 6414f56..9e2f210 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -44,6 +45,7 @@ import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.exec.util.Utilities;
 
 // TODO except for a couple of tests, this is only created by Foreman
 // TODO the many methods that just return drillbitContext.getXxx() should be replaced with getDrillbitContext()
@@ -64,7 +66,8 @@ public class QueryContext implements AutoCloseable, UdfUtilities {
 
   private final BufferAllocator allocator;
   private final BufferManager bufferManager;
-  private final QueryDateTimeInfo queryDateTimeInfo;
+  private final ContextInformation contextInformation;
+  private final QueryContextInformation queryContextInfo;
   private final ViewExpansionContext viewExpansionContext;
 
   /*
@@ -82,9 +85,8 @@ public class QueryContext implements AutoCloseable, UdfUtilities {
     plannerSettings.setNumEndPoints(drillbitContext.getBits().size());
     table = new DrillOperatorTable(getFunctionRegistry());
 
-    final long queryStartTime = System.currentTimeMillis();
-    final int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
-    queryDateTimeInfo = new QueryDateTimeInfo(queryStartTime, timeZone);
+    queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName());
+    contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
 
     try {
       allocator = drillbitContext.getAllocator().getChildAllocator(null, INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES,
@@ -212,9 +214,13 @@ public class QueryContext implements AutoCloseable, UdfUtilities {
     return table;
   }
 
+  public QueryContextInformation getQueryContextInfo() {
+    return queryContextInfo;
+  }
+
   @Override
-  public QueryDateTimeInfo getQueryDateTimeInfo() {
-    return queryDateTimeInfo;
+  public ContextInformation getContextInformation() {
+    return contextInformation;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryDateTimeInfo.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryDateTimeInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryDateTimeInfo.java
deleted file mode 100644
index f3cc666..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryDateTimeInfo.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- ******************************************************************************/
-package org.apache.drill.exec.ops;
-
-public class QueryDateTimeInfo {
-  private final long queryStartTime;
-  private final int rootFragmentTimeZone;
-
-  public QueryDateTimeInfo(long queryStartTime, int rootFragmentTimeZone) {
-    this.queryStartTime = queryStartTime;
-    this.rootFragmentTimeZone = rootFragmentTimeZone;
-  }
-
-  public long getQueryStartTime() {
-    return this.queryStartTime;
-  }
-
-  public int getRootFragmentTimeZone() {
-    return this.rootFragmentTimeZone;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
index 1cdece1..9c91331 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
@@ -34,18 +34,22 @@ public interface UdfUtilities {
   public static final ImmutableMap<Class, String> INJECTABLE_GETTER_METHODS =
       new ImmutableMap.Builder<Class, String>()
           .put(DrillBuf.class, "getManagedBuffer")
-          .put(QueryDateTimeInfo.class, "getQueryDateTimeInfo")
           .put(PartitionExplorer.class, "getPartitionExplorer")
+          .put(ContextInformation.class, "getContextInformation")
           .build();
 
+
   /**
-   * Get the query start time and timezone recorded by the head node during
-   * planning. This allows for SQL functions like now() to return a stable
-   * result within the context of a distributed query.
+   * Get the context information such as:
+   *  - query start time
+   *  - root fragment timezone
+   *  - query userName
+   *  - system userName
+   *  - default schema name in current session at the time of query.
    *
-   * @return - object wrapping the raw time and timezone values
+   * @return - ContextInformation
    */
-  QueryDateTimeInfo getQueryDateTimeInfo();
+  ContextInformation getContextInformation();
 
   /**
    * For UDFs to allocate general purpose intermediate buffers we provide the

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
index 66ba229..d36ad42 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -34,7 +34,6 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.util.DrillStringUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
 import org.apache.drill.exec.physical.base.Exchange.ParallelizationDependency;
@@ -44,6 +43,7 @@ import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
 import org.apache.drill.exec.planner.fragment.Materializer.IndexedFragmentNode;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -107,12 +107,13 @@ public class SimpleParallelizer {
    * @param reader          Tool used to read JSON plans
    * @param rootFragment    The root node of the PhysicalPlan that we will be parallelizing.
    * @param session         UserSession of user who launched this query.
+   * @param queryContextInfo Info related to the context when query has started.
    * @return The list of generated PlanFragment protobuf objects to be assigned out to the individual nodes.
    * @throws ExecutionSetupException
    */
   public QueryWorkUnit getFragments(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId,
       Collection<DrillbitEndpoint> activeEndpoints, PhysicalPlanReader reader, Fragment rootFragment,
-      UserSession session, QueryDateTimeInfo queryDateTimeInfo) throws ExecutionSetupException {
+      UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException {
 
     final PlanningSet planningSet = new PlanningSet();
 
@@ -125,7 +126,8 @@ public class SimpleParallelizer {
       parallelizeFragment(wrapper, planningSet, activeEndpoints);
     }
 
-    return generateWorkUnit(options, foremanNode, queryId, reader, rootFragment, planningSet, session, queryDateTimeInfo);
+    return generateWorkUnit(
+        options, foremanNode, queryId, reader, rootFragment, planningSet, session, queryContextInfo);
   }
 
   // For every fragment, create a Wrapper in PlanningSet.
@@ -320,15 +322,12 @@ public class SimpleParallelizer {
 
   private QueryWorkUnit generateWorkUnit(OptionList options, DrillbitEndpoint foremanNode, QueryId queryId,
       PhysicalPlanReader reader, Fragment rootNode, PlanningSet planningSet,
-      UserSession session, QueryDateTimeInfo queryDateTimeInfo) throws ExecutionSetupException {
+      UserSession session, QueryContextInformation queryContextInfo) throws ExecutionSetupException {
     List<PlanFragment> fragments = Lists.newArrayList();
 
     PlanFragment rootFragment = null;
     FragmentRoot rootOperator = null;
 
-    long queryStartTime = queryDateTimeInfo.getQueryStartTime();
-    int timeZone = queryDateTimeInfo.getRootFragmentTimeZone();
-
     // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
     // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
     for (Wrapper wrapper : planningSet) {
@@ -374,8 +373,7 @@ public class SimpleParallelizer {
             .setHandle(handle) //
             .setAssignment(wrapper.getAssignedEndpoint(minorFragmentId)) //
             .setLeafFragment(isLeafFragment) //
-            .setQueryStartTime(queryStartTime)
-            .setTimeZone(timeZone)//
+            .setContext(queryContextInfo)
             .setMemInitial(wrapper.getInitialAllocation())//
             .setMemMax(wrapper.getMaxAllocation())
             .setOptionsJson(optionsData)

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
index 527bac0..9f1a695 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
@@ -118,6 +118,10 @@ public class UserSession {
     return credentials;
   }
 
+  public String getDefaultSchemaName() {
+    return getProp(SCHEMA);
+  }
+
   public void incrementQueryCount(final QueryCountIncrementer incrementer) {
     assert incrementer != null;
     queryCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
index 8efb9e7..b0496bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Utilities.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.util;
 
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 
@@ -38,4 +40,21 @@ public class Utilities {
 
     return fileName;
   }
+
+  /**
+   * Create QueryContextInformation with given <i>defaultSchemaName</i>. Rest of the members of the
+   * QueryContextInformation is derived from the current state of the process.
+   *
+   * @param defaultSchemaName
+   * @return
+   */
+  public static QueryContextInformation createQueryContextInfo(final String defaultSchemaName) {
+    final long queryStartTime = System.currentTimeMillis();
+    final int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
+    return QueryContextInformation.newBuilder()
+        .setDefaultSchemaName(defaultSchemaName)
+        .setQueryStartTime(queryStartTime)
+        .setTimeZone(timeZone)
+        .build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/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 4249cbe..d678cc5 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
@@ -443,7 +443,7 @@ public class Foreman implements Runnable {
     final QueryWorkUnit queryWorkUnit = parallelizer.getFragments(
         queryContext.getOptions().getOptionList(), queryContext.getCurrentEndpoint(),
         queryId, queryContext.getActiveEndpoints(), drillbitContext.getPlanReader(), rootFragment,
-        initiatingClient.getSession(), queryContext.getQueryDateTimeInfo());
+        initiatingClient.getSession(), queryContext.getQueryContextInfo());
 
     if (logger.isTraceEnabled()) {
       final StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestContextFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestContextFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestContextFunctions.java
new file mode 100644
index 0000000..a3c473f
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestContextFunctions.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.fn.impl;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestContextFunctions extends BaseTestQuery {
+
+  @Test
+  public void userUDFForAnonymousConnection() throws Exception {
+    updateClient("");
+    testBuilder()
+        .sqlQuery("select user, session_user, system_user from cp.`employee.json` limit 1")
+        .unOrdered()
+        .baselineColumns("user", "session_user", "system_user")
+        .baselineValues("anonymous", "anonymous", "anonymous")
+        .go();
+  }
+
+  @Test
+  public void userUDFForNamedConnection() throws Exception {
+    final String testUserName = "testUser1";
+    updateClient(testUserName);
+    testBuilder()
+        .sqlQuery("select user, session_user, system_user from cp.`employee.json` limit 1")
+        .unOrdered()
+        .baselineColumns("user", "session_user", "system_user")
+        .baselineValues(testUserName, testUserName, testUserName)
+        .go();
+  }
+
+  @Test
+  public void userUDFInFilterCondition() throws Exception {
+    final String testUserName = "testUser2";
+    updateClient(testUserName);
+    final String query = String.format(
+        "select employee_id from cp.`employee.json` where '%s' = user order by employee_id limit 1", testUserName);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("employee_id")
+        .baselineValues(1L)
+        .go();
+  }
+
+  @Test
+  public void currentSchemaUDFWhenDefaultSchemaNotSet() throws Exception {
+    testBuilder()
+        .sqlQuery("select current_schema from cp.`employee.json` limit 1")
+        .unOrdered()
+        .baselineColumns("current_schema")
+        .baselineValues("")
+        .go();
+  }
+
+  @Test
+  public void currentSchemaUDFWithSingleLevelDefaultSchema() throws Exception {
+    testBuilder()
+        .optionSettingQueriesForTestQuery("USE dfs_test")
+        .sqlQuery("select current_schema from cp.`employee.json` limit 1")
+        .unOrdered()
+        .baselineColumns("current_schema")
+        .baselineValues("dfs_test")
+        .go();
+  }
+
+  @Test
+  public void currentSchemaUDFWithMultiLevelDefaultSchema() throws Exception {
+    testBuilder()
+        .optionSettingQueriesForTestQuery("USE dfs_test.tmp")
+        .sqlQuery("select current_schema from cp.`employee.json` limit 1")
+        .unOrdered()
+        .baselineColumns("current_schema")
+        .baselineValues("dfs_test.tmp")
+        .go();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
index 04e1980..2a83a53 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java
@@ -39,10 +39,10 @@ import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.fn.interpreter.InterpreterEvaluator;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.BitControl;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.Drillbit;
@@ -127,10 +127,10 @@ public class ExpressionInterpreterTest  extends PopUnitTestBase {
     TypeProtos.MajorType[] colTypes = {Types.optional(TypeProtos.MinorType.INT)};
     String expressionStr = "now()";
     BitControl.PlanFragment planFragment = BitControl.PlanFragment.getDefaultInstance();
-    QueryDateTimeInfo dateTime = new QueryDateTimeInfo(planFragment.getQueryStartTime(), planFragment.getTimeZone());
-    int                        timeZoneIndex = dateTime.getRootFragmentTimeZone();
+    QueryContextInformation queryContextInfo = planFragment.getContext();
+    int                        timeZoneIndex = queryContextInfo.getTimeZone();
     org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-    org.joda.time.DateTime     now = new org.joda.time.DateTime(dateTime.getQueryStartTime(), timeZone);
+    org.joda.time.DateTime     now = new org.joda.time.DateTime(queryContextInfo.getQueryStartTime(), timeZone);
 
     long queryStartDate = now.getMillis();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
index 9758eb0..09f8fcb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestLocalExchange.java
@@ -24,8 +24,6 @@ import com.google.common.collect.Lists;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.TestBuilder;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
 import org.apache.drill.exec.physical.base.Exchange;
 import org.apache.drill.exec.physical.config.UnorderedDeMuxExchange;
 import org.apache.drill.exec.physical.config.HashToRandomExchange;
@@ -34,16 +32,16 @@ import org.apache.drill.exec.planner.fragment.Fragment;
 import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
 import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
-import org.apache.drill.exec.planner.physical.HashPrelUtil;
-import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionList;
+import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.work.QueryWorkUnit;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
@@ -409,13 +407,10 @@ public class TestLocalExchange extends PlanTestBase {
 
     findFragmentsWithPartitionSender(rootFragment, planningSet, deMuxFragments, htrFragments);
 
-    long queryStartTime = System.currentTimeMillis();
-    int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
-    QueryDateTimeInfo queryDateTimeInfo = new QueryDateTimeInfo(queryStartTime, timeZone);
-
+    final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName");
     QueryWorkUnit qwu = PARALLELIZER.getFragments(new OptionList(), drillbitContext.getEndpoint(),
         QueryId.getDefaultInstance(),
-        drillbitContext.getBits(), planReader, rootFragment, USER_SESSION, queryDateTimeInfo);
+        drillbitContext.getBits(), planReader, rootFragment, USER_SESSION, queryContextInfo);
 
     // Make sure the number of minor fragments with HashPartitioner within a major fragment is not more than the
     // number of Drillbits in cluster

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
index 6a6a7e0..320e722 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/partitionsender/TestPartitionSender.java
@@ -30,11 +30,9 @@ import java.util.Random;
 
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorStats;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
 import org.apache.drill.exec.physical.MinorFragmentEndpoint;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -50,6 +48,7 @@ import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.MetricValue;
 import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
@@ -64,6 +63,7 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.work.QueryWorkUnit;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -216,13 +216,10 @@ public class TestPartitionSender extends PlanTestBase {
       RecordBatch incoming, FunctionImplementationRegistry registry, PhysicalPlanReader planReader, PlanningSet planningSet, Fragment rootFragment,
       int expectedThreadsCount) throws Exception {
 
-    long queryStartTime = System.currentTimeMillis();
-    int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
-    QueryDateTimeInfo queryDateTimeInfo = new QueryDateTimeInfo(queryStartTime, timeZone);
-
+    final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName");
     final QueryWorkUnit qwu = PARALLELIZER.getFragments(options, drillbitContext.getEndpoint(),
         QueryId.getDefaultInstance(),
-        drillbitContext.getBits(), planReader, rootFragment, USER_SESSION, queryDateTimeInfo);
+        drillbitContext.getBits(), planReader, rootFragment, USER_SESSION, queryContextInfo);
 
     final List<MinorFragmentEndpoint> mfEndPoints = PhysicalOperatorUtil.getIndexOrderedEndpoints(Lists.newArrayList(drillbitContext.getBits()));
 

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
index 32e3bf9..0fbf0bd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/pop/TestFragmentChecker.java
@@ -19,18 +19,17 @@ package org.apache.drill.exec.pop;
 
 import java.util.List;
 
-import org.apache.drill.exec.expr.fn.impl.DateUtility;
-import org.apache.drill.exec.ops.QueryDateTimeInfo;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.fragment.Fragment;
-import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.options.OptionList;
+import org.apache.drill.exec.util.Utilities;
 import org.apache.drill.exec.work.QueryWorkUnit;
 import org.junit.Test;
 
@@ -62,13 +61,10 @@ public class TestFragmentChecker extends PopUnitTestBase{
       endpoints.add(b1);
     }
 
-    long queryStartTime = System.currentTimeMillis();
-    int timeZone = DateUtility.getIndex(System.getProperty("user.timezone"));
-    QueryDateTimeInfo queryDateTimeInfo = new QueryDateTimeInfo(queryStartTime, timeZone);
-
+    final QueryContextInformation queryContextInfo = Utilities.createQueryContextInfo("dummySchemaName");
     QueryWorkUnit qwu = par.getFragments(new OptionList(), localBit, QueryId.getDefaultInstance(), endpoints, ppr, fragmentRoot,
         UserSession.Builder.newBuilder().withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build()).build(),
-        queryDateTimeInfo);
+        queryContextInfo);
     System.out.println(String.format("=========ROOT FRAGMENT [%d:%d] =========", qwu.getRootFragment().getHandle().getMajorFragmentId(), qwu.getRootFragment().getHandle().getMinorFragmentId()));
 
     System.out.print(qwu.getRootFragment().getFragmentJson());

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
index 70d43b6..d9b4634 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
@@ -61,7 +61,7 @@ public class TestCustomUserAuthenticator extends BaseTestQuery {
     negativeAuthHelper(TEST_USER_1, "blah.. blah..");
     negativeAuthHelper(TEST_USER_2, "blah.. blah..");
     negativeAuthHelper(TEST_USER_2, "");
-    negativeAuthHelper("", "blah.. blah..");
+    negativeAuthHelper("invalidUserName", "blah.. blah..");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
index 604f375..e3558a1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -40,6 +41,7 @@ public class TestExceptionInjection extends BaseTestQuery {
 
   private static final UserSession session = UserSession.Builder.newBuilder()
       .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
+      .withUserProperties(UserProperties.getDefaultInstance())
       .withOptionManager(bits[0].getContext().getOptionManager())
       .build();
 
@@ -251,6 +253,7 @@ public class TestExceptionInjection extends BaseTestQuery {
 
     final UserSession session = UserSession.Builder.newBuilder()
         .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
+        .withUserProperties(UserProperties.getDefaultInstance())
         .withOptionManager(drillbitContext1.getOptionManager())
         .build();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/703314ba/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
index 508b10c..5fa2b3f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.testing;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -31,6 +32,7 @@ public class TestPauseInjection extends BaseTestQuery {
 
   private static final UserSession session = UserSession.Builder.newBuilder()
       .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
+      .withUserProperties(UserProperties.getDefaultInstance())
       .withOptionManager(bits[0].getContext().getOptionManager())
       .build();