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

[1/9] drill git commit: DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData

Repository: drill
Updated Branches:
  refs/heads/master 10be89fe0 -> 4f213570f


http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index 048bd20..0b81926 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -56,6 +56,10 @@ public final class UserProtos {
     RUN_QUERY(3, 3),
     /**
      * <code>CANCEL_QUERY = 4;</code>
+     *
+     * <pre>
+     * user is sending a query cancellation request to the drillbit
+     * </pre>
      */
     CANCEL_QUERY(4, 4),
     /**
@@ -63,13 +67,13 @@ public final class UserProtos {
      */
     REQUEST_RESULTS(5, 5),
     /**
-     * <code>QUERY_RESULT = 6;</code>
+     * <code>QUERY_DATA = 6;</code>
      *
      * <pre>
      * bit to user
      * </pre>
      */
-    QUERY_RESULT(6, 6),
+    QUERY_DATA(6, 6),
     /**
      * <code>QUERY_HANDLE = 7;</code>
      */
@@ -82,6 +86,14 @@ public final class UserProtos {
      * <code>RESP_FUNCTION_LIST = 9;</code>
      */
     RESP_FUNCTION_LIST(9, 9),
+    /**
+     * <code>QUERY_RESULT = 10;</code>
+     *
+     * <pre>
+     * drillbit is reporting a query status change, most likely a terminal message, to the user
+     * </pre>
+     */
+    QUERY_RESULT(10, 10),
     ;
 
     /**
@@ -106,6 +118,10 @@ public final class UserProtos {
     public static final int RUN_QUERY_VALUE = 3;
     /**
      * <code>CANCEL_QUERY = 4;</code>
+     *
+     * <pre>
+     * user is sending a query cancellation request to the drillbit
+     * </pre>
      */
     public static final int CANCEL_QUERY_VALUE = 4;
     /**
@@ -113,13 +129,13 @@ public final class UserProtos {
      */
     public static final int REQUEST_RESULTS_VALUE = 5;
     /**
-     * <code>QUERY_RESULT = 6;</code>
+     * <code>QUERY_DATA = 6;</code>
      *
      * <pre>
      * bit to user
      * </pre>
      */
-    public static final int QUERY_RESULT_VALUE = 6;
+    public static final int QUERY_DATA_VALUE = 6;
     /**
      * <code>QUERY_HANDLE = 7;</code>
      */
@@ -132,6 +148,14 @@ public final class UserProtos {
      * <code>RESP_FUNCTION_LIST = 9;</code>
      */
     public static final int RESP_FUNCTION_LIST_VALUE = 9;
+    /**
+     * <code>QUERY_RESULT = 10;</code>
+     *
+     * <pre>
+     * drillbit is reporting a query status change, most likely a terminal message, to the user
+     * </pre>
+     */
+    public static final int QUERY_RESULT_VALUE = 10;
 
 
     public final int getNumber() { return value; }
@@ -144,10 +168,11 @@ public final class UserProtos {
         case 3: return RUN_QUERY;
         case 4: return CANCEL_QUERY;
         case 5: return REQUEST_RESULTS;
-        case 6: return QUERY_RESULT;
+        case 6: return QUERY_DATA;
         case 7: return QUERY_HANDLE;
         case 8: return REQ_META_FUNCTIONS;
         case 9: return RESP_FUNCTION_LIST;
+        case 10: return QUERY_RESULT;
         default: return null;
       }
     }
@@ -4343,14 +4368,14 @@ public final class UserProtos {
       "lts_mode\030\001 \001(\0162\033.exec.user.QueryResultsM" +
       "ode\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryTyp" +
       "e\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022\023\n" +
-      "\013rpc_version\030\002 \001(\005*\270\001\n\007RpcType\022\r\n\tHANDSH" +
+      "\013rpc_version\030\002 \001(\005*\310\001\n\007RpcType\022\r\n\tHANDSH" +
       "AKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY" +
       "\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020" +
-      "\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n",
-      "\022REQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_L" +
-      "IST\020\t*#\n\020QueryResultsMode\022\017\n\013STREAM_FULL" +
-      "\020\001B+\n\033org.apache.drill.exec.protoB\nUserP" +
-      "rotosH\001"
+      "\005\022\016\n\nQUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n\022R",
+      "EQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_LIS" +
+      "T\020\t\022\020\n\014QUERY_RESULT\020\n*#\n\020QueryResultsMod" +
+      "e\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.e" +
+      "xec.protoB\nUserProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
new file mode 100644
index 0000000..70a6426
--- /dev/null
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
@@ -0,0 +1,211 @@
+/**
+ * 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 QueryData implements Externalizable, Message<QueryData>, Schema<QueryData>
+{
+
+    public static Schema<QueryData> getSchema()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    public static QueryData getDefaultInstance()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    static final QueryData DEFAULT_INSTANCE = new QueryData();
+
+    
+    private QueryId queryId;
+    private int rowCount;
+    private RecordBatchDef def;
+
+    public QueryData()
+    {
+        
+    }
+
+    // getters and setters
+
+    // queryId
+
+    public QueryId getQueryId()
+    {
+        return queryId;
+    }
+
+    public QueryData setQueryId(QueryId queryId)
+    {
+        this.queryId = queryId;
+        return this;
+    }
+
+    // rowCount
+
+    public int getRowCount()
+    {
+        return rowCount;
+    }
+
+    public QueryData setRowCount(int rowCount)
+    {
+        this.rowCount = rowCount;
+        return this;
+    }
+
+    // def
+
+    public RecordBatchDef getDef()
+    {
+        return def;
+    }
+
+    public QueryData setDef(RecordBatchDef def)
+    {
+        this.def = def;
+        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<QueryData> cachedSchema()
+    {
+        return DEFAULT_INSTANCE;
+    }
+
+    // schema methods
+
+    public QueryData newMessage()
+    {
+        return new QueryData();
+    }
+
+    public Class<QueryData> typeClass()
+    {
+        return QueryData.class;
+    }
+
+    public String messageName()
+    {
+        return QueryData.class.getSimpleName();
+    }
+
+    public String messageFullName()
+    {
+        return QueryData.class.getName();
+    }
+
+    public boolean isInitialized(QueryData message)
+    {
+        return true;
+    }
+
+    public void mergeFrom(Input input, QueryData message) throws IOException
+    {
+        for(int number = input.readFieldNumber(this);; number = input.readFieldNumber(this))
+        {
+            switch(number)
+            {
+                case 0:
+                    return;
+                case 1:
+                    message.queryId = input.mergeObject(message.queryId, QueryId.getSchema());
+                    break;
+
+                case 2:
+                    message.rowCount = input.readInt32();
+                    break;
+                case 3:
+                    message.def = input.mergeObject(message.def, RecordBatchDef.getSchema());
+                    break;
+
+                default:
+                    input.handleUnknownField(number, this);
+            }   
+        }
+    }
+
+
+    public void writeTo(Output output, QueryData message) throws IOException
+    {
+        if(message.queryId != null)
+             output.writeObject(1, message.queryId, QueryId.getSchema(), false);
+
+
+        if(message.rowCount != 0)
+            output.writeInt32(2, message.rowCount, false);
+
+        if(message.def != null)
+             output.writeObject(3, message.def, RecordBatchDef.getSchema(), false);
+
+    }
+
+    public String getFieldName(int number)
+    {
+        switch(number)
+        {
+            case 1: return "queryId";
+            case 2: return "rowCount";
+            case 3: return "def";
+            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("queryId", 1);
+        __fieldMap.put("rowCount", 2);
+        __fieldMap.put("def", 3);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryResult.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryResult.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryResult.java
index d8eb92a..474e330 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryResult.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryResult.java
@@ -87,15 +87,7 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
     
     private QueryState queryState;
     private QueryId queryId;
-    private Boolean isLastChunk;
-    private int rowCount;
-    private long recordsScan;
-    private long recordsError;
-    private long submissionTime;
-    private List<NodeStatus> nodeStatus;
     private List<DrillPBError> error;
-    private RecordBatchDef def;
-    private Boolean schemaChanged;
 
     public QueryResult()
     {
@@ -130,84 +122,6 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
         return this;
     }
 
-    // isLastChunk
-
-    public Boolean getIsLastChunk()
-    {
-        return isLastChunk;
-    }
-
-    public QueryResult setIsLastChunk(Boolean isLastChunk)
-    {
-        this.isLastChunk = isLastChunk;
-        return this;
-    }
-
-    // rowCount
-
-    public int getRowCount()
-    {
-        return rowCount;
-    }
-
-    public QueryResult setRowCount(int rowCount)
-    {
-        this.rowCount = rowCount;
-        return this;
-    }
-
-    // recordsScan
-
-    public long getRecordsScan()
-    {
-        return recordsScan;
-    }
-
-    public QueryResult setRecordsScan(long recordsScan)
-    {
-        this.recordsScan = recordsScan;
-        return this;
-    }
-
-    // recordsError
-
-    public long getRecordsError()
-    {
-        return recordsError;
-    }
-
-    public QueryResult setRecordsError(long recordsError)
-    {
-        this.recordsError = recordsError;
-        return this;
-    }
-
-    // submissionTime
-
-    public long getSubmissionTime()
-    {
-        return submissionTime;
-    }
-
-    public QueryResult setSubmissionTime(long submissionTime)
-    {
-        this.submissionTime = submissionTime;
-        return this;
-    }
-
-    // nodeStatus
-
-    public List<NodeStatus> getNodeStatusList()
-    {
-        return nodeStatus;
-    }
-
-    public QueryResult setNodeStatusList(List<NodeStatus> nodeStatus)
-    {
-        this.nodeStatus = nodeStatus;
-        return this;
-    }
-
     // error
 
     public List<DrillPBError> getErrorList()
@@ -221,32 +135,6 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
         return this;
     }
 
-    // def
-
-    public RecordBatchDef getDef()
-    {
-        return def;
-    }
-
-    public QueryResult setDef(RecordBatchDef def)
-    {
-        this.def = def;
-        return this;
-    }
-
-    // schemaChanged
-
-    public Boolean getSchemaChanged()
-    {
-        return schemaChanged;
-    }
-
-    public QueryResult setSchemaChanged(Boolean schemaChanged)
-    {
-        this.schemaChanged = schemaChanged;
-        return this;
-    }
-
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -309,39 +197,11 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
                     break;
 
                 case 3:
-                    message.isLastChunk = input.readBool();
-                    break;
-                case 4:
-                    message.rowCount = input.readInt32();
-                    break;
-                case 5:
-                    message.recordsScan = input.readInt64();
-                    break;
-                case 6:
-                    message.recordsError = input.readInt64();
-                    break;
-                case 7:
-                    message.submissionTime = input.readInt64();
-                    break;
-                case 8:
-                    if(message.nodeStatus == null)
-                        message.nodeStatus = new ArrayList<NodeStatus>();
-                    message.nodeStatus.add(input.mergeObject(null, NodeStatus.getSchema()));
-                    break;
-
-                case 9:
                     if(message.error == null)
                         message.error = new ArrayList<DrillPBError>();
                     message.error.add(input.mergeObject(null, DrillPBError.getSchema()));
                     break;
 
-                case 10:
-                    message.def = input.mergeObject(message.def, RecordBatchDef.getSchema());
-                    break;
-
-                case 11:
-                    message.schemaChanged = input.readBool();
-                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -358,47 +218,15 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
              output.writeObject(2, message.queryId, QueryId.getSchema(), false);
 
 
-        if(message.isLastChunk != null)
-            output.writeBool(3, message.isLastChunk, false);
-
-        if(message.rowCount != 0)
-            output.writeInt32(4, message.rowCount, false);
-
-        if(message.recordsScan != 0)
-            output.writeInt64(5, message.recordsScan, false);
-
-        if(message.recordsError != 0)
-            output.writeInt64(6, message.recordsError, false);
-
-        if(message.submissionTime != 0)
-            output.writeInt64(7, message.submissionTime, false);
-
-        if(message.nodeStatus != null)
-        {
-            for(NodeStatus nodeStatus : message.nodeStatus)
-            {
-                if(nodeStatus != null)
-                    output.writeObject(8, nodeStatus, NodeStatus.getSchema(), true);
-            }
-        }
-
-
         if(message.error != null)
         {
             for(DrillPBError error : message.error)
             {
                 if(error != null)
-                    output.writeObject(9, error, DrillPBError.getSchema(), true);
+                    output.writeObject(3, error, DrillPBError.getSchema(), true);
             }
         }
 
-
-        if(message.def != null)
-             output.writeObject(10, message.def, RecordBatchDef.getSchema(), false);
-
-
-        if(message.schemaChanged != null)
-            output.writeBool(11, message.schemaChanged, false);
     }
 
     public String getFieldName(int number)
@@ -407,15 +235,7 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
         {
             case 1: return "queryState";
             case 2: return "queryId";
-            case 3: return "isLastChunk";
-            case 4: return "rowCount";
-            case 5: return "recordsScan";
-            case 6: return "recordsError";
-            case 7: return "submissionTime";
-            case 8: return "nodeStatus";
-            case 9: return "error";
-            case 10: return "def";
-            case 11: return "schemaChanged";
+            case 3: return "error";
             default: return null;
         }
     }
@@ -431,15 +251,7 @@ public final class QueryResult implements Externalizable, Message<QueryResult>,
     {
         __fieldMap.put("queryState", 1);
         __fieldMap.put("queryId", 2);
-        __fieldMap.put("isLastChunk", 3);
-        __fieldMap.put("rowCount", 4);
-        __fieldMap.put("recordsScan", 5);
-        __fieldMap.put("recordsError", 6);
-        __fieldMap.put("submissionTime", 7);
-        __fieldMap.put("nodeStatus", 8);
-        __fieldMap.put("error", 9);
-        __fieldMap.put("def", 10);
-        __fieldMap.put("schemaChanged", 11);
+        __fieldMap.put("error", 3);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcType.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcType.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcType.java
index 3f1f9fd..4d03073 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcType.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RpcType.java
@@ -28,10 +28,11 @@ public enum RpcType implements com.dyuproject.protostuff.EnumLite<RpcType>
     RUN_QUERY(3),
     CANCEL_QUERY(4),
     REQUEST_RESULTS(5),
-    QUERY_RESULT(6),
+    QUERY_DATA(6),
     QUERY_HANDLE(7),
     REQ_META_FUNCTIONS(8),
-    RESP_FUNCTION_LIST(9);
+    RESP_FUNCTION_LIST(9),
+    QUERY_RESULT(10);
     
     public final int number;
     
@@ -55,10 +56,11 @@ public enum RpcType implements com.dyuproject.protostuff.EnumLite<RpcType>
             case 3: return RUN_QUERY;
             case 4: return CANCEL_QUERY;
             case 5: return REQUEST_RESULTS;
-            case 6: return QUERY_RESULT;
+            case 6: return QUERY_DATA;
             case 7: return QUERY_HANDLE;
             case 8: return REQ_META_FUNCTIONS;
             case 9: return RESP_FUNCTION_LIST;
+            case 10: return QUERY_RESULT;
             default: return null;
         }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index 6c41a37..a083c59 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -14,16 +14,18 @@ enum RpcType {
   GOODBYE = 2;
 
   // user to bit
-  RUN_QUERY = 3;
-  CANCEL_QUERY = 4;
+  RUN_QUERY = 3; // user is submitting a new query to the drillbit
+  CANCEL_QUERY = 4; // user is sending a query cancellation request to the drillbit
   REQUEST_RESULTS = 5;
 
   // bit to user
-  QUERY_RESULT = 6;
+  QUERY_DATA = 6; // drillbit is sending a query result data batch to the user
   QUERY_HANDLE = 7;
 
   REQ_META_FUNCTIONS = 8;
   RESP_FUNCTION_LIST = 9;
+
+  QUERY_RESULT = 10; // drillbit is reporting a query status change, most likely a terminal message, to the user
 }
 
 message Property {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 1971e62..5e44655 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -95,6 +95,9 @@ message NodeStatus {
 	optional int64 memory_footprint = 2;
 }
 
+/*
+ * Used by the server to report informations about the query state to the client
+ */
 message QueryResult {
 	enum QueryState {
 	  PENDING = 0;
@@ -107,15 +110,16 @@ message QueryResult {
 
 	optional QueryState query_state = 1;
 	optional QueryId query_id = 2;
-	optional bool is_last_chunk = 3;
-	optional int32 row_count = 4;
-	optional int64 records_scan = 5;
-	optional int64 records_error = 6;
-	optional int64 submission_time = 7;
-	repeated NodeStatus node_status = 8;
-	repeated DrillPBError error = 9;
-	optional RecordBatchDef def = 10;
-	optional bool schema_changed = 11;
+	repeated DrillPBError error = 3;
+}
+
+/*
+ * Used by the server when sending query result data batches to the client
+ */
+message QueryData {
+  optional QueryId query_id = 1;
+  optional int32 row_count = 2;
+  optional RecordBatchDef def = 3;
 }
 
 message QueryInfo {


[3/9] drill git commit: DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
index 9bc0552..64033a5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
@@ -39,14 +39,14 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.testing.ExceptionInjectionUtil;
 import org.apache.drill.exec.testing.SimulatedExceptions.InjectionOption;
 import org.apache.drill.exec.testing.SimulatedExceptions.InjectionOptions;
@@ -184,11 +184,11 @@ public class TestDrillbitResilience extends ExecTest {
           private final RecordBatchLoader loader = new RecordBatchLoader(bufferAllocator);
 
           @Override
-          public void rowArrived(final QueryResultBatch queryResultBatch) {
+          public void rowArrived(final QueryDataBatch queryResultBatch) {
             // load the single record
-            final QueryResult queryResult = queryResultBatch.getHeader();
+            final QueryData queryData = queryResultBatch.getHeader();
             try {
-              loader.load(queryResult.getDef(), queryResultBatch.getData());
+              loader.load(queryData.getDef(), queryResultBatch.getData());
             } catch(SchemaChangeException e) {
               fail(e.toString());
             }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 9999be0..acfb522 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -49,7 +49,7 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.CachedSingleFileSystem;
@@ -145,11 +145,11 @@ public class ParquetRecordReaderTest extends BaseTestQuery{
   @Test
   public void testNullableAgg() throws Exception {
 
-    List<QueryResultBatch> result = testSqlWithResults("select sum(a) as total_sum from dfs.`/tmp/parquet_with_nulls_should_sum_100000_nulls_first.parquet`");
+    List<QueryDataBatch> result = testSqlWithResults("select sum(a) as total_sum from dfs.`/tmp/parquet_with_nulls_should_sum_100000_nulls_first.parquet`");
     assertEquals("Only expected one batch with data, and then the empty finishing batch.", 2, result.size());
     RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
 
-    QueryResultBatch b = result.get(0);
+    QueryDataBatch b = result.get(0);
     loader.load(b.getHeader().getDef(), b.getData());
 
     VectorWrapper vw = loader.getValueAccessorById(
@@ -163,11 +163,11 @@ public class ParquetRecordReaderTest extends BaseTestQuery{
 
   @Test
   public void testNullableFilter() throws Exception {
-    List<QueryResultBatch> result = testSqlWithResults("select count(wr_return_quantity) as row_count from dfs.`/tmp/web_returns` where wr_return_quantity = 1");
+    List<QueryDataBatch> result = testSqlWithResults("select count(wr_return_quantity) as row_count from dfs.`/tmp/web_returns` where wr_return_quantity = 1");
     assertEquals("Only expected one batch with data, and then the empty finishing batch.", 2, result.size());
     RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
 
-    QueryResultBatch b = result.get(0);
+    QueryDataBatch b = result.get(0);
     loader.load(b.getHeader().getDef(), b.getData());
 
     VectorWrapper vw = loader.getValueAccessorById(

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
index 52d5086..55f0d75 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
@@ -31,7 +31,7 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -39,7 +39,7 @@ import com.google.common.base.Strings;
 import com.google.common.util.concurrent.SettableFuture;
 
 public class ParquetResultListener implements UserResultsListener {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetResultListener.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetResultListener.class);
 
   private SettableFuture<Void> future = SettableFuture.create();
   int count = 0;
@@ -65,6 +65,10 @@ public class ParquetResultListener implements UserResultsListener {
     future.setException(ex);
   }
 
+  @Override
+  public void queryCompleted() {
+    checkLastChunk();
+  }
 
   private <T> void assertField(ValueVector valueVector, int index, TypeProtos.MinorType expectedMinorType, Object value, String name) {
     assertField(valueVector, index, expectedMinorType, value, name, 0);
@@ -94,11 +98,8 @@ public class ParquetResultListener implements UserResultsListener {
   }
 
   @Override
-  synchronized public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+  synchronized public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
     logger.debug("result arrived in test batch listener.");
-    if(result.getHeader().getIsLastChunk()){
-      future.set(null);
-    }
     int columnValCounter = 0;
     FieldInfo currentField;
     count += result.getHeader().getRowCount();
@@ -147,15 +148,12 @@ public class ParquetResultListener implements UserResultsListener {
       printRowMajor(batchLoader);
     }
     batchCounter++;
-    if(result.getHeader().getIsLastChunk()){
-      checkLastChunk(batchLoader, result);
-    }
 
     batchLoader.clear();
     result.release();
   }
 
-  public void checkLastChunk(RecordBatchLoader batchLoader, QueryResultBatch result) {
+  private void checkLastChunk() {
     int recordsInBatch = -1;
     // ensure the right number of columns was returned, especially important to ensure selective column read is working
     if (testValues) {
@@ -173,8 +171,6 @@ public class ParquetResultListener implements UserResultsListener {
     }
 
     assert valuesChecked.keySet().size() > 0;
-    batchLoader.clear();
-    result.release();
     future.set(null);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
index 6cb412c..882cdbd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
@@ -43,7 +43,7 @@ import com.google.common.base.Stopwatch;
 import com.google.common.io.Resources;
 
 public class TestParquetPhysicalPlan extends ExecTest {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestParquetPhysicalPlan.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestParquetPhysicalPlan.class);
 
   public String fileName = "parquet/parquet_scan_filter_union_screen_physical.json";
 
@@ -56,10 +56,10 @@ public class TestParquetPhysicalPlan extends ExecTest {
     try (Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         count += b.getHeader().getRowCount();
         loader.load(b.getHeader().getDef(), b.getData());
@@ -96,13 +96,15 @@ public class TestParquetPhysicalPlan extends ExecTest {
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+    public void queryCompleted() {
+      latch.countDown();
+    }
+
+    @Override
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
       int rows = result.getHeader().getRowCount();
       System.out.println(String.format("Result batch arrived. Number of records: %d", rows));
       count.addAndGet(rows);
-      if (result.getHeader().getIsLastChunk()) {
-        latch.countDown();
-      }
       result.release();
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
index e5a2c94..3c1a38a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
@@ -27,7 +27,7 @@ import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
@@ -41,7 +41,7 @@ public class TestTextColumn extends BaseTestQuery{
 
   @Test
   public void testDefaultDelimiterColumnSelection() throws Exception {
-    List<QueryResultBatch> batches = testSqlWithResults("SELECT columns[0] as entire_row " +
+    List<QueryDataBatch> batches = testSqlWithResults("SELECT columns[0] as entire_row " +
       "from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.txt`");
 
     List<List<String>> expectedOutput = Arrays.asList(
@@ -55,7 +55,7 @@ public class TestTextColumn extends BaseTestQuery{
 
   @Test
   public void testCsvColumnSelectionCommasInsideQuotes() throws Exception {
-    List<QueryResultBatch> batches = testSqlWithResults("SELECT columns[0] as col1, columns[1] as col2, columns[2] as col3," +
+    List<QueryDataBatch> batches = testSqlWithResults("SELECT columns[0] as col1, columns[1] as col2, columns[2] as col3," +
       "columns[3] as col4 from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.csv`");
 
     List<List<String>> expectedOutput = Arrays.asList(
@@ -67,11 +67,11 @@ public class TestTextColumn extends BaseTestQuery{
     validateOutput(expectedOutput, actualOutput);
   }
 
-  private List<List<String>> getOutput(List<QueryResultBatch> batches) throws SchemaChangeException {
+  private List<List<String>> getOutput(List<QueryDataBatch> batches) throws SchemaChangeException {
     List<List<String>> output = new ArrayList<>();
     RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
     int last = 0;
-    for(QueryResultBatch batch : batches) {
+    for(QueryDataBatch batch : batches) {
       int rows = batch.getHeader().getRowCount();
       if(batch.getData() != null) {
         loader.load(batch.getHeader().getDef(), batch.getData());

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
index fa43d55..5e781d2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.util.VectorUtil;
@@ -45,13 +45,13 @@ public class TextRecordReaderTest extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(
                       FileUtils.getResourceAsFile("/store/text/test.json"), Charsets.UTF_8)
                       .replace("#{DATA_FILE}", FileUtils.getResourceAsFile("/store/text/data/regions.csv").toURI().toString()));
       int count = 0;
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for(QueryResultBatch b : results) {
+      for(QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/fn/TestJsonReaderWithSparseFiles.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/fn/TestJsonReaderWithSparseFiles.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/fn/TestJsonReaderWithSparseFiles.java
index 437bbb5..d674d47 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/fn/TestJsonReaderWithSparseFiles.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/fn/TestJsonReaderWithSparseFiles.java
@@ -23,7 +23,7 @@ import java.util.Objects;
 
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.exec.util.JsonStringHashMap;
 import org.apache.drill.exec.vector.ValueVector;
@@ -82,15 +82,15 @@ public class TestJsonReaderWithSparseFiles extends BaseTestQuery {
   }
 
   protected void query(final String query, final Function<RecordBatchLoader> testBody) throws Exception {
-    List<QueryResultBatch> batches = testSqlWithResults(query);
+    List<QueryDataBatch> batches = testSqlWithResults(query);
     RecordBatchLoader loader = new RecordBatchLoader(client.getAllocator());
     try {
       // first batch at index 0 is empty and used for fast schema return. Load the second one for the tests
-      QueryResultBatch batch = batches.get(0);
+      QueryDataBatch batch = batches.get(0);
       loader.load(batch.getHeader().getDef(), batch.getData());
       testBody.apply(loader);
     } finally {
-      for (QueryResultBatch batch:batches) {
+      for (QueryDataBatch batch:batches) {
         batch.release();
       }
       loader.clear();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexToJson.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexToJson.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexToJson.java
index eedbf6f..656429e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexToJson.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexToJson.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.Test;
 
 public class TestComplexToJson extends BaseTestQuery {
@@ -36,7 +36,7 @@ public class TestComplexToJson extends BaseTestQuery {
   public void test() throws Exception {
     DrillClient parent_client = client;
 
-    List<QueryResultBatch> results;
+    List<QueryDataBatch> results;
     RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
 
     client = new DrillClient(config, serviceSet.getCoordinator());
@@ -50,7 +50,7 @@ public class TestComplexToJson extends BaseTestQuery {
     // with setSupportComplexTypes == false, the column mode should be REQUIRED
     assertTrue(def.getField(0).getMajorType().getMode() == DataMode.REQUIRED);
     loader.clear();
-    for(QueryResultBatch result : results) {
+    for(QueryDataBatch result : results) {
       result.release();
     }
     client.close();
@@ -66,7 +66,7 @@ public class TestComplexToJson extends BaseTestQuery {
     // with setSupportComplexTypes == true, the column mode should be REPEATED
     assertTrue(def.getField(0).getMajorType().getMode() == DataMode.REPEATED);
     loader.clear();
-    for(QueryResultBatch result : results) {
+    for(QueryDataBatch result : results) {
       result.release();
     }
     client.close();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
index fe86192..3f69fd0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.RepeatedBigIntVector;
 import org.junit.Ignore;
@@ -48,7 +48,7 @@ import com.google.common.io.Files;
 import org.junit.rules.TemporaryFolder;
 
 public class TestJsonReader extends BaseTestQuery {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJsonReader.class);
+//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJsonReader.class);
 
   private static final boolean VERBOSE_DEBUG = false;
 
@@ -196,15 +196,15 @@ public class TestJsonReader extends BaseTestQuery {
 
   @Test
   public void readComplexWithStar() throws Exception {
-    List<QueryResultBatch> results = testSqlWithResults("select * from cp.`/store/json/test_complex_read_with_star.json`");
-    assertEquals(2, results.size());
+    List<QueryDataBatch> results = testSqlWithResults("select * from cp.`/store/json/test_complex_read_with_star.json`");
+    assertEquals(1, results.size());
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
-    QueryResultBatch batch = results.get(0);
+    QueryDataBatch batch = results.get(0);
 
     assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
     assertEquals(3, batchLoader.getSchema().getFieldCount());
-    testExistentColumns(batchLoader, batch);
+    testExistentColumns(batchLoader);
 
     batch.release();
     batchLoader.clear();
@@ -249,24 +249,24 @@ public class TestJsonReader extends BaseTestQuery {
     test("alter system set `store.json.all_text_mode` = false");
     runTestsOnFile(filename, UserBitShared.QueryType.PHYSICAL, queries, rowCounts);
 
-    List<QueryResultBatch> results = testPhysicalWithResults(queries[0]);
-    assertEquals(2, results.size());
+    List<QueryDataBatch> results = testPhysicalWithResults(queries[0]);
+    assertEquals(1, results.size());
     // "`field_1`", "`field_3`.`inner_1`", "`field_3`.`inner_2`", "`field_4`.`inner_1`"
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
-    QueryResultBatch batch = results.get(0);
+    QueryDataBatch batch = results.get(0);
     assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
     // this used to be five.  It is now three.  This is because the plan doesn't have a project.
     // Scanners are not responsible for projecting non-existent columns (as long as they project one column)
     assertEquals(3, batchLoader.getSchema().getFieldCount());
-    testExistentColumns(batchLoader, batch);
+    testExistentColumns(batchLoader);
 
     batch.release();
     batchLoader.clear();
   }
 
-  private void testExistentColumns(RecordBatchLoader batchLoader, QueryResultBatch batch) throws SchemaChangeException {
+  private void testExistentColumns(RecordBatchLoader batchLoader) throws SchemaChangeException {
     VectorWrapper<?> vw = batchLoader.getValueAccessorById(
         RepeatedBigIntVector.class, //
         batchLoader.getValueVectorId(SchemaPath.getCompoundPath("field_1")).getFieldIds() //

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
index b3c653f..dcea9bb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -46,11 +46,11 @@ public class TestSpoolingBuffer extends ExecTest {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/work/batch/multiple_exchange.json"),
                       Charsets.UTF_8));
       int count = 0;
-      for(QueryResultBatch b : results) {
+      for(QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
index cddd999..3b38a09 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillCursor.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 
 public class DrillCursor implements Cursor {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCursor.class);
@@ -113,7 +113,7 @@ public class DrillCursor implements Cursor {
       // Next index is not in current batch (including initial empty batch--
       // (try to) get next batch.
       try {
-        QueryResultBatch qrb = resultsListener.getNext();
+        QueryDataBatch qrb = resultsListener.getNext();
         recordBatchCount++;
         while (qrb != null && qrb.getHeader().getRowCount() == 0 && !first) {
           qrb.release();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
index 0ce33f4..fb27d2d 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
@@ -19,8 +19,6 @@ package org.apache.drill.jdbc;
 
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.TimeZone;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingDeque;
@@ -30,24 +28,21 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import net.hydromatic.avatica.AvaticaPrepareResult;
 import net.hydromatic.avatica.AvaticaResultSet;
 import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.avatica.Cursor.Accessor;
 
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 
 import com.google.common.collect.Queues;
 
 public class DrillResultSet extends AvaticaResultSet {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillResultSet.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillResultSet.class);
 
   SchemaChangeListener changeListener;
   final ResultsListener resultslistener = new ResultsListener();
@@ -107,8 +102,7 @@ public class DrillResultSet extends AvaticaResultSet {
     // don't return with metadata until we've achieved at least one return message.
     try {
       resultslistener.latch.await();
-      boolean notAtEnd = cursor.next();
-      assert notAtEnd;
+      cursor.next();
     } catch (InterruptedException e) {
      // TODO:  Check:  Should this call Thread.currentThread.interrupt()?   If
      // not, at least document why this is empty.
@@ -137,7 +131,7 @@ public class DrillResultSet extends AvaticaResultSet {
 
 
 
-    final LinkedBlockingDeque<QueryResultBatch> queue = Queues.newLinkedBlockingDeque();
+    final LinkedBlockingDeque<QueryDataBatch> queue = Queues.newLinkedBlockingDeque();
 
     // TODO:  Doc.:  Release what if what is first relative to what?
     private boolean releaseIfFirst() {
@@ -151,7 +145,6 @@ public class DrillResultSet extends AvaticaResultSet {
 
     @Override
     public void submissionFailed(RpcException ex) {
-      releaseIfFirst();
       this.ex = ex;
       completed = true;
       close();
@@ -159,13 +152,14 @@ public class DrillResultSet extends AvaticaResultSet {
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      logger.debug("Result arrived {}", result);
+    public void queryCompleted() {
+      releaseIfFirst();
+      completed = true;
+    }
 
-      if (result.getHeader().hasQueryState() && result.getHeader().getQueryState() == QueryState.COMPLETED && result.getHeader().getRowCount() == 0) {
-        result.release();
-        return;
-      }
+    @Override
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+      logger.debug("Result arrived {}", result);
 
       // If we're in a closed state, just release the message.
       if (closed) {
@@ -182,20 +176,11 @@ public class DrillResultSet extends AvaticaResultSet {
         autoread = false;
       }
 
-      if (result.getHeader().getIsLastChunk()) {
-        completed = true;
-      }
-
-      if (result.getHeader().getErrorCount() > 0) {
-        submissionFailed(new RpcException(String.format("%s", result.getHeader().getErrorList())));
-      }
-
       releaseIfFirst();
-
     }
 
     // TODO:  Doc.:  Specify whether result can be null and what that means.
-    public QueryResultBatch getNext() throws RpcException, InterruptedException {
+    public QueryDataBatch getNext() throws RpcException, InterruptedException {
       while (true) {
         if (ex != null) {
           throw ex;
@@ -203,7 +188,7 @@ public class DrillResultSet extends AvaticaResultSet {
         if (completed && queue.isEmpty()) {
           return null;
         } else {
-          QueryResultBatch q = queue.poll(50, TimeUnit.MILLISECONDS);
+          QueryDataBatch q = queue.poll(50, TimeUnit.MILLISECONDS);
           if (q != null) {
             if (!autoread && queue.size() < MAX / 2) {
               autoread = true;
@@ -219,7 +204,7 @@ public class DrillResultSet extends AvaticaResultSet {
     void close() {
       closed = true;
       while (!queue.isEmpty()) {
-        QueryResultBatch qrb = queue.poll();
+        QueryDataBatch qrb = queue.poll();
         if (qrb != null && qrb.getData() != null) {
           qrb.getData().release();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
index 68e86db..f72d5e1 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
@@ -1359,27 +1359,9 @@ public final class SchemaUserBitShared
                 if(message.hasQueryId())
                     output.writeObject(2, message.getQueryId(), org.apache.drill.exec.proto.SchemaUserBitShared.QueryId.WRITE, false);
 
-                if(message.hasIsLastChunk())
-                    output.writeBool(3, message.getIsLastChunk(), false);
-                if(message.hasRowCount())
-                    output.writeInt32(4, message.getRowCount(), false);
-                if(message.hasRecordsScan())
-                    output.writeInt64(5, message.getRecordsScan(), false);
-                if(message.hasRecordsError())
-                    output.writeInt64(6, message.getRecordsError(), false);
-                if(message.hasSubmissionTime())
-                    output.writeInt64(7, message.getSubmissionTime(), false);
-                for(org.apache.drill.exec.proto.UserBitShared.NodeStatus nodeStatus : message.getNodeStatusList())
-                    output.writeObject(8, nodeStatus, org.apache.drill.exec.proto.SchemaUserBitShared.NodeStatus.WRITE, true);
-
                 for(org.apache.drill.exec.proto.UserBitShared.DrillPBError error : message.getErrorList())
-                    output.writeObject(9, error, org.apache.drill.exec.proto.SchemaUserBitShared.DrillPBError.WRITE, true);
-
-                if(message.hasDef())
-                    output.writeObject(10, message.getDef(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.WRITE, false);
+                    output.writeObject(3, error, org.apache.drill.exec.proto.SchemaUserBitShared.DrillPBError.WRITE, true);
 
-                if(message.hasSchemaChanged())
-                    output.writeBool(11, message.getSchemaChanged(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.QueryResult message)
             {
@@ -1427,35 +1409,9 @@ public final class SchemaUserBitShared
 
                             break;
                         case 3:
-                            builder.setIsLastChunk(input.readBool());
-                            break;
-                        case 4:
-                            builder.setRowCount(input.readInt32());
-                            break;
-                        case 5:
-                            builder.setRecordsScan(input.readInt64());
-                            break;
-                        case 6:
-                            builder.setRecordsError(input.readInt64());
-                            break;
-                        case 7:
-                            builder.setSubmissionTime(input.readInt64());
-                            break;
-                        case 8:
-                            builder.addNodeStatus(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.NodeStatus.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.NodeStatus.MERGE));
-
-                            break;
-                        case 9:
                             builder.addError(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.DrillPBError.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.DrillPBError.MERGE));
 
                             break;
-                        case 10:
-                            builder.setDef(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.MERGE));
-
-                            break;
-                        case 11:
-                            builder.setSchemaChanged(input.readBool());
-                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -1498,15 +1454,7 @@ public final class SchemaUserBitShared
             {
                 case 1: return "queryState";
                 case 2: return "queryId";
-                case 3: return "isLastChunk";
-                case 4: return "rowCount";
-                case 5: return "recordsScan";
-                case 6: return "recordsError";
-                case 7: return "submissionTime";
-                case 8: return "nodeStatus";
-                case 9: return "error";
-                case 10: return "def";
-                case 11: return "schemaChanged";
+                case 3: return "error";
                 default: return null;
             }
         }
@@ -1520,15 +1468,136 @@ public final class SchemaUserBitShared
         {
             fieldMap.put("queryState", 1);
             fieldMap.put("queryId", 2);
-            fieldMap.put("isLastChunk", 3);
-            fieldMap.put("rowCount", 4);
-            fieldMap.put("recordsScan", 5);
-            fieldMap.put("recordsError", 6);
-            fieldMap.put("submissionTime", 7);
-            fieldMap.put("nodeStatus", 8);
-            fieldMap.put("error", 9);
-            fieldMap.put("def", 10);
-            fieldMap.put("schemaChanged", 11);
+            fieldMap.put("error", 3);
+        }
+    }
+
+    public static final class QueryData
+    {
+        public static final org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.MessageSchema WRITE =
+            new org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.MessageSchema();
+        public static final org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.BuilderSchema MERGE =
+            new org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.BuilderSchema();
+        
+        public static class MessageSchema implements com.dyuproject.protostuff.Schema<org.apache.drill.exec.proto.UserBitShared.QueryData>
+        {
+            public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.exec.proto.UserBitShared.QueryData message) throws java.io.IOException
+            {
+                if(message.hasQueryId())
+                    output.writeObject(1, message.getQueryId(), org.apache.drill.exec.proto.SchemaUserBitShared.QueryId.WRITE, false);
+
+                if(message.hasRowCount())
+                    output.writeInt32(2, message.getRowCount(), false);
+                if(message.hasDef())
+                    output.writeObject(3, message.getDef(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.WRITE, false);
+
+            }
+            public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.QueryData message)
+            {
+                return message.isInitialized();
+            }
+            public java.lang.String getFieldName(int number)
+            {
+                return org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.getFieldName(number);
+            }
+            public int getFieldNumber(java.lang.String name)
+            {
+                return org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.getFieldNumber(name);
+            }
+            public java.lang.Class<org.apache.drill.exec.proto.UserBitShared.QueryData> typeClass()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.class;
+            }
+            public java.lang.String messageName()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.class.getSimpleName();
+            }
+            public java.lang.String messageFullName()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.class.getName();
+            }
+            //unused
+            public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.exec.proto.UserBitShared.QueryData message) throws java.io.IOException {}
+            public org.apache.drill.exec.proto.UserBitShared.QueryData newMessage() { return null; }
+        }
+        public static class BuilderSchema implements com.dyuproject.protostuff.Schema<org.apache.drill.exec.proto.UserBitShared.QueryData.Builder>
+        {
+            public void mergeFrom(com.dyuproject.protostuff.Input input, org.apache.drill.exec.proto.UserBitShared.QueryData.Builder builder) throws java.io.IOException
+            {
+                for(int number = input.readFieldNumber(this);; number = input.readFieldNumber(this))
+                {
+                    switch(number)
+                    {
+                        case 0:
+                            return;
+                        case 1:
+                            builder.setQueryId(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.QueryId.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.QueryId.MERGE));
+
+                            break;
+                        case 2:
+                            builder.setRowCount(input.readInt32());
+                            break;
+                        case 3:
+                            builder.setDef(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.MERGE));
+
+                            break;
+                        default:
+                            input.handleUnknownField(number, this);
+                    }
+                }
+            }
+            public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.QueryData.Builder builder)
+            {
+                return builder.isInitialized();
+            }
+            public org.apache.drill.exec.proto.UserBitShared.QueryData.Builder newMessage()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.newBuilder();
+            }
+            public java.lang.String getFieldName(int number)
+            {
+                return org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.getFieldName(number);
+            }
+            public int getFieldNumber(java.lang.String name)
+            {
+                return org.apache.drill.exec.proto.SchemaUserBitShared.QueryData.getFieldNumber(name);
+            }
+            public java.lang.Class<org.apache.drill.exec.proto.UserBitShared.QueryData.Builder> typeClass()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.Builder.class;
+            }
+            public java.lang.String messageName()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.class.getSimpleName();
+            }
+            public java.lang.String messageFullName()
+            {
+                return org.apache.drill.exec.proto.UserBitShared.QueryData.class.getName();
+            }
+            //unused
+            public void writeTo(com.dyuproject.protostuff.Output output, org.apache.drill.exec.proto.UserBitShared.QueryData.Builder builder) throws java.io.IOException {}
+        }
+        public static java.lang.String getFieldName(int number)
+        {
+            switch(number)
+            {
+                case 1: return "queryId";
+                case 2: return "rowCount";
+                case 3: return "def";
+                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("queryId", 1);
+            fieldMap.put("rowCount", 2);
+            fieldMap.put("def", 3);
         }
     }
 


[7/9] drill git commit: DRILL-2674: Add user authenticator interface and PAM based implementation.

Posted by pa...@apache.org.
DRILL-2674: Add user authenticator interface and PAM based implementation.


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

Branch: refs/heads/master
Commit: 6a8e0ec101cf082d9200b22c668a9b04451263ad
Parents: a218ee3
Author: vkorukanti <ve...@gmail.com>
Authored: Thu Apr 2 22:41:00 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Apr 3 18:40:53 2015 -0700

----------------------------------------------------------------------
 .../src/resources/drill-override-example.conf   |   8 +-
 exec/java-exec/pom.xml                          |   4 +
 .../org/apache/drill/exec/ExecConstants.java    |   4 +
 .../apache/drill/exec/rpc/user/UserServer.java  |  42 +++++++-
 .../apache/drill/exec/rpc/user/UserSession.java |   2 +
 .../rpc/user/security/PamUserAuthenticator.java |  71 +++++++++++++
 .../security/UserAuthenticationException.java   |  32 ++++++
 .../rpc/user/security/UserAuthenticator.java    |  53 ++++++++++
 .../user/security/UserAuthenticatorFactory.java | 106 +++++++++++++++++++
 .../security/UserAuthenticatorTemplate.java     |  37 +++++++
 .../drill/exec/service/ServiceEngine.java       |   8 +-
 .../src/main/resources/drill-module.conf        |   8 +-
 .../java/org/apache/drill/BaseTestQuery.java    |  59 ++++++++---
 .../java/org/apache/drill/QueryTestUtil.java    |   9 +-
 .../exec/physical/impl/TestLocalExchange.java   |   2 +-
 .../partitionsender/TestPartitionSender.java    |   2 +-
 .../security/TestCustomUserAuthenticator.java   | 102 ++++++++++++++++++
 .../security/UserAuthenticatorTestImpl.java     |  62 +++++++++++
 .../exec/server/TestDrillbitResilience.java     |   2 +-
 pom.xml                                         |  24 +++++
 20 files changed, 607 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/distribution/src/resources/drill-override-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-override-example.conf b/distribution/src/resources/drill-override-example.conf
index 4cd342a..943d644 100644
--- a/distribution/src/resources/drill-override-example.conf
+++ b/distribution/src/resources/drill-override-example.conf
@@ -105,7 +105,13 @@ drill.exec: {
       path: "/tmp/drill",
       write: true
     }
-  }
+  },
+  security.user.auth {
+    enabled: false,
+    packages += "org.apache.drill.exec.rpc.user.security",
+    impl: "pam",
+    pam_profiles: [ "sudo", "login" ]
+  },
   trace: {
     directory: "/tmp/drill-trace",
     filesystem: "file:///"

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index dfb1a6b..7c08534 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -133,6 +133,10 @@
       <artifactId>optiq-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>net.sf.jpam</groupId>
+      <artifactId>jpam</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.freemarker</groupId>
       <artifactId>freemarker</artifactId>
       <version>2.3.19</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 14e6ad1..bd93206 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -74,6 +74,10 @@ public interface ExecConstants {
   public static final String SYS_STORE_PROVIDER_CLASS = "drill.exec.sys.store.provider.class";
   public static final String SYS_STORE_PROVIDER_LOCAL_PATH = "drill.exec.sys.store.provider.local.path";
   public static final String SYS_STORE_PROVIDER_LOCAL_ENABLE_WRITE = "drill.exec.sys.store.provider.local.write";
+  public static final String USER_AUTHENTICATOR_IMPL_PACKAGES = "drill.exec.security.user.auth.packages";
+  public static final String USER_AUTHENTICATION_ENABLED = "drill.exec.security.user.auth.enabled";
+  public static final String USER_AUTHENTICATOR_IMPL = "drill.exec.security.user.auth.impl";
+  public static final String PAM_AUTHENTICATOR_PROFILES = "drill.exec.security.user.auth.pam_profiles";
   public static final String ERROR_ON_MEMORY_LEAK = "drill.exec.debug.error_on_leak";
   /** Fragment memory planning */
   public static final String ENABLE_FRAGMENT_MEMORY_LIMIT = "drill.exec.memory.enable_frag_limit";

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 17f189d..877bc08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.rpc.user;
 
+import com.google.common.io.Closeables;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.channel.Channel;
@@ -26,6 +27,9 @@ import io.netty.channel.EventLoopGroup;
 import java.io.IOException;
 import java.util.UUID;
 
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
@@ -34,8 +38,10 @@ import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.HandshakeStatus;
+import org.apache.drill.exec.proto.UserProtos.Property;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.OutOfMemoryHandler;
@@ -45,6 +51,9 @@ import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
 import org.apache.drill.exec.work.user.UserWorker;
 
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -55,11 +64,18 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
 
   final UserWorker worker;
   final BufferAllocator alloc;
+  final UserAuthenticator authenticator;
 
-  public UserServer(BufferAllocator alloc, EventLoopGroup eventLoopGroup, UserWorker worker) {
+  public UserServer(DrillConfig config, BufferAllocator alloc, EventLoopGroup eventLoopGroup,
+      UserWorker worker) throws DrillbitStartupException {
     super(UserRpcConfig.MAPPING, alloc.getUnderlyingAllocator(), eventLoopGroup);
     this.worker = worker;
     this.alloc = alloc;
+    if (config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED)) {
+      authenticator = UserAuthenticatorFactory.createAuthenticator(config);
+    } else {
+      authenticator = null;
+    }
   }
 
   @Override
@@ -101,7 +117,6 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
 
   }
 
-
   public class UserClientConnection extends RemoteConnection {
 
     private UserSession session;
@@ -180,6 +195,23 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
             return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null);
           }
 
+          if (authenticator != null) {
+            try {
+              String password = "";
+              final UserProperties props = inbound.getProperties();
+              for (int i = 0; i < props.getPropertiesCount(); i++) {
+                Property prop = props.getProperties(i);
+                if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
+                  password = prop.getValue();
+                  break;
+                }
+              }
+              authenticator.authenticate(inbound.getCredentials().getUserName(), password);
+            } catch (UserAuthenticationException ex) {
+              return handleFailure(respBuilder, HandshakeStatus.AUTH_FAILED, ex.getMessage(), ex);
+            }
+          }
+
           connection.setUser(inbound);
 
           return respBuilder.setStatus(HandshakeStatus.SUCCESS).build();
@@ -221,4 +253,10 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
   public ProtobufLengthDecoder getDecoder(BufferAllocator allocator, OutOfMemoryHandler outOfMemoryHandler) {
     return new UserProtobufLengthDecoder(allocator, outOfMemoryHandler);
   }
+
+  @Override
+  public void close() throws IOException {
+    Closeables.closeQuietly(authenticator);
+    super.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/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 efb0cdf..19d77b0 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
@@ -33,6 +33,8 @@ public class UserSession {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class);
 
   public static final String SCHEMA = "schema";
+  public static final String USER = "user";
+  public static final String PASSWORD = "password";
 
   private DrillUser user;
   private boolean enableExchanges = true;

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/PamUserAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/PamUserAuthenticator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/PamUserAuthenticator.java
new file mode 100644
index 0000000..2928bfb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/PamUserAuthenticator.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import net.sf.jpam.Pam;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Implement {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} based on Pluggable Authentication
+ * Module (PAM) configuration. Configure the PAM profiles using "drill.exec.security.user.auth.pam_profiles" BOOT
+ * option. Ex. value  <i>[ "login", "sudo" ]</i> (value is an array of strings).
+ */
+@UserAuthenticatorTemplate(type = "pam")
+public class PamUserAuthenticator implements UserAuthenticator {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PamUserAuthenticator.class);
+
+  private List<String> profiles;
+
+  @Override
+  public void setup(DrillConfig drillConfig) throws DrillbitStartupException {
+    profiles = DrillConfig.create().getStringList(ExecConstants.PAM_AUTHENTICATOR_PROFILES);
+
+    // Create a JPAM object so that it triggers loading of native "jpamlib" needed. Issues in loading/finding native
+    // "jpamlib" will be found it Drillbit start rather than when authenticating the first user.
+    try {
+      new Pam();
+    } catch(LinkageError e) {
+      final String errMsg = "Problem in finding the native library of JPAM (Pluggable Authenticator Module API). " +
+          "Make sure to set Drillbit JVM option 'java.library.path' to point to the directory where the native " +
+          "JPAM exists.";
+      logger.error(errMsg, e);
+      throw new DrillbitStartupException(errMsg + ":" + e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void authenticate(String user, String password) throws UserAuthenticationException {
+    for (String pamProfile : profiles) {
+      Pam pam = new Pam(pamProfile);
+      if (!pam.authenticateSuccessful(user, password)) {
+        throw new UserAuthenticationException(String.format("PAM profile '%s' validation failed", pamProfile));
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // No-op as no resources are occupied by PAM authenticator.
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticationException.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticationException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticationException.java
new file mode 100644
index 0000000..ae1ce32
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticationException.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import org.apache.drill.common.exceptions.DrillException;
+
+public class UserAuthenticationException extends DrillException {
+  private static final String ERROR_MSG = "Invalid user credentials";
+
+  public UserAuthenticationException() {
+    super(ERROR_MSG);
+  }
+
+  public UserAuthenticationException(String reason) {
+    super(String.format("%s: %s", ERROR_MSG, reason));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticator.java
new file mode 100644
index 0000000..4d83138
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticator.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Interface to provide various username/password based implementations for authentication.
+ */
+public interface UserAuthenticator extends Closeable {
+
+  /**
+   * Setup for authenticating user credentials.
+   */
+  public void setup(DrillConfig drillConfig) throws DrillbitStartupException;
+
+  /**
+   * Authenticate the given <i>user</i> and <i>password</i> combination.
+   *
+   * @param user
+   * @param password
+   * @throws UserAuthenticationException if authentication fails for given user and password.
+   */
+  public void authenticate(String user, String password) throws UserAuthenticationException;
+
+  /**
+   * Close the authenticator. Used to release resources. Ex. LDAP authenticator opens connections to LDAP server,
+   * such connections resources are released in a safe manner as part of close.
+   *
+   * @throws IOException
+   */
+  @Override
+  void close() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
new file mode 100644
index 0000000..51a5979
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import com.google.common.base.Strings;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.PathScanner;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+
+import java.lang.reflect.Constructor;
+import java.util.Collection;
+
+import static org.apache.drill.exec.ExecConstants.USER_AUTHENTICATOR_IMPL;
+
+/**
+ * Factory class which provides {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} implementation
+ * based on the BOOT options.
+ */
+public class UserAuthenticatorFactory {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticatorFactory.class);
+
+  /**
+   * Create a {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} implementation based on BOOT settings in
+   * given <i>drillConfig</i>.
+   *
+   * @param config DrillConfig containing BOOT options.
+   * @return Initialized {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} implementation instance.
+   *         It is responsibility of the caller to close the authenticator when no longer needed.
+   *
+   * @throws DrillbitStartupException when no implementation found for given BOOT options.
+   */
+  public static UserAuthenticator createAuthenticator(final DrillConfig config) throws DrillbitStartupException {
+    final String authImplConfigured = config.getString(USER_AUTHENTICATOR_IMPL);
+
+    if (Strings.isNullOrEmpty(authImplConfigured)) {
+      throw new DrillbitStartupException(String.format("Invalid value '%s' for BOOT option '%s'", authImplConfigured,
+          USER_AUTHENTICATOR_IMPL));
+    }
+
+    final Collection<Class<? extends UserAuthenticator>> authImpls =
+        PathScanner.scanForImplementations(UserAuthenticator.class,
+            config.getStringList(ExecConstants.USER_AUTHENTICATOR_IMPL_PACKAGES));
+
+    for(Class<? extends UserAuthenticator> clazz : authImpls) {
+      final UserAuthenticatorTemplate template = clazz.getAnnotation(UserAuthenticatorTemplate.class);
+      if (template == null) {
+        logger.warn("{} doesn't have {} annotation. Skipping.", clazz.getCanonicalName(), UserAuthenticatorTemplate.class);
+        continue;
+      }
+
+      if (Strings.isNullOrEmpty(template.type())) {
+        logger.warn("{} annotation doesn't have valid type field for UserAuthenticator implementation {}. Skipping..",
+            UserAuthenticatorTemplate.class, clazz.getCanonicalName());
+        continue;
+      }
+
+      if (template.type().equalsIgnoreCase(authImplConfigured)) {
+        Constructor<?> validConstructor = null;
+        for (Constructor<?> c : clazz.getConstructors()) {
+          if (c.getParameterTypes().length == 0) {
+            validConstructor = c;
+            break;
+          }
+        }
+
+        if (validConstructor == null) {
+          logger.warn("Skipping UserAuthenticator implementation class '{}' since it doesn't " +
+              "implement a constructor [{}()]", clazz.getCanonicalName(), clazz.getName());
+          continue;
+        }
+
+        // Instantiate authenticator and initialize it
+        try {
+          final UserAuthenticator authenticator = clazz.newInstance();
+          authenticator.setup(config);
+          return authenticator;
+        } catch(IllegalArgumentException | IllegalAccessException | InstantiationException e) {
+          throw new DrillbitStartupException(
+              String.format("Failed to create and initialize the UserAuthenticator class '{}'",
+                  clazz.getCanonicalName()), e);
+        }
+      }
+    }
+
+    String errMsg = String.format("Failed to find the implementation of '{}' for type '{}'",
+        UserAuthenticator.class.getCanonicalName(), authImplConfigured);
+    logger.error(errMsg);
+    throw new DrillbitStartupException(errMsg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTemplate.java
new file mode 100644
index 0000000..04be8d1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTemplate.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation for {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} implementation to identify the
+ * implementation type. Implementation type is set in BOOT option <i>drill.exec.security.user.auth.impl</i>.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface UserAuthenticatorTemplate {
+  /**
+   * {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} implementation type.
+   * @return
+   */
+  String type();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index 2efc9a9..25ea307 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 
+import io.netty.channel.EventLoopGroup;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
@@ -49,8 +50,11 @@ public class ServiceEngine implements Closeable{
   boolean useIP = false;
   private final boolean allowPortHunting;
 
-  public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting){
-    this.userServer = new UserServer(context.getAllocator(), TransportCheck.createEventLoopGroup(context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS), "UserServer-"), userWorker);
+  public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context,
+      WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting) throws DrillbitStartupException {
+    final EventLoopGroup eventLoopGroup = TransportCheck.createEventLoopGroup(
+        context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS), "UserServer-");
+    this.userServer = new UserServer(context.getConfig(), context.getAllocator(), eventLoopGroup, userWorker);
     this.controller = new ControllerImpl(context, controlMessageHandler, allowPortHunting);
     this.dataPool = new DataConnectionCreator(context, workBus, dataHandler, allowPortHunting);
     this.config = context.getConfig();

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index af225c4..6bd8db0 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -100,7 +100,13 @@ drill.exec: {
       path: "/tmp/drill",
       write: true
     }
-  }
+  },
+  security.user.auth {
+    enabled: false,
+    packages += "org.apache.drill.exec.rpc.user.security",
+    impl: "pam",
+    pam_profiles: [ "sudo", "login" ]
+  },
   trace: {
     directory: "/tmp/drill-trace",
     filesystem: "file:///"

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index c602a01..725594a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -56,16 +56,8 @@ import com.google.common.io.Resources;
 public class BaseTestQuery extends ExecTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
 
-  /**
-   * Number of Drillbits in test cluster. Default is 1.
-   *
-   * Tests can update the cluster size through {@link #setDrillbitCount(int)}
-   */
-  private static int drillbitCount = 1;
-
-  private int[] columnWidths = new int[] { 8 };
-
   private static final String ENABLE_FULL_CACHE = "drill.exec.test.use-full-cache";
+  private static final int MAX_WIDTH_PER_NODE = 2;
 
   @SuppressWarnings("serial")
   private static final Properties TEST_CONFIGURATIONS = new Properties() {
@@ -93,17 +85,37 @@ public class BaseTestQuery extends ExecTest {
   protected static QuerySubmitter submitter = new QuerySubmitter();
   protected static BufferAllocator allocator;
 
-  protected static void setDrillbitCount(int newDrillbitCount) {
+  /**
+   * Number of Drillbits in test cluster. Default is 1.
+   *
+   * Tests can update the cluster size through {@link #updateTestCluster(int, DrillConfig)}
+   */
+  private static int drillbitCount = 1;
+
+  private int[] columnWidths = new int[] { 8 };
+
+  @BeforeClass
+  public static void setupDefaultTestCluster() throws Exception {
+    config = DrillConfig.create(TEST_CONFIGURATIONS);
+    openClient();
+  }
+
+  protected static void updateTestCluster(int newDrillbitCount, DrillConfig newConfig) {
     Preconditions.checkArgument(newDrillbitCount > 0, "Number of Drillbits must be at least one");
-    if (drillbitCount != newDrillbitCount) {
+    if (drillbitCount != newDrillbitCount || config != null) {
       // TODO: Currently we have to shutdown the existing Drillbit cluster before starting a new one with the given
       // Drillbit count. Revisit later to avoid stopping the cluster.
       try {
         closeClient();
         drillbitCount = newDrillbitCount;
+        if (newConfig != null) {
+          // For next test class, updated DrillConfig will be replaced by default DrillConfig in BaseTestQuery as part
+          // of the @BeforeClass method of test class.
+          config = newConfig;
+        }
         openClient();
       } catch(Exception e) {
-        throw new RuntimeException("Failure while changing the number of Drillbits in test cluster.", e);
+        throw new RuntimeException("Failure while updating the test Drillbit cluster.", e);
       }
     }
   }
@@ -118,14 +130,12 @@ public class BaseTestQuery extends ExecTest {
     return bits[0].getContext();
   }
 
-  static void resetClientAndBit() throws Exception{
+  private static void resetClientAndBit() throws Exception{
     closeClient();
     openClient();
   }
 
-  @BeforeClass
-  public static void openClient() throws Exception {
-    config = DrillConfig.create(TEST_CONFIGURATIONS);
+  private static void openClient() throws Exception {
     allocator = new TopLevelAllocator(config);
     if (config.hasPath(ENABLE_FULL_CACHE) && config.getBoolean(ENABLE_FULL_CACHE)) {
       serviceSet = RemoteServiceSet.getServiceSetWithFullCache(config, allocator);
@@ -139,7 +149,22 @@ public class BaseTestQuery extends ExecTest {
       bits[i].run();
     }
 
-    client = QueryTestUtil.createClient(config,  serviceSet, 2);
+    client = QueryTestUtil.createClient(config,  serviceSet, MAX_WIDTH_PER_NODE, null);
+  }
+
+  /**
+   * Close the current <i>client</i> and open a new client using the given <i>properties</i>. All tests executed
+   * after this method call use the new <i>client</i>.
+   *
+   * @param properties
+   */
+  public static void updateClient(Properties properties) throws Exception {
+    if (client != null) {
+      client.close();
+      client = null;
+    }
+
+    client = QueryTestUtil.createClient(config, serviceSet, MAX_WIDTH_PER_NODE, properties);
   }
 
   protected static BufferAllocator getAllocator() {

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
index 82f1752..e218d6c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
@@ -18,6 +18,7 @@
 package org.apache.drill;
 
 import java.util.List;
+import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -50,14 +51,14 @@ public class QueryTestUtil {
    * @param drillConfig
    * @param remoteServiceSet remote service set
    * @param maxWidth maximum width per node
+   * @param props Connection properties contains properties such as "user", "password", "schema" etc
    * @return the newly created client
    * @throws RpcException if there is a problem setting up the client
    */
-  public static DrillClient createClient(
-      final DrillConfig drillConfig, final RemoteServiceSet remoteServiceSet, final int maxWidth)
-      throws RpcException {
+  public static DrillClient createClient(final DrillConfig drillConfig, final RemoteServiceSet remoteServiceSet,
+      final int maxWidth, final Properties props) throws RpcException {
     final DrillClient drillClient = new DrillClient(drillConfig, remoteServiceSet.getCoordinator());
-    drillClient.connect();
+    drillClient.connect(props);
 
     final List<QueryDataBatch> results = drillClient.runQuery(
         QueryType.SQL, String.format("alter session set `%s` = %d",

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/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 08655e3..2080fce 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
@@ -109,7 +109,7 @@ public class TestLocalExchange extends PlanTestBase {
 
   @BeforeClass
   public static void setupClusterSize() {
-    setDrillbitCount(CLUSTER_SIZE);
+    updateTestCluster(CLUSTER_SIZE, null);
   }
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/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 bdb020b..46bcc60 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
@@ -148,7 +148,7 @@ public class TestPartitionSender extends PlanTestBase {
 
     final TopNBatch.SimpleRecordBatch incoming = new TopNBatch.SimpleRecordBatch(container, sv, null);
 
-    setDrillbitCount(DRILLBITS_COUNT);
+    updateTestCluster(DRILLBITS_COUNT, null);
 
     test("ALTER SESSION SET `planner.slice_target`=1");
     String plan = getPlanInString("EXPLAIN PLAN FOR " + groupByQuery, JSON_FORMAT);

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/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
new file mode 100644
index 0000000..06a2d1a
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.UserSession;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_1;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_1_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_2;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_2_PASSWORD;
+import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public class TestCustomUserAuthenticator extends BaseTestQuery {
+
+  @BeforeClass
+  public static void setupCluster() {
+    // Create a new DrillConfig which has user authentication enabled and authenticator set to
+    // UserAuthenticatorTestImpl.
+    final Properties props = new Properties();
+    props.setProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, "true");
+    props.setProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE);
+    final DrillConfig newConfig = DrillConfig.create(props);
+
+    updateTestCluster(3, newConfig);
+  }
+
+  @Test
+  public void positiveUserAuth() throws Exception {
+    runTest(TEST_USER_1, TEST_USER_1_PASSWORD);
+    runTest(TEST_USER_2, TEST_USER_2_PASSWORD);
+  }
+
+
+  @Test
+  public void negativeUserAuth() throws Exception {
+    negativeAuthHelper(TEST_USER_1, "blah.. blah..");
+    negativeAuthHelper(TEST_USER_2, "blah.. blah..");
+    negativeAuthHelper(TEST_USER_2, "");
+    negativeAuthHelper("", "blah.. blah..");
+  }
+
+  @Test
+  public void positiveUserAuthAfterNegativeUserAuth() throws Exception {
+    negativeAuthHelper("blah.. blah..", "blah.. blah..");
+    runTest(TEST_USER_2, TEST_USER_2_PASSWORD);
+  }
+
+  private static void negativeAuthHelper(final String user, final String password) throws Exception {
+    RpcException negativeAuthEx = null;
+    try {
+      runTest(user, password);
+    } catch (RpcException e) {
+      negativeAuthEx = e;
+    }
+
+    assertNotNull("Expected RpcException.", negativeAuthEx);
+    final String exMsg = negativeAuthEx.getMessage();
+    assertThat(exMsg, containsString("HANDSHAKE_VALIDATION : Status: AUTH_FAILED"));
+    assertThat(exMsg, containsString("Invalid user credentials"));
+  }
+
+  private static void runTest(final String user, final String password) throws Exception {
+    final Properties connectionProps = new Properties();
+
+    connectionProps.setProperty(UserSession.USER, user);
+    connectionProps.setProperty(UserSession.PASSWORD, password);
+
+    updateClient(connectionProps);
+
+    // Run few queries using the new client
+    test("SHOW SCHEMAS");
+    test("USE INFORMATION_SCHEMA");
+    test("SHOW TABLES");
+    test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_NAME LIKE 'COLUMNS'");
+    test("SELECT * FROM cp.`region.json` LIMIT 5");
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
new file mode 100644
index 0000000..c89471f
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user.security;
+
+import com.google.common.base.Strings;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+
+import java.io.IOException;
+
+/*
+ * Implement {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} for testing UserAuthenticator and
+ * authentication of users from Java client to Drillbit.
+ */
+@UserAuthenticatorTemplate(type = UserAuthenticatorTestImpl.TYPE)
+public class UserAuthenticatorTestImpl implements UserAuthenticator {
+  public static final String TYPE = "drillTestAuthenticator";
+
+  public static final String TEST_USER_1 = "testUser1";
+  public static final String TEST_USER_2 = "testUser2";
+  public static final String TEST_USER_1_PASSWORD = "testUser1Password";
+  public static final String TEST_USER_2_PASSWORD = "testUser2Password";
+
+  @Override
+  public void setup(DrillConfig drillConfig) throws DrillbitStartupException {
+    // Nothing to setup.
+  }
+
+  @Override
+  public void authenticate(String user, String password) throws UserAuthenticationException {
+
+    if ("anonymous".equals(user)) {
+      // Allow user "anonymous" for test framework to work.
+      return;
+    }
+
+    if (!(TEST_USER_1.equals(user) && TEST_USER_1_PASSWORD.equals(password)) &&
+        !(TEST_USER_2.equals(user) && TEST_USER_2_PASSWORD.equals(password))) {
+      throw new UserAuthenticationException();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Nothing to cleanup.
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
index 64033a5..e03098a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java
@@ -143,7 +143,7 @@ public class TestDrillbitResilience extends ExecTest {
 
     // create a client
     final DrillConfig drillConfig = zkHelper.getConfig();
-    drillClient = QueryTestUtil.createClient(drillConfig, remoteServiceSet, 1);
+    drillClient = QueryTestUtil.createClient(drillConfig, remoteServiceSet, 1, null);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/drill/blob/6a8e0ec1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8c9f09e..35a65f2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -969,6 +969,30 @@
             </exclusions>
           </dependency>
 
+          <dependency>
+            <groupId>net.sf.jpam</groupId>
+            <artifactId>jpam</artifactId>
+            <version>1.1</version>
+            <exclusions>
+              <exclusion>
+                <groupId>commons-logging</groupId>
+                <artifactId>commons-logging</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>junit</groupId>
+                <artifactId>junit</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>checkstyle</groupId>
+                <artifactId>checkstyle</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>checkstyle</groupId>
+                <artifactId>checkstyle-optional</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
+
           <!-- Test Dependencies -->
           <dependency>
             <groupId>org.apache.hadoop</groupId>


[4/9] drill git commit: DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
index 3a744fd..e85ed8b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
@@ -28,7 +28,7 @@ import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -48,12 +48,12 @@ public class TestAggregateFunction extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile(planPath), Charsets.UTF_8).replace("#{TEST_FILE}", dataPath));
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(1);
+      QueryDataBatch batch = results.get(1);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       int i = 0;
@@ -63,7 +63,7 @@ public class TestAggregateFunction extends PopUnitTestBase {
       }
 
       batchLoader.clear();
-      for(QueryResultBatch b : results) {
+      for(QueryDataBatch b : results) {
         b.release();
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
index 67aa4dd..088ca06 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -48,13 +48,13 @@ public class TestDateFunctions extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile(physicalPlan), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", resourceFile));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
 
@@ -67,7 +67,7 @@ public class TestDateFunctions extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
                 b.release();
             }
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
index 65848eb..cf5c239 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -43,7 +43,8 @@ import com.google.common.io.Files;
 
 public class TestMultiInputAdd extends PopUnitTestBase {
 
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class);
+//    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMathFunctions.class);
+
     DrillConfig c = DrillConfig.create();
 
 
@@ -57,12 +58,12 @@ public class TestMultiInputAdd extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/functions/multi_input_add_test.json"), Charsets.UTF_8));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -73,7 +74,7 @@ public class TestMultiInputAdd extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
                 b.release();
             }
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewAggregateFunctions.java
index 0238932..68e6eac 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewAggregateFunctions.java
@@ -28,7 +28,7 @@ import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -50,7 +50,7 @@ public class TestNewAggregateFunctions extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(
+      List<QueryDataBatch> results = client.runQuery(
           QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile(physicalPlan),
               Charsets.UTF_8).replace("#{TEST_FILE}",
@@ -59,7 +59,7 @@ public class TestNewAggregateFunctions extends PopUnitTestBase {
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit
           .getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(1);
+      QueryDataBatch batch = results.get(1);
       assertTrue(batchLoader.load(batch.getHeader().getDef(),
           batch.getData()));
 
@@ -71,7 +71,7 @@ public class TestNewAggregateFunctions extends PopUnitTestBase {
       }
 
       batchLoader.clear();
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         b.release();
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
index 5212125..93a9570 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -50,9 +50,9 @@ public class TestBroadcastExchange extends PopUnitTestBase {
               FileUtils.getResourceAsFile("/sender/broadcast_exchange.json"), Charsets.UTF_8)
               .replace("#{LEFT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
               .replace("#{RIGHT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString());
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -76,9 +76,9 @@ public class TestBroadcastExchange extends PopUnitTestBase {
 
       String physicalPlan = Files.toString(
           FileUtils.getResourceAsFile("/sender/broadcast_exchange_long_run.json"), Charsets.UTF_8);
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
index 0d9f014..e5448ac 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
@@ -51,7 +51,7 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -450,10 +450,10 @@ public class TestCastFunctions extends PopUnitTestBase{
       bit.run();
 
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarCharNull.json"), Charsets.UTF_8).replace("#{TEST_FILE}", "/jsoninput/input1.json"));
 
-      QueryResultBatch batch = results.get(0);
+      QueryDataBatch batch = results.get(0);
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
       batchLoader.load(batch.getHeader().getDef(), batch.getData());
@@ -477,7 +477,7 @@ public class TestCastFunctions extends PopUnitTestBase{
         }
       }
       batchLoader.clear();
-      for(QueryResultBatch b : results){
+      for(QueryDataBatch b : results){
         b.release();
       }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
index 84ac8cf..8d904eb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -38,7 +38,7 @@ import com.google.common.io.Files;
 
 
 public class TestCastVarCharToBigInt extends PopUnitTestBase {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestCastVarCharToBigInt.class);
+//    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestCastVarCharToBigInt.class);
 
     @Test
     public void testCastToBigInt() throws Exception {
@@ -49,14 +49,14 @@ public class TestCastVarCharToBigInt extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/functions/cast/test_cast_varchar_to_bigint.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/scan_json_test_cast.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -68,7 +68,7 @@ public class TestCastVarCharToBigInt extends PopUnitTestBase {
                 assertEquals(accessor.getObject(2), 2006L);
             }
 
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
             batchLoader.clear();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index fc4c0cc..7a52130 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -48,6 +48,7 @@ import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.joda.time.DateTime;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -331,10 +332,10 @@ public class TestConvertFunctions extends BaseTestQuery {
   public void testBigIntVarCharReturnTripConvertLogical() throws Exception {
     final String logicalPlan = Resources.toString(
         Resources.getResource(CONVERSION_TEST_LOGICAL_PLAN), Charsets.UTF_8);
-    final List<QueryResultBatch> results =  testLogicalWithResults(logicalPlan);
+    final List<QueryDataBatch> results =  testLogicalWithResults(logicalPlan);
     int count = 0;
     final RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
-    for (QueryResultBatch result : results) {
+    for (QueryDataBatch result : results) {
       count += result.getHeader().getRowCount();
       loader.load(result.getHeader().getDef(), result.getData());
       if (loader.getRecordCount() > 0) {
@@ -493,11 +494,11 @@ public class TestConvertFunctions extends BaseTestQuery {
   }
 
   protected Object[] getRunResult(QueryType queryType, String planString) throws Exception {
-    List<QueryResultBatch> resultList = testRunAndReturn(queryType, planString);
+    List<QueryDataBatch> resultList = testRunAndReturn(queryType, planString);
 
     List<Object> res = new ArrayList<Object>();
     RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
-    for(QueryResultBatch result : resultList) {
+    for(QueryDataBatch result : resultList) {
       if (result.getData() != null) {
         loader.load(result.getHeader().getDef(), result.getData());
         ValueVector v = loader.iterator().next().getValueVector();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
index 800f172..1424a08 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -54,14 +54,14 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/cast_simple_decimal.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(0);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String decimal9Output[] = {"99.0000", "11.1235", "0.1000", "-0.1200", "-123.1234", "-1.0001"};
@@ -82,7 +82,7 @@ public class TestDecimal extends PopUnitTestBase{
             assertEquals(6, dec18Accessor.getValueCount());
 
             batchLoader.clear();
-            for (QueryResultBatch result : results) {
+            for (QueryDataBatch result : results) {
               result.release();
             }
         }
@@ -99,14 +99,14 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/cast_float_decimal.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(0);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String decimal9Output[] = {"99.0000", "11.1235", "0.1000", "-0.1200", "-123.1234", "-1.0001"};
@@ -127,7 +127,7 @@ public class TestDecimal extends PopUnitTestBase{
             assertEquals(6, dec38Accessor.getValueCount());
 
             batchLoader.clear();
-            for (QueryResultBatch result : results) {
+            for (QueryDataBatch result : results) {
               result.release();
             }
         }
@@ -144,14 +144,14 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/simple_decimal_arithmetic.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(0);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String addOutput[] = {"123456888.0", "22.2", "0.2", "-0.2", "-987654444.2","-3.0"};
@@ -176,7 +176,7 @@ public class TestDecimal extends PopUnitTestBase{
             assertEquals(6, mulAccessor.getValueCount());
 
             batchLoader.clear();
-            for (QueryResultBatch result : results) {
+            for (QueryDataBatch result : results) {
               result.release();
             }
         }
@@ -195,14 +195,14 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_complex.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_complex_decimal.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(0);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String addOutput[] = {"-99999998877.700000000", "11.423456789", "123456789.100000000", "-0.119998000", "100000000112.423456789" , "-99999999879.907000000", "123456789123456801.300000000"};
@@ -221,7 +221,7 @@ public class TestDecimal extends PopUnitTestBase{
             assertEquals(7, subAccessor.getValueCount());
 
             batchLoader.clear();
-            for (QueryResultBatch result : results) {
+            for (QueryDataBatch result : results) {
               result.release();
             }
         }
@@ -238,14 +238,14 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_sort_complex.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_sort_complex_decimal.json")
             );
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(1);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String sortOutput[] = {"-100000000001.000000000000",
@@ -271,7 +271,7 @@ public class TestDecimal extends PopUnitTestBase{
             assertEquals(10, accessor.getValueCount());
 
             batchLoader.clear();
-            for (QueryResultBatch result : results) {
+            for (QueryDataBatch result : results) {
               result.release();
             }
         }
@@ -287,14 +287,14 @@ public class TestDecimal extends PopUnitTestBase{
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/decimal/simple_decimal_math.json"), Charsets.UTF_8)
               .replace("#{TEST_FILE}", "/input_simple_decimal.json")
       );
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(0);
+      QueryDataBatch batch = results.get(0);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
@@ -305,7 +305,7 @@ public class TestDecimal extends PopUnitTestBase{
       assertEquals(6, dec18Accessor.getValueCount());
 
       batchLoader.clear();
-      for (QueryResultBatch result : results) {
+      for (QueryDataBatch result : results) {
         result.release();
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index 3de9c63..099b7bd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -44,9 +44,9 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
     try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
       int count = 0;
-      for(QueryResultBatch b : results){
+      for(QueryDataBatch b : results){
         count += b.getHeader().getRowCount();
         b.release();
       }
@@ -64,9 +64,9 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
     try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
       int count = 0;
-      for(QueryResultBatch b : results){
+      for(QueryDataBatch b : results){
         count += b.getHeader().getRowCount();
         b.release();
       }
@@ -83,9 +83,9 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
         try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
             bit1.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile("/scan_screen_logical.json"), Charsets.UTF_8));
+            List<QueryDataBatch> results = client.runQuery(QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile("/scan_screen_logical.json"), Charsets.UTF_8));
             int count = 0;
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
                 count += b.getHeader().getRowCount();
                 b.release();
             }
@@ -103,9 +103,9 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
         bit1.run();
         bit2.run();
         client.connect();
-        List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
+        List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange_double_entry.json"), Charsets.UTF_8));
         int count = 0;
-      for(QueryResultBatch b : results){
+      for(QueryDataBatch b : results){
         count += b.getHeader().getRowCount();
         b.release();
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
index 12c1c03..5740512 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
@@ -26,7 +26,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.NullableBigIntVector;
@@ -110,7 +110,7 @@ public class TestExtractFunctions extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
         Files.toString(FileUtils.getResourceAsFile("/functions/extractFrom.json"), Charsets.UTF_8)
         .replace("#{TEST_TYPE}", fromType)
         .replace("#{TEST_FILE}", testDataFile)
@@ -118,7 +118,7 @@ public class TestExtractFunctions extends PopUnitTestBase {
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(0);
+      QueryDataBatch batch = results.get(0);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       for(int i=0; i<expectedValues.length; i++) {
@@ -130,7 +130,7 @@ public class TestExtractFunctions extends PopUnitTestBase {
         }
       }
 
-      for(QueryResultBatch b : results){
+      for(QueryDataBatch b : results){
         b.release();
       }
       batchLoader.clear();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
index 10ee46a..1d090ec 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -46,11 +46,11 @@ public class TestHashToRandomExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/hash_exchange.json"),
               Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
index 6bf23ec..ba905c4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
@@ -45,7 +45,7 @@ import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.data.DataConnectionCreator;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.BootStrapContext;
@@ -129,10 +129,10 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_filter.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
@@ -159,10 +159,10 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_join.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
@@ -189,10 +189,10 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_string_filter.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
@@ -229,10 +229,10 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_json_scan.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper vw : loader) {
@@ -269,10 +269,10 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_order_varbinary.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper vw : loader) {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
index 2880b18..76c4718 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -55,12 +55,12 @@ public class TestReverseImplicitCast extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/functions/cast/two_way_implicit_cast.json"), Charsets.UTF_8));
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(0);
+      QueryDataBatch batch = results.get(0);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
@@ -76,7 +76,7 @@ public class TestReverseImplicitCast extends PopUnitTestBase {
       }
 
       batchLoader.clear();
-      for (QueryResultBatch result : results) {
+      for (QueryDataBatch result : results) {
         result.release();
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index bd15309..09ba1a5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -31,7 +31,7 @@ import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -56,12 +56,12 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
     client.connect();
     String path = "/physical_test2.json";
 //      String path = "/filter/test1.json";
-    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(path), Charsets.UTF_8));
+    List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(path), Charsets.UTF_8));
 
     // look at records
     RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
     int recordCount = 0;
-    for (QueryResultBatch batch : results) {
+    for (QueryDataBatch batch : results) {
       boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
       boolean firstColumn = true;
 
@@ -115,7 +115,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/physical_json_scan_test1.json"), Charsets.UTF_8)
               .replace("#{TEST_FILE}", FileUtils.getResourceAsFile("/scan_json_test_1.json").toURI().toString())
       );
@@ -129,7 +129,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
       //assertEquals(expectedBatchCount, results.size());
 
       for (int i = 0; i < results.size(); ++i) {
-        QueryResultBatch batch = results.get(i);
+        QueryDataBatch batch = results.get(i);
         if (i == 0) {
           assertTrue(batch.hasData());
         } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
index 271af72..9c24f79 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -46,11 +46,11 @@ public class TestUnionExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/union_exchange.json"),
               Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
index 5a897c6..85d62c3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
@@ -29,7 +29,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.BigIntVector;
@@ -53,11 +53,11 @@ public class TestSimpleTopN extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/topN/one_key_sort.json"),
                       Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -69,7 +69,7 @@ public class TestSimpleTopN extends PopUnitTestBase {
       int recordCount = 0;
       int batchCount = 0;
 
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() == 0) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
index 6c21a28..ef3a330 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
@@ -45,7 +45,7 @@ import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -133,14 +133,14 @@ public class TestHashJoin extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/hash_join.json"), Charsets.UTF_8)
                       .replace("#{TEST_FILE_1}", FileUtils.getResourceAsFile("/build_side_input.json").toURI().toString())
                       .replace("#{TEST_FILE_2}", FileUtils.getResourceAsFile("/probe_side_input.json").toURI().toString()));
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(1);
+      QueryDataBatch batch = results.get(1);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
@@ -158,7 +158,7 @@ public class TestHashJoin extends PopUnitTestBase {
       assertEquals(6, intAccessor1.getValueCount());
 
       batchLoader.clear();
-      for (QueryResultBatch result : results) {
+      for (QueryDataBatch result : results) {
         result.release();
       }
     }
@@ -176,11 +176,11 @@ public class TestHashJoin extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/hj_exchanges.json"), Charsets.UTF_8));
 
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -204,14 +204,14 @@ public class TestHashJoin extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/hj_multi_condition_join.json"), Charsets.UTF_8)
                       .replace("#{TEST_FILE_1}", FileUtils.getResourceAsFile("/build_side_input.json").toURI().toString())
                       .replace("#{TEST_FILE_2}", FileUtils.getResourceAsFile("/probe_side_input.json").toURI().toString()));
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-      QueryResultBatch batch = results.get(1);
+      QueryDataBatch batch = results.get(1);
       assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
       Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
@@ -232,7 +232,7 @@ public class TestHashJoin extends PopUnitTestBase {
       assertEquals(3, intAccessor1.getValueCount());
 
       batchLoader.clear();
-      for (QueryResultBatch result : results) {
+      for (QueryDataBatch result : results) {
         result.release();
       }
     }
@@ -251,11 +251,11 @@ public class TestHashJoin extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/hj_exchanges1.json"), Charsets.UTF_8));
 
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -277,10 +277,10 @@ public class TestHashJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/hashJoinExpr.json"), Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index d105272..18555c7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -42,7 +42,7 @@ import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.Drillbit;
@@ -328,12 +328,12 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
                       Charsets.UTF_8)
                       .replace("${JOIN_TYPE}", "INNER"));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -352,12 +352,12 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
               Charsets.UTF_8)
               .replace("${JOIN_TYPE}", "LEFT"));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -376,12 +376,12 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
                       Charsets.UTF_8)
                       .replace("${JOIN_TYPE}", "RIGHT"));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -400,10 +400,10 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/mergeJoinExpr.json"), Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
index 996b675..34c15c9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
@@ -25,7 +25,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Rule;
@@ -55,11 +55,11 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/mj_multi_condition.json"),
               Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -80,10 +80,10 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/merge_join_nullkey.json"), Charsets.UTF_8).replace("${JOIN_TYPE}", "INNER"));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -105,10 +105,10 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/merge_join_nullkey.json"), Charsets.UTF_8).replace("${JOIN_TYPE}", "LEFT"));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index d61c123..cf9dd84 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -28,7 +28,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
@@ -50,13 +50,13 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
         Files.toString(FileUtils.getResourceAsFile("/mergerecv/merging_receiver.json"),
           Charsets.UTF_8));
       int count = 0;
       RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
       // print the results
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         count += b.getHeader().getRowCount();
         for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) {
           List<Object> row = Lists.newArrayList();
@@ -95,14 +95,14 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                                                         Files.toString(FileUtils.getResourceAsFile("/mergerecv/multiple_providers.json"),
                                                                         Charsets.UTF_8));
       int count = 0;
       RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
       // print the results
       Long lastBlueValue = null;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         count += b.getHeader().getRowCount();
         for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) {
           List<Object> row = Lists.newArrayList();
@@ -147,13 +147,13 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                                                         Files.toString(FileUtils.getResourceAsFile("/mergerecv/empty_batch.json"),
                                                                         Charsets.UTF_8));
       int count = 0;
       RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
       // print the results
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         count += b.getHeader().getRowCount();
         for (int valueIdx = 0; valueIdx < b.getHeader().getRowCount(); valueIdx++) {
           List<Object> row = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
index 27d38e6..7cd35be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
@@ -30,7 +30,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
@@ -71,12 +71,12 @@ public class TestOrderedPartitionExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/ordered_exchange.json"),
               Charsets.UTF_8));
       int count = 0;
       List<Integer> partitionRecordCounts = Lists.newArrayList();
-      for(QueryResultBatch b : results) {
+      for(QueryDataBatch b : results) {
         if (b.getData() != null) {
           int rows = b.getHeader().getRowCount();
           count += rows;

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index 669a21f..eae65ec 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -213,7 +213,7 @@ public class TestParquetWriter extends BaseTestQuery {
   @Test
   public void testMulipleRowGroups() throws Exception {
     try {
-      test(String.format("ALTER SESSION SET `%s` = %d", ExecConstants.PARQUET_BLOCK_SIZE, 1*1024*1024));
+      test(String.format("ALTER SESSION SET `%s` = %d", ExecConstants.PARQUET_BLOCK_SIZE, 1024*1024));
       String selection = "mi";
       String inputTable = "cp.`customer.json`";
       runTestAndValidate(selection, selection, inputTable, "foodmart_customer_parquet");

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
index 530883b..783abc7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
@@ -26,7 +26,7 @@ import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.apache.hadoop.conf.Configuration;
@@ -62,11 +62,11 @@ public class TestWriter extends BaseTestQuery {
 
     String plan = Files.toString(FileUtils.getResourceAsFile("/writer/simple_csv_writer.json"), Charsets.UTF_8);
 
-    List<QueryResultBatch> results = testPhysicalWithResults(plan);
+    List<QueryDataBatch> results = testPhysicalWithResults(plan);
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
 
-    QueryResultBatch batch = results.get(0);
+    QueryDataBatch batch = results.get(0);
     assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
     VarCharVector fragmentIdV = (VarCharVector) batchLoader.getValueAccessorById(VarCharVector.class, 0).getValueVector();
@@ -85,7 +85,7 @@ public class TestWriter extends BaseTestQuery {
     FileStatus[] fileStatuses = fs.globStatus(new Path(path.toString(), "*.csv"));
     assertTrue(2 == fileStatuses.length);
 
-    for (QueryResultBatch b : results) {
+    for (QueryDataBatch b : results) {
       b.release();
     }
     batchLoader.clear();
@@ -141,12 +141,12 @@ public class TestWriter extends BaseTestQuery {
       fs.delete(tableLocation, true);
     }
 
-    List<QueryResultBatch> results = testSqlWithResults(testQuery);
+    List<QueryDataBatch> results = testSqlWithResults(testQuery);
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
 
     int recordsWritten = 0;
-    for (QueryResultBatch batch : results) {
+    for (QueryDataBatch batch : results) {
       batchLoader.load(batch.getHeader().getDef(), batch.getData());
 
       if (batchLoader.getRecordCount() <= 0) {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index a96c507..b34a466 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -30,7 +30,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.BigIntVector;
@@ -52,9 +52,9 @@ public class TestSimpleExternalSort extends BaseTestQuery {
 
   @Test
   public void mergeSortWithSv2() throws Exception {
-    List<QueryResultBatch> results = testPhysicalFromFileWithResults("xsort/one_key_sort_descending_sv2.json");
+    List<QueryDataBatch> results = testPhysicalFromFileWithResults("xsort/one_key_sort_descending_sv2.json");
     int count = 0;
-    for(QueryResultBatch b : results) {
+    for(QueryDataBatch b : results) {
       if (b.getHeader().getRowCount() != 0) {
         count += b.getHeader().getRowCount();
       }
@@ -66,7 +66,7 @@ public class TestSimpleExternalSort extends BaseTestQuery {
     int recordCount = 0;
     int batchCount = 0;
 
-    for (QueryResultBatch b : results) {
+    for (QueryDataBatch b : results) {
       if (b.getHeader().getRowCount() == 0) {
         break;
       }
@@ -93,9 +93,9 @@ public class TestSimpleExternalSort extends BaseTestQuery {
 
   @Test
   public void sortOneKeyDescendingMergeSort() throws Throwable{
-    List<QueryResultBatch> results = testPhysicalFromFileWithResults("xsort/one_key_sort_descending.json");
+    List<QueryDataBatch> results = testPhysicalFromFileWithResults("xsort/one_key_sort_descending.json");
     int count = 0;
-    for (QueryResultBatch b : results) {
+    for (QueryDataBatch b : results) {
       if (b.getHeader().getRowCount() != 0) {
         count += b.getHeader().getRowCount();
       }
@@ -107,7 +107,7 @@ public class TestSimpleExternalSort extends BaseTestQuery {
     int recordCount = 0;
     int batchCount = 0;
 
-    for (QueryResultBatch b : results) {
+    for (QueryDataBatch b : results) {
       if (b.getHeader().getRowCount() == 0) {
         break;
       }
@@ -144,11 +144,11 @@ public class TestSimpleExternalSort extends BaseTestQuery {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/xsort/one_key_sort_descending.json"),
                       Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -160,7 +160,7 @@ public class TestSimpleExternalSort extends BaseTestQuery {
       int recordCount = 0;
       int batchCount = 0;
 
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() == 0) {
           break;
         }
@@ -196,11 +196,11 @@ public class TestSimpleExternalSort extends BaseTestQuery {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+      List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/xsort/oom_sort_test.json"),
                       Charsets.UTF_8));
       int count = 0;
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() != 0) {
           count += b.getHeader().getRowCount();
         }
@@ -212,7 +212,7 @@ public class TestSimpleExternalSort extends BaseTestQuery {
       int recordCount = 0;
       int batchCount = 0;
 
-      for (QueryResultBatch b : results) {
+      for (QueryDataBatch b : results) {
         if (b.getHeader().getRowCount() == 0) {
           break;
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
index 4cc82e4..a2401e8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
@@ -28,7 +28,7 @@ import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.vector.ValueVector;
@@ -42,7 +42,7 @@ import com.google.common.io.Files;
  * back to VarChar.
  */
 public class TestDateTypes extends PopUnitTestBase {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDateTypes.class);
+//    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDateTypes.class);
 
     @Test
     public void testDate() throws Exception {
@@ -53,13 +53,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_date.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -72,7 +72,7 @@ public class TestDateTypes extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }
@@ -87,13 +87,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_sort_date.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(1);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -106,7 +106,7 @@ public class TestDateTypes extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }
@@ -121,13 +121,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_timestamp.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -140,7 +140,7 @@ public class TestDateTypes extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }
@@ -155,13 +155,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_interval.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_interval.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(0);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             Iterator<VectorWrapper<?>> itr = batchLoader.iterator();
@@ -198,7 +198,7 @@ public class TestDateTypes extends PopUnitTestBase {
             assertEquals((accessor.getObject(5).toString()), ("1 day 0:-39:-25.0"));
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }
@@ -213,13 +213,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_all_date_literals.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             String result[] = {"2008-02-27",
@@ -237,7 +237,7 @@ public class TestDateTypes extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }
@@ -252,13 +252,13 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
+            List<QueryDataBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_date_add.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
 
-            QueryResultBatch batch = results.get(1);
+            QueryDataBatch batch = results.get(0);
             assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
 
             for (VectorWrapper<?> v : batchLoader) {
@@ -271,7 +271,7 @@ public class TestDateTypes extends PopUnitTestBase {
             }
 
             batchLoader.clear();
-            for(QueryResultBatch b : results){
+            for(QueryDataBatch b : results){
               b.release();
             }
         }


[8/9] drill git commit: DRILL-2573: C++ Client - Separate QueryResult into QueryResult and QueryData

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/UserBitShared.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index 7c237f6..b07ecda 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -56,6 +56,9 @@ const ::google::protobuf::Descriptor* QueryResult_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   QueryResult_reflection_ = NULL;
 const ::google::protobuf::EnumDescriptor* QueryResult_QueryState_descriptor_ = NULL;
+const ::google::protobuf::Descriptor* QueryData_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  QueryData_reflection_ = NULL;
 const ::google::protobuf::Descriptor* QueryInfo_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   QueryInfo_reflection_ = NULL;
@@ -270,18 +273,10 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(NodeStatus));
   QueryResult_descriptor_ = file->message_type(10);
-  static const int QueryResult_offsets_[11] = {
+  static const int QueryResult_offsets_[3] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, query_state_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, query_id_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, is_last_chunk_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, row_count_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, records_scan_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, records_error_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, submission_time_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, node_status_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, error_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, def_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryResult, schema_changed_),
   };
   QueryResult_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -295,7 +290,24 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(QueryResult));
   QueryResult_QueryState_descriptor_ = QueryResult_descriptor_->enum_type(0);
-  QueryInfo_descriptor_ = file->message_type(11);
+  QueryData_descriptor_ = file->message_type(11);
+  static const int QueryData_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, query_id_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, row_count_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, def_),
+  };
+  QueryData_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      QueryData_descriptor_,
+      QueryData::default_instance_,
+      QueryData_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(QueryData));
+  QueryInfo_descriptor_ = file->message_type(12);
   static const int QueryInfo_offsets_[5] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryInfo, query_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryInfo, start_),
@@ -314,7 +326,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(QueryInfo));
-  QueryProfile_descriptor_ = file->message_type(12);
+  QueryProfile_descriptor_ = file->message_type(13);
   static const int QueryProfile_offsets_[11] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryProfile, type_),
@@ -339,7 +351,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(QueryProfile));
-  MajorFragmentProfile_descriptor_ = file->message_type(13);
+  MajorFragmentProfile_descriptor_ = file->message_type(14);
   static const int MajorFragmentProfile_offsets_[2] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MajorFragmentProfile, major_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MajorFragmentProfile, minor_fragment_profile_),
@@ -355,7 +367,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(MajorFragmentProfile));
-  MinorFragmentProfile_descriptor_ = file->message_type(14);
+  MinorFragmentProfile_descriptor_ = file->message_type(15);
   static const int MinorFragmentProfile_offsets_[9] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MinorFragmentProfile, state_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MinorFragmentProfile, error_),
@@ -378,14 +390,14 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(MinorFragmentProfile));
-  OperatorProfile_descriptor_ = file->message_type(15);
+  OperatorProfile_descriptor_ = file->message_type(16);
   static const int OperatorProfile_offsets_[8] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, input_profile_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, operator_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, operator_type_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, setup_nanos_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, process_nanos_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, local_memory_allocated_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, peak_local_memory_allocated_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, metric_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(OperatorProfile, wait_nanos_),
   };
@@ -400,7 +412,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(OperatorProfile));
-  StreamProfile_descriptor_ = file->message_type(16);
+  StreamProfile_descriptor_ = file->message_type(17);
   static const int StreamProfile_offsets_[3] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(StreamProfile, records_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(StreamProfile, batches_),
@@ -417,7 +429,7 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(StreamProfile));
-  MetricValue_descriptor_ = file->message_type(17);
+  MetricValue_descriptor_ = file->message_type(18);
   static const int MetricValue_offsets_[3] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MetricValue, metric_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(MetricValue, long_value_),
@@ -473,6 +485,8 @@ void protobuf_RegisterTypes(const ::std::string&) {
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     QueryResult_descriptor_, &QueryResult::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    QueryData_descriptor_, &QueryData::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     QueryInfo_descriptor_, &QueryInfo::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     QueryProfile_descriptor_, &QueryProfile::default_instance());
@@ -513,6 +527,8 @@ void protobuf_ShutdownFile_UserBitShared_2eproto() {
   delete NodeStatus_reflection_;
   delete QueryResult::default_instance_;
   delete QueryResult_reflection_;
+  delete QueryData::default_instance_;
+  delete QueryData_reflection_;
   delete QueryInfo::default_instance_;
   delete QueryInfo_reflection_;
   delete QueryProfile::default_instance_;
@@ -572,76 +588,74 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "t\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\023\n\013grou"
     "p_count\030\006 \001(\005\022\025\n\rbuffer_length\030\007 \001(\005\"7\n\n"
     "NodeStatus\022\017\n\007node_id\030\001 \001(\005\022\030\n\020memory_fo"
-    "otprint\030\002 \001(\003\"\335\003\n\013QueryResult\0228\n\013query_s"
+    "otprint\030\002 \001(\003\"\206\002\n\013QueryResult\0228\n\013query_s"
     "tate\030\001 \001(\0162#.exec.shared.QueryResult.Que"
     "ryState\022&\n\010query_id\030\002 \001(\0132\024.exec.shared."
-    "QueryId\022\025\n\ris_last_chunk\030\003 \001(\010\022\021\n\trow_co"
-    "unt\030\004 \001(\005\022\024\n\014records_scan\030\005 \001(\003\022\025\n\rrecor"
-    "ds_error\030\006 \001(\003\022\027\n\017submission_time\030\007 \001(\003\022"
-    ",\n\013node_status\030\010 \003(\0132\027.exec.shared.NodeS"
-    "tatus\022(\n\005error\030\t \003(\0132\031.exec.shared.Drill"
-    "PBError\022(\n\003def\030\n \001(\0132\033.exec.shared.Recor"
-    "dBatchDef\022\026\n\016schema_changed\030\013 \001(\010\"b\n\nQue"
-    "ryState\022\013\n\007PENDING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOM"
-    "PLETED\020\002\022\014\n\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\021\n\rUN"
-    "KNOWN_QUERY\020\005\"\224\001\n\tQueryInfo\022\r\n\005query\030\001 \001"
-    "(\t\022\r\n\005start\030\002 \001(\003\0222\n\005state\030\003 \001(\0162#.exec."
-    "shared.QueryResult.QueryState\022\014\n\004user\030\004 "
-    "\001(\t\022\'\n\007foreman\030\005 \001(\0132\026.exec.DrillbitEndp"
-    "oint\"\336\002\n\014QueryProfile\022 \n\002id\030\001 \001(\0132\024.exec"
-    ".shared.QueryId\022$\n\004type\030\002 \001(\0162\026.exec.sha"
-    "red.QueryType\022\r\n\005start\030\003 \001(\003\022\013\n\003end\030\004 \001("
-    "\003\022\r\n\005query\030\005 \001(\t\022\014\n\004plan\030\006 \001(\t\022\'\n\007forema"
-    "n\030\007 \001(\0132\026.exec.DrillbitEndpoint\0222\n\005state"
-    "\030\010 \001(\0162#.exec.shared.QueryResult.QuerySt"
-    "ate\022\027\n\017total_fragments\030\t \001(\005\022\032\n\022finished"
-    "_fragments\030\n \001(\005\022;\n\020fragment_profile\030\013 \003"
-    "(\0132!.exec.shared.MajorFragmentProfile\"t\n"
-    "\024MajorFragmentProfile\022\031\n\021major_fragment_"
-    "id\030\001 \001(\005\022A\n\026minor_fragment_profile\030\002 \003(\013"
-    "2!.exec.shared.MinorFragmentProfile\"\274\002\n\024"
-    "MinorFragmentProfile\022)\n\005state\030\001 \001(\0162\032.ex"
-    "ec.shared.FragmentState\022(\n\005error\030\002 \001(\0132\031"
-    ".exec.shared.DrillPBError\022\031\n\021minor_fragm"
-    "ent_id\030\003 \001(\005\0226\n\020operator_profile\030\004 \003(\0132\034"
-    ".exec.shared.OperatorProfile\022\022\n\nstart_ti"
-    "me\030\005 \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memory_use"
-    "d\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n\010endp"
-    "oint\030\t \001(\0132\026.exec.DrillbitEndpoint\"\372\001\n\017O"
-    "peratorProfile\0221\n\rinput_profile\030\001 \003(\0132\032."
-    "exec.shared.StreamProfile\022\023\n\013operator_id"
-    "\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022\023\n\013setup_n"
-    "anos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 \001(\003\022\036\n\026loc"
-    "al_memory_allocated\030\007 \001(\003\022(\n\006metric\030\010 \003("
-    "\0132\030.exec.shared.MetricValue\022\022\n\nwait_nano"
-    "s\030\t \001(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001("
-    "\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013M"
-    "etricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_va"
-    "lue\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001*5\n\nRpcCh"
-    "annel\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004"
-    "USER\020\002*/\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020"
-    "\002\022\014\n\010PHYSICAL\020\003*k\n\rFragmentState\022\013\n\007SEND"
-    "ING\020\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNIN"
-    "G\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAIL"
-    "ED\020\005*\264\005\n\020CoreOperatorType\022\021\n\rSINGLE_SEND"
-    "ER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER\020\002\022\022"
-    "\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMER"
-    "GE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006\022\t\n\005"
-    "LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030ORDERED"
-    "_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNOR"
-    "DERED_RECEIVER\020\013\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SC"
-    "REEN\020\r\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023"
-    "STREAMING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n"
-    "\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014"
-    "\n\010OLD_SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025"
-    "\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN"
-    "\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020"
-    "\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022"
-    "\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n"
-    "\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JS"
-    "ON\020\037\022\025\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB"
-    "_SCAN\020!\022\n\n\006WINDOW\020\"B.\n\033org.apache.drill."
-    "exec.protoB\rUserBitSharedH\001", 4107);
+    "QueryId\022(\n\005error\030\003 \003(\0132\031.exec.shared.Dri"
+    "llPBError\"k\n\nQueryState\022\013\n\007PENDING\020\000\022\013\n\007"
+    "RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED\020\003\022\n"
+    "\n\006FAILED\020\004\022\032\n\026CANCELLATION_REQUESTED\020\005\"p"
+    "\n\tQueryData\022&\n\010query_id\030\001 \001(\0132\024.exec.sha"
+    "red.QueryId\022\021\n\trow_count\030\002 \001(\005\022(\n\003def\030\003 "
+    "\001(\0132\033.exec.shared.RecordBatchDef\"\224\001\n\tQue"
+    "ryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222\n\005"
+    "state\030\003 \001(\0162#.exec.shared.QueryResult.Qu"
+    "eryState\022\014\n\004user\030\004 \001(\t\022\'\n\007foreman\030\005 \001(\0132"
+    "\026.exec.DrillbitEndpoint\"\336\002\n\014QueryProfile"
+    "\022 \n\002id\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004ty"
+    "pe\030\002 \001(\0162\026.exec.shared.QueryType\022\r\n\005star"
+    "t\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n\004p"
+    "lan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.Drillb"
+    "itEndpoint\0222\n\005state\030\010 \001(\0162#.exec.shared."
+    "QueryResult.QueryState\022\027\n\017total_fragment"
+    "s\030\t \001(\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n\020f"
+    "ragment_profile\030\013 \003(\0132!.exec.shared.Majo"
+    "rFragmentProfile\"t\n\024MajorFragmentProfile"
+    "\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026minor_fra"
+    "gment_profile\030\002 \003(\0132!.exec.shared.MinorF"
+    "ragmentProfile\"\274\002\n\024MinorFragmentProfile\022"
+    ")\n\005state\030\001 \001(\0162\032.exec.shared.FragmentSta"
+    "te\022(\n\005error\030\002 \001(\0132\031.exec.shared.DrillPBE"
+    "rror\022\031\n\021minor_fragment_id\030\003 \001(\005\0226\n\020opera"
+    "tor_profile\030\004 \003(\0132\034.exec.shared.Operator"
+    "Profile\022\022\n\nstart_time\030\005 \001(\003\022\020\n\010end_time\030"
+    "\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017max_memory"
+    "_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.exec.Dri"
+    "llbitEndpoint\"\377\001\n\017OperatorProfile\0221\n\rinp"
+    "ut_profile\030\001 \003(\0132\032.exec.shared.StreamPro"
+    "file\022\023\n\013operator_id\030\003 \001(\005\022\025\n\roperator_ty"
+    "pe\030\004 \001(\005\022\023\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess"
+    "_nanos\030\006 \001(\003\022#\n\033peak_local_memory_alloca"
+    "ted\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132\030.exec.shared."
+    "MetricValue\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStrea"
+    "mProfile\022\017\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001"
+    "(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tme"
+    "tric_id\030\001 \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014dou"
+    "ble_value\030\003 \001(\001*5\n\nRpcChannel\022\017\n\013BIT_CON"
+    "TROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*/\n\tQueryT"
+    "ype\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*"
+    "k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAITIN"
+    "G_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020"
+    "\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005*\264\005\n\020CoreOpe"
+    "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS"
+    "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE"
+    "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS"
+    "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI"
+    "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE"
+    "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013"
+    "\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECT"
+    "ION_VECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREG"
+    "ATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021"
+    "\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026"
+    "PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCA"
+    "N\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_S"
+    "CAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_"
+    "SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN"
+    "\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB"
+    "_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER"
+    "_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDO"
+    "W\020\"B.\n\033org.apache.drill.exec.protoB\rUser"
+    "BitSharedH\001", 4011);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -655,6 +669,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
   SerializedField::default_instance_ = new SerializedField();
   NodeStatus::default_instance_ = new NodeStatus();
   QueryResult::default_instance_ = new QueryResult();
+  QueryData::default_instance_ = new QueryData();
   QueryInfo::default_instance_ = new QueryInfo();
   QueryProfile::default_instance_ = new QueryProfile();
   MajorFragmentProfile::default_instance_ = new MajorFragmentProfile();
@@ -673,6 +688,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
   SerializedField::default_instance_->InitAsDefaultInstance();
   NodeStatus::default_instance_->InitAsDefaultInstance();
   QueryResult::default_instance_->InitAsDefaultInstance();
+  QueryData::default_instance_->InitAsDefaultInstance();
   QueryInfo::default_instance_->InitAsDefaultInstance();
   QueryProfile::default_instance_->InitAsDefaultInstance();
   MajorFragmentProfile::default_instance_->InitAsDefaultInstance();
@@ -4173,7 +4189,7 @@ const QueryResult_QueryState QueryResult::RUNNING;
 const QueryResult_QueryState QueryResult::COMPLETED;
 const QueryResult_QueryState QueryResult::CANCELED;
 const QueryResult_QueryState QueryResult::FAILED;
-const QueryResult_QueryState QueryResult::UNKNOWN_QUERY;
+const QueryResult_QueryState QueryResult::CANCELLATION_REQUESTED;
 const QueryResult_QueryState QueryResult::QueryState_MIN;
 const QueryResult_QueryState QueryResult::QueryState_MAX;
 const int QueryResult::QueryState_ARRAYSIZE;
@@ -4181,15 +4197,7 @@ const int QueryResult::QueryState_ARRAYSIZE;
 #ifndef _MSC_VER
 const int QueryResult::kQueryStateFieldNumber;
 const int QueryResult::kQueryIdFieldNumber;
-const int QueryResult::kIsLastChunkFieldNumber;
-const int QueryResult::kRowCountFieldNumber;
-const int QueryResult::kRecordsScanFieldNumber;
-const int QueryResult::kRecordsErrorFieldNumber;
-const int QueryResult::kSubmissionTimeFieldNumber;
-const int QueryResult::kNodeStatusFieldNumber;
 const int QueryResult::kErrorFieldNumber;
-const int QueryResult::kDefFieldNumber;
-const int QueryResult::kSchemaChangedFieldNumber;
 #endif  // !_MSC_VER
 
 QueryResult::QueryResult()
@@ -4199,7 +4207,6 @@ QueryResult::QueryResult()
 
 void QueryResult::InitAsDefaultInstance() {
   query_id_ = const_cast< ::exec::shared::QueryId*>(&::exec::shared::QueryId::default_instance());
-  def_ = const_cast< ::exec::shared::RecordBatchDef*>(&::exec::shared::RecordBatchDef::default_instance());
 }
 
 QueryResult::QueryResult(const QueryResult& from)
@@ -4212,13 +4219,6 @@ void QueryResult::SharedCtor() {
   _cached_size_ = 0;
   query_state_ = 0;
   query_id_ = NULL;
-  is_last_chunk_ = false;
-  row_count_ = 0;
-  records_scan_ = GOOGLE_LONGLONG(0);
-  records_error_ = GOOGLE_LONGLONG(0);
-  submission_time_ = GOOGLE_LONGLONG(0);
-  def_ = NULL;
-  schema_changed_ = false;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -4229,7 +4229,6 @@ QueryResult::~QueryResult() {
 void QueryResult::SharedDtor() {
   if (this != default_instance_) {
     delete query_id_;
-    delete def_;
   }
 }
 
@@ -4260,19 +4259,7 @@ void QueryResult::Clear() {
     if (has_query_id()) {
       if (query_id_ != NULL) query_id_->::exec::shared::QueryId::Clear();
     }
-    is_last_chunk_ = false;
-    row_count_ = 0;
-    records_scan_ = GOOGLE_LONGLONG(0);
-    records_error_ = GOOGLE_LONGLONG(0);
-    submission_time_ = GOOGLE_LONGLONG(0);
   }
-  if (_has_bits_[9 / 32] & (0xffu << (9 % 32))) {
-    if (has_def()) {
-      if (def_ != NULL) def_->::exec::shared::RecordBatchDef::Clear();
-    }
-    schema_changed_ = false;
-  }
-  node_status_.Clear();
   error_.Clear();
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
@@ -4314,122 +4301,311 @@ bool QueryResult::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(24)) goto parse_is_last_chunk;
+        if (input->ExpectTag(26)) goto parse_error;
         break;
       }
 
-      // optional bool is_last_chunk = 3;
+      // repeated .exec.shared.DrillPBError error = 3;
       case 3: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_is_last_chunk:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
-                 input, &is_last_chunk_)));
-          set_has_is_last_chunk();
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_error:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+                input, add_error()));
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(32)) goto parse_row_count;
+        if (input->ExpectTag(26)) goto parse_error;
+        if (input->ExpectAtEnd()) return true;
         break;
       }
 
-      // optional int32 row_count = 4;
-      case 4: {
+      default: {
+      handle_uninterpreted:
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_row_count:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
-                 input, &row_count_)));
-          set_has_row_count();
-        } else {
-          goto handle_uninterpreted;
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+          return true;
         }
-        if (input->ExpectTag(40)) goto parse_records_scan;
+        DO_(::google::protobuf::internal::WireFormat::SkipField(
+              input, tag, mutable_unknown_fields()));
         break;
       }
+    }
+  }
+  return true;
+#undef DO_
+}
 
-      // optional int64 records_scan = 5;
-      case 5: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_records_scan:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>(
-                 input, &records_scan_)));
-          set_has_records_scan();
-        } else {
-          goto handle_uninterpreted;
-        }
-        if (input->ExpectTag(48)) goto parse_records_error;
-        break;
-      }
+void QueryResult::SerializeWithCachedSizes(
+    ::google::protobuf::io::CodedOutputStream* output) const {
+  // optional .exec.shared.QueryResult.QueryState query_state = 1;
+  if (has_query_state()) {
+    ::google::protobuf::internal::WireFormatLite::WriteEnum(
+      1, this->query_state(), output);
+  }
 
-      // optional int64 records_error = 6;
-      case 6: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_records_error:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>(
-                 input, &records_error_)));
-          set_has_records_error();
-        } else {
-          goto handle_uninterpreted;
-        }
-        if (input->ExpectTag(56)) goto parse_submission_time;
-        break;
-      }
+  // optional .exec.shared.QueryId query_id = 2;
+  if (has_query_id()) {
+    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
+      2, this->query_id(), output);
+  }
 
-      // optional int64 submission_time = 7;
-      case 7: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_submission_time:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>(
-                 input, &submission_time_)));
-          set_has_submission_time();
-        } else {
-          goto handle_uninterpreted;
-        }
-        if (input->ExpectTag(66)) goto parse_node_status;
-        break;
-      }
+  // repeated .exec.shared.DrillPBError error = 3;
+  for (int i = 0; i < this->error_size(); i++) {
+    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
+      3, this->error(i), output);
+  }
 
-      // repeated .exec.shared.NodeStatus node_status = 8;
-      case 8: {
+  if (!unknown_fields().empty()) {
+    ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
+        unknown_fields(), output);
+  }
+}
+
+::google::protobuf::uint8* QueryResult::SerializeWithCachedSizesToArray(
+    ::google::protobuf::uint8* target) const {
+  // optional .exec.shared.QueryResult.QueryState query_state = 1;
+  if (has_query_state()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteEnumToArray(
+      1, this->query_state(), target);
+  }
+
+  // optional .exec.shared.QueryId query_id = 2;
+  if (has_query_id()) {
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteMessageNoVirtualToArray(
+        2, this->query_id(), target);
+  }
+
+  // repeated .exec.shared.DrillPBError error = 3;
+  for (int i = 0; i < this->error_size(); i++) {
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteMessageNoVirtualToArray(
+        3, this->error(i), target);
+  }
+
+  if (!unknown_fields().empty()) {
+    target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
+        unknown_fields(), target);
+  }
+  return target;
+}
+
+int QueryResult::ByteSize() const {
+  int total_size = 0;
+
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    // optional .exec.shared.QueryResult.QueryState query_state = 1;
+    if (has_query_state()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::EnumSize(this->query_state());
+    }
+
+    // optional .exec.shared.QueryId query_id = 2;
+    if (has_query_id()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+          this->query_id());
+    }
+
+  }
+  // repeated .exec.shared.DrillPBError error = 3;
+  total_size += 1 * this->error_size();
+  for (int i = 0; i < this->error_size(); i++) {
+    total_size +=
+      ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+        this->error(i));
+  }
+
+  if (!unknown_fields().empty()) {
+    total_size +=
+      ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
+        unknown_fields());
+  }
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = total_size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+  return total_size;
+}
+
+void QueryResult::MergeFrom(const ::google::protobuf::Message& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  const QueryResult* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const QueryResult*>(
+      &from);
+  if (source == NULL) {
+    ::google::protobuf::internal::ReflectionOps::Merge(from, this);
+  } else {
+    MergeFrom(*source);
+  }
+}
+
+void QueryResult::MergeFrom(const QueryResult& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  error_.MergeFrom(from.error_);
+  if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (from.has_query_state()) {
+      set_query_state(from.query_state());
+    }
+    if (from.has_query_id()) {
+      mutable_query_id()->::exec::shared::QueryId::MergeFrom(from.query_id());
+    }
+  }
+  mutable_unknown_fields()->MergeFrom(from.unknown_fields());
+}
+
+void QueryResult::CopyFrom(const ::google::protobuf::Message& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+void QueryResult::CopyFrom(const QueryResult& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+bool QueryResult::IsInitialized() const {
+
+  return true;
+}
+
+void QueryResult::Swap(QueryResult* other) {
+  if (other != this) {
+    std::swap(query_state_, other->query_state_);
+    std::swap(query_id_, other->query_id_);
+    error_.Swap(&other->error_);
+    std::swap(_has_bits_[0], other->_has_bits_[0]);
+    _unknown_fields_.Swap(&other->_unknown_fields_);
+    std::swap(_cached_size_, other->_cached_size_);
+  }
+}
+
+::google::protobuf::Metadata QueryResult::GetMetadata() const {
+  protobuf_AssignDescriptorsOnce();
+  ::google::protobuf::Metadata metadata;
+  metadata.descriptor = QueryResult_descriptor_;
+  metadata.reflection = QueryResult_reflection_;
+  return metadata;
+}
+
+
+// ===================================================================
+
+#ifndef _MSC_VER
+const int QueryData::kQueryIdFieldNumber;
+const int QueryData::kRowCountFieldNumber;
+const int QueryData::kDefFieldNumber;
+#endif  // !_MSC_VER
+
+QueryData::QueryData()
+  : ::google::protobuf::Message() {
+  SharedCtor();
+}
+
+void QueryData::InitAsDefaultInstance() {
+  query_id_ = const_cast< ::exec::shared::QueryId*>(&::exec::shared::QueryId::default_instance());
+  def_ = const_cast< ::exec::shared::RecordBatchDef*>(&::exec::shared::RecordBatchDef::default_instance());
+}
+
+QueryData::QueryData(const QueryData& from)
+  : ::google::protobuf::Message() {
+  SharedCtor();
+  MergeFrom(from);
+}
+
+void QueryData::SharedCtor() {
+  _cached_size_ = 0;
+  query_id_ = NULL;
+  row_count_ = 0;
+  def_ = NULL;
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+QueryData::~QueryData() {
+  SharedDtor();
+}
+
+void QueryData::SharedDtor() {
+  if (this != default_instance_) {
+    delete query_id_;
+    delete def_;
+  }
+}
+
+void QueryData::SetCachedSize(int size) const {
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const ::google::protobuf::Descriptor* QueryData::descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return QueryData_descriptor_;
+}
+
+const QueryData& QueryData::default_instance() {
+  if (default_instance_ == NULL) protobuf_AddDesc_UserBitShared_2eproto();
+  return *default_instance_;
+}
+
+QueryData* QueryData::default_instance_ = NULL;
+
+QueryData* QueryData::New() const {
+  return new QueryData;
+}
+
+void QueryData::Clear() {
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (has_query_id()) {
+      if (query_id_ != NULL) query_id_->::exec::shared::QueryId::Clear();
+    }
+    row_count_ = 0;
+    if (has_def()) {
+      if (def_ != NULL) def_->::exec::shared::RecordBatchDef::Clear();
+    }
+  }
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+  mutable_unknown_fields()->Clear();
+}
+
+bool QueryData::MergePartialFromCodedStream(
+    ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) return false
+  ::google::protobuf::uint32 tag;
+  while ((tag = input->ReadTag()) != 0) {
+    switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+      // optional .exec.shared.QueryId query_id = 1;
+      case 1: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
-         parse_node_status:
           DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-                input, add_node_status()));
+               input, mutable_query_id()));
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(66)) goto parse_node_status;
-        if (input->ExpectTag(74)) goto parse_error;
+        if (input->ExpectTag(16)) goto parse_row_count;
         break;
       }
 
-      // repeated .exec.shared.DrillPBError error = 9;
-      case 9: {
+      // optional int32 row_count = 2;
+      case 2: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
-         parse_error:
-          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-                input, add_error()));
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_row_count:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &row_count_)));
+          set_has_row_count();
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(74)) goto parse_error;
-        if (input->ExpectTag(82)) goto parse_def;
+        if (input->ExpectTag(26)) goto parse_def;
         break;
       }
 
-      // optional .exec.shared.RecordBatchDef def = 10;
-      case 10: {
+      // optional .exec.shared.RecordBatchDef def = 3;
+      case 3: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
          parse_def:
@@ -4438,22 +4614,6 @@ bool QueryResult::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(88)) goto parse_schema_changed;
-        break;
-      }
-
-      // optional bool schema_changed = 11;
-      case 11: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_schema_changed:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
-                 input, &schema_changed_)));
-          set_has_schema_changed();
-        } else {
-          goto handle_uninterpreted;
-        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -4474,66 +4634,23 @@ bool QueryResult::MergePartialFromCodedStream(
 #undef DO_
 }
 
-void QueryResult::SerializeWithCachedSizes(
+void QueryData::SerializeWithCachedSizes(
     ::google::protobuf::io::CodedOutputStream* output) const {
-  // optional .exec.shared.QueryResult.QueryState query_state = 1;
-  if (has_query_state()) {
-    ::google::protobuf::internal::WireFormatLite::WriteEnum(
-      1, this->query_state(), output);
-  }
-
-  // optional .exec.shared.QueryId query_id = 2;
+  // optional .exec.shared.QueryId query_id = 1;
   if (has_query_id()) {
     ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      2, this->query_id(), output);
-  }
-
-  // optional bool is_last_chunk = 3;
-  if (has_is_last_chunk()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(3, this->is_last_chunk(), output);
+      1, this->query_id(), output);
   }
 
-  // optional int32 row_count = 4;
+  // optional int32 row_count = 2;
   if (has_row_count()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt32(4, this->row_count(), output);
-  }
-
-  // optional int64 records_scan = 5;
-  if (has_records_scan()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt64(5, this->records_scan(), output);
-  }
-
-  // optional int64 records_error = 6;
-  if (has_records_error()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt64(6, this->records_error(), output);
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(2, this->row_count(), output);
   }
 
-  // optional int64 submission_time = 7;
-  if (has_submission_time()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt64(7, this->submission_time(), output);
-  }
-
-  // repeated .exec.shared.NodeStatus node_status = 8;
-  for (int i = 0; i < this->node_status_size(); i++) {
-    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      8, this->node_status(i), output);
-  }
-
-  // repeated .exec.shared.DrillPBError error = 9;
-  for (int i = 0; i < this->error_size(); i++) {
-    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      9, this->error(i), output);
-  }
-
-  // optional .exec.shared.RecordBatchDef def = 10;
+  // optional .exec.shared.RecordBatchDef def = 3;
   if (has_def()) {
     ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      10, this->def(), output);
-  }
-
-  // optional bool schema_changed = 11;
-  if (has_schema_changed()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(11, this->schema_changed(), output);
+      3, this->def(), output);
   }
 
   if (!unknown_fields().empty()) {
@@ -4542,70 +4659,25 @@ void QueryResult::SerializeWithCachedSizes(
   }
 }
 
-::google::protobuf::uint8* QueryResult::SerializeWithCachedSizesToArray(
+::google::protobuf::uint8* QueryData::SerializeWithCachedSizesToArray(
     ::google::protobuf::uint8* target) const {
-  // optional .exec.shared.QueryResult.QueryState query_state = 1;
-  if (has_query_state()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteEnumToArray(
-      1, this->query_state(), target);
-  }
-
-  // optional .exec.shared.QueryId query_id = 2;
+  // optional .exec.shared.QueryId query_id = 1;
   if (has_query_id()) {
     target = ::google::protobuf::internal::WireFormatLite::
       WriteMessageNoVirtualToArray(
-        2, this->query_id(), target);
+        1, this->query_id(), target);
   }
 
-  // optional bool is_last_chunk = 3;
-  if (has_is_last_chunk()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(3, this->is_last_chunk(), target);
-  }
-
-  // optional int32 row_count = 4;
+  // optional int32 row_count = 2;
   if (has_row_count()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(4, this->row_count(), target);
-  }
-
-  // optional int64 records_scan = 5;
-  if (has_records_scan()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(5, this->records_scan(), target);
-  }
-
-  // optional int64 records_error = 6;
-  if (has_records_error()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(6, this->records_error(), target);
-  }
-
-  // optional int64 submission_time = 7;
-  if (has_submission_time()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(7, this->submission_time(), target);
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(2, this->row_count(), target);
   }
 
-  // repeated .exec.shared.NodeStatus node_status = 8;
-  for (int i = 0; i < this->node_status_size(); i++) {
-    target = ::google::protobuf::internal::WireFormatLite::
-      WriteMessageNoVirtualToArray(
-        8, this->node_status(i), target);
-  }
-
-  // repeated .exec.shared.DrillPBError error = 9;
-  for (int i = 0; i < this->error_size(); i++) {
-    target = ::google::protobuf::internal::WireFormatLite::
-      WriteMessageNoVirtualToArray(
-        9, this->error(i), target);
-  }
-
-  // optional .exec.shared.RecordBatchDef def = 10;
+  // optional .exec.shared.RecordBatchDef def = 3;
   if (has_def()) {
     target = ::google::protobuf::internal::WireFormatLite::
       WriteMessageNoVirtualToArray(
-        10, this->def(), target);
-  }
-
-  // optional bool schema_changed = 11;
-  if (has_schema_changed()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(11, this->schema_changed(), target);
+        3, this->def(), target);
   }
 
   if (!unknown_fields().empty()) {
@@ -4615,87 +4687,32 @@ void QueryResult::SerializeWithCachedSizes(
   return target;
 }
 
-int QueryResult::ByteSize() const {
+int QueryData::ByteSize() const {
   int total_size = 0;
 
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    // optional .exec.shared.QueryResult.QueryState query_state = 1;
-    if (has_query_state()) {
-      total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::EnumSize(this->query_state());
-    }
-
-    // optional .exec.shared.QueryId query_id = 2;
+    // optional .exec.shared.QueryId query_id = 1;
     if (has_query_id()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
           this->query_id());
     }
 
-    // optional bool is_last_chunk = 3;
-    if (has_is_last_chunk()) {
-      total_size += 1 + 1;
-    }
-
-    // optional int32 row_count = 4;
+    // optional int32 row_count = 2;
     if (has_row_count()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::Int32Size(
           this->row_count());
     }
 
-    // optional int64 records_scan = 5;
-    if (has_records_scan()) {
-      total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::Int64Size(
-          this->records_scan());
-    }
-
-    // optional int64 records_error = 6;
-    if (has_records_error()) {
-      total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::Int64Size(
-          this->records_error());
-    }
-
-    // optional int64 submission_time = 7;
-    if (has_submission_time()) {
-      total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::Int64Size(
-          this->submission_time());
-    }
-
-  }
-  if (_has_bits_[9 / 32] & (0xffu << (9 % 32))) {
-    // optional .exec.shared.RecordBatchDef def = 10;
+    // optional .exec.shared.RecordBatchDef def = 3;
     if (has_def()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
           this->def());
     }
 
-    // optional bool schema_changed = 11;
-    if (has_schema_changed()) {
-      total_size += 1 + 1;
-    }
-
   }
-  // repeated .exec.shared.NodeStatus node_status = 8;
-  total_size += 1 * this->node_status_size();
-  for (int i = 0; i < this->node_status_size(); i++) {
-    total_size +=
-      ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-        this->node_status(i));
-  }
-
-  // repeated .exec.shared.DrillPBError error = 9;
-  total_size += 1 * this->error_size();
-  for (int i = 0; i < this->error_size(); i++) {
-    total_size +=
-      ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-        this->error(i));
-  }
-
   if (!unknown_fields().empty()) {
     total_size +=
       ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
@@ -4707,10 +4724,10 @@ int QueryResult::ByteSize() const {
   return total_size;
 }
 
-void QueryResult::MergeFrom(const ::google::protobuf::Message& from) {
+void QueryData::MergeFrom(const ::google::protobuf::Message& from) {
   GOOGLE_CHECK_NE(&from, this);
-  const QueryResult* source =
-    ::google::protobuf::internal::dynamic_cast_if_available<const QueryResult*>(
+  const QueryData* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const QueryData*>(
       &from);
   if (source == NULL) {
     ::google::protobuf::internal::ReflectionOps::Merge(from, this);
@@ -4719,85 +4736,55 @@ void QueryResult::MergeFrom(const ::google::protobuf::Message& from) {
   }
 }
 
-void QueryResult::MergeFrom(const QueryResult& from) {
+void QueryData::MergeFrom(const QueryData& from) {
   GOOGLE_CHECK_NE(&from, this);
-  node_status_.MergeFrom(from.node_status_);
-  error_.MergeFrom(from.error_);
   if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    if (from.has_query_state()) {
-      set_query_state(from.query_state());
-    }
     if (from.has_query_id()) {
       mutable_query_id()->::exec::shared::QueryId::MergeFrom(from.query_id());
     }
-    if (from.has_is_last_chunk()) {
-      set_is_last_chunk(from.is_last_chunk());
-    }
     if (from.has_row_count()) {
       set_row_count(from.row_count());
     }
-    if (from.has_records_scan()) {
-      set_records_scan(from.records_scan());
-    }
-    if (from.has_records_error()) {
-      set_records_error(from.records_error());
-    }
-    if (from.has_submission_time()) {
-      set_submission_time(from.submission_time());
-    }
-  }
-  if (from._has_bits_[9 / 32] & (0xffu << (9 % 32))) {
     if (from.has_def()) {
       mutable_def()->::exec::shared::RecordBatchDef::MergeFrom(from.def());
     }
-    if (from.has_schema_changed()) {
-      set_schema_changed(from.schema_changed());
-    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
 
-void QueryResult::CopyFrom(const ::google::protobuf::Message& from) {
+void QueryData::CopyFrom(const ::google::protobuf::Message& from) {
   if (&from == this) return;
   Clear();
   MergeFrom(from);
 }
 
-void QueryResult::CopyFrom(const QueryResult& from) {
+void QueryData::CopyFrom(const QueryData& from) {
   if (&from == this) return;
   Clear();
   MergeFrom(from);
 }
 
-bool QueryResult::IsInitialized() const {
+bool QueryData::IsInitialized() const {
 
   return true;
 }
 
-void QueryResult::Swap(QueryResult* other) {
+void QueryData::Swap(QueryData* other) {
   if (other != this) {
-    std::swap(query_state_, other->query_state_);
     std::swap(query_id_, other->query_id_);
-    std::swap(is_last_chunk_, other->is_last_chunk_);
     std::swap(row_count_, other->row_count_);
-    std::swap(records_scan_, other->records_scan_);
-    std::swap(records_error_, other->records_error_);
-    std::swap(submission_time_, other->submission_time_);
-    node_status_.Swap(&other->node_status_);
-    error_.Swap(&other->error_);
     std::swap(def_, other->def_);
-    std::swap(schema_changed_, other->schema_changed_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
   }
 }
 
-::google::protobuf::Metadata QueryResult::GetMetadata() const {
+::google::protobuf::Metadata QueryData::GetMetadata() const {
   protobuf_AssignDescriptorsOnce();
   ::google::protobuf::Metadata metadata;
-  metadata.descriptor = QueryResult_descriptor_;
-  metadata.reflection = QueryResult_reflection_;
+  metadata.descriptor = QueryData_descriptor_;
+  metadata.reflection = QueryData_reflection_;
   return metadata;
 }
 
@@ -6691,7 +6678,7 @@ const int OperatorProfile::kOperatorIdFieldNumber;
 const int OperatorProfile::kOperatorTypeFieldNumber;
 const int OperatorProfile::kSetupNanosFieldNumber;
 const int OperatorProfile::kProcessNanosFieldNumber;
-const int OperatorProfile::kLocalMemoryAllocatedFieldNumber;
+const int OperatorProfile::kPeakLocalMemoryAllocatedFieldNumber;
 const int OperatorProfile::kMetricFieldNumber;
 const int OperatorProfile::kWaitNanosFieldNumber;
 #endif  // !_MSC_VER
@@ -6716,7 +6703,7 @@ void OperatorProfile::SharedCtor() {
   operator_type_ = 0;
   setup_nanos_ = GOOGLE_LONGLONG(0);
   process_nanos_ = GOOGLE_LONGLONG(0);
-  local_memory_allocated_ = GOOGLE_LONGLONG(0);
+  peak_local_memory_allocated_ = GOOGLE_LONGLONG(0);
   wait_nanos_ = GOOGLE_LONGLONG(0);
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
@@ -6757,7 +6744,7 @@ void OperatorProfile::Clear() {
     operator_type_ = 0;
     setup_nanos_ = GOOGLE_LONGLONG(0);
     process_nanos_ = GOOGLE_LONGLONG(0);
-    local_memory_allocated_ = GOOGLE_LONGLONG(0);
+    peak_local_memory_allocated_ = GOOGLE_LONGLONG(0);
     wait_nanos_ = GOOGLE_LONGLONG(0);
   }
   input_profile_.Clear();
@@ -6847,19 +6834,19 @@ bool OperatorProfile::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(56)) goto parse_local_memory_allocated;
+        if (input->ExpectTag(56)) goto parse_peak_local_memory_allocated;
         break;
       }
 
-      // optional int64 local_memory_allocated = 7;
+      // optional int64 peak_local_memory_allocated = 7;
       case 7: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_local_memory_allocated:
+         parse_peak_local_memory_allocated:
           DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
                    ::google::protobuf::int64, ::google::protobuf::internal::WireFormatLite::TYPE_INT64>(
-                 input, &local_memory_allocated_)));
-          set_has_local_memory_allocated();
+                 input, &peak_local_memory_allocated_)));
+          set_has_peak_local_memory_allocated();
         } else {
           goto handle_uninterpreted;
         }
@@ -6942,9 +6929,9 @@ void OperatorProfile::SerializeWithCachedSizes(
     ::google::protobuf::internal::WireFormatLite::WriteInt64(6, this->process_nanos(), output);
   }
 
-  // optional int64 local_memory_allocated = 7;
-  if (has_local_memory_allocated()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt64(7, this->local_memory_allocated(), output);
+  // optional int64 peak_local_memory_allocated = 7;
+  if (has_peak_local_memory_allocated()) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt64(7, this->peak_local_memory_allocated(), output);
   }
 
   // repeated .exec.shared.MetricValue metric = 8;
@@ -6993,9 +6980,9 @@ void OperatorProfile::SerializeWithCachedSizes(
     target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(6, this->process_nanos(), target);
   }
 
-  // optional int64 local_memory_allocated = 7;
-  if (has_local_memory_allocated()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(7, this->local_memory_allocated(), target);
+  // optional int64 peak_local_memory_allocated = 7;
+  if (has_peak_local_memory_allocated()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(7, this->peak_local_memory_allocated(), target);
   }
 
   // repeated .exec.shared.MetricValue metric = 8;
@@ -7049,11 +7036,11 @@ int OperatorProfile::ByteSize() const {
           this->process_nanos());
     }
 
-    // optional int64 local_memory_allocated = 7;
-    if (has_local_memory_allocated()) {
+    // optional int64 peak_local_memory_allocated = 7;
+    if (has_peak_local_memory_allocated()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::Int64Size(
-          this->local_memory_allocated());
+          this->peak_local_memory_allocated());
     }
 
     // optional int64 wait_nanos = 9;
@@ -7120,8 +7107,8 @@ void OperatorProfile::MergeFrom(const OperatorProfile& from) {
     if (from.has_process_nanos()) {
       set_process_nanos(from.process_nanos());
     }
-    if (from.has_local_memory_allocated()) {
-      set_local_memory_allocated(from.local_memory_allocated());
+    if (from.has_peak_local_memory_allocated()) {
+      set_peak_local_memory_allocated(from.peak_local_memory_allocated());
     }
     if (from.has_wait_nanos()) {
       set_wait_nanos(from.wait_nanos());
@@ -7154,7 +7141,7 @@ void OperatorProfile::Swap(OperatorProfile* other) {
     std::swap(operator_type_, other->operator_type_);
     std::swap(setup_nanos_, other->setup_nanos_);
     std::swap(process_nanos_, other->process_nanos_);
-    std::swap(local_memory_allocated_, other->local_memory_allocated_);
+    std::swap(peak_local_memory_allocated_, other->peak_local_memory_allocated_);
     metric_.Swap(&other->metric_);
     std::swap(wait_nanos_, other->wait_nanos_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/UserBitShared.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index bbf3fdc..e2f5fd0 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -49,6 +49,7 @@ class NamePart;
 class SerializedField;
 class NodeStatus;
 class QueryResult;
+class QueryData;
 class QueryInfo;
 class QueryProfile;
 class MajorFragmentProfile;
@@ -82,11 +83,11 @@ enum QueryResult_QueryState {
   QueryResult_QueryState_COMPLETED = 2,
   QueryResult_QueryState_CANCELED = 3,
   QueryResult_QueryState_FAILED = 4,
-  QueryResult_QueryState_UNKNOWN_QUERY = 5
+  QueryResult_QueryState_CANCELLATION_REQUESTED = 5
 };
 bool QueryResult_QueryState_IsValid(int value);
 const QueryResult_QueryState QueryResult_QueryState_QueryState_MIN = QueryResult_QueryState_PENDING;
-const QueryResult_QueryState QueryResult_QueryState_QueryState_MAX = QueryResult_QueryState_UNKNOWN_QUERY;
+const QueryResult_QueryState QueryResult_QueryState_QueryState_MAX = QueryResult_QueryState_CANCELLATION_REQUESTED;
 const int QueryResult_QueryState_QueryState_ARRAYSIZE = QueryResult_QueryState_QueryState_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* QueryResult_QueryState_descriptor();
@@ -1457,7 +1458,7 @@ class QueryResult : public ::google::protobuf::Message {
   static const QueryState COMPLETED = QueryResult_QueryState_COMPLETED;
   static const QueryState CANCELED = QueryResult_QueryState_CANCELED;
   static const QueryState FAILED = QueryResult_QueryState_FAILED;
-  static const QueryState UNKNOWN_QUERY = QueryResult_QueryState_UNKNOWN_QUERY;
+  static const QueryState CANCELLATION_REQUESTED = QueryResult_QueryState_CANCELLATION_REQUESTED;
   static inline bool QueryState_IsValid(int value) {
     return QueryResult_QueryState_IsValid(value);
   }
@@ -1497,57 +1498,10 @@ class QueryResult : public ::google::protobuf::Message {
   inline ::exec::shared::QueryId* release_query_id();
   inline void set_allocated_query_id(::exec::shared::QueryId* query_id);
 
-  // optional bool is_last_chunk = 3;
-  inline bool has_is_last_chunk() const;
-  inline void clear_is_last_chunk();
-  static const int kIsLastChunkFieldNumber = 3;
-  inline bool is_last_chunk() const;
-  inline void set_is_last_chunk(bool value);
-
-  // optional int32 row_count = 4;
-  inline bool has_row_count() const;
-  inline void clear_row_count();
-  static const int kRowCountFieldNumber = 4;
-  inline ::google::protobuf::int32 row_count() const;
-  inline void set_row_count(::google::protobuf::int32 value);
-
-  // optional int64 records_scan = 5;
-  inline bool has_records_scan() const;
-  inline void clear_records_scan();
-  static const int kRecordsScanFieldNumber = 5;
-  inline ::google::protobuf::int64 records_scan() const;
-  inline void set_records_scan(::google::protobuf::int64 value);
-
-  // optional int64 records_error = 6;
-  inline bool has_records_error() const;
-  inline void clear_records_error();
-  static const int kRecordsErrorFieldNumber = 6;
-  inline ::google::protobuf::int64 records_error() const;
-  inline void set_records_error(::google::protobuf::int64 value);
-
-  // optional int64 submission_time = 7;
-  inline bool has_submission_time() const;
-  inline void clear_submission_time();
-  static const int kSubmissionTimeFieldNumber = 7;
-  inline ::google::protobuf::int64 submission_time() const;
-  inline void set_submission_time(::google::protobuf::int64 value);
-
-  // repeated .exec.shared.NodeStatus node_status = 8;
-  inline int node_status_size() const;
-  inline void clear_node_status();
-  static const int kNodeStatusFieldNumber = 8;
-  inline const ::exec::shared::NodeStatus& node_status(int index) const;
-  inline ::exec::shared::NodeStatus* mutable_node_status(int index);
-  inline ::exec::shared::NodeStatus* add_node_status();
-  inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::NodeStatus >&
-      node_status() const;
-  inline ::google::protobuf::RepeatedPtrField< ::exec::shared::NodeStatus >*
-      mutable_node_status();
-
-  // repeated .exec.shared.DrillPBError error = 9;
+  // repeated .exec.shared.DrillPBError error = 3;
   inline int error_size() const;
   inline void clear_error();
-  static const int kErrorFieldNumber = 9;
+  static const int kErrorFieldNumber = 3;
   inline const ::exec::shared::DrillPBError& error(int index) const;
   inline ::exec::shared::DrillPBError* mutable_error(int index);
   inline ::exec::shared::DrillPBError* add_error();
@@ -1556,66 +1510,134 @@ class QueryResult : public ::google::protobuf::Message {
   inline ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError >*
       mutable_error();
 
-  // optional .exec.shared.RecordBatchDef def = 10;
+  // @@protoc_insertion_point(class_scope:exec.shared.QueryResult)
+ private:
+  inline void set_has_query_state();
+  inline void clear_has_query_state();
+  inline void set_has_query_id();
+  inline void clear_has_query_id();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::shared::QueryId* query_id_;
+  ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError > error_;
+  int query_state_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_UserBitShared_2eproto();
+  friend void protobuf_AssignDesc_UserBitShared_2eproto();
+  friend void protobuf_ShutdownFile_UserBitShared_2eproto();
+
+  void InitAsDefaultInstance();
+  static QueryResult* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class QueryData : public ::google::protobuf::Message {
+ public:
+  QueryData();
+  virtual ~QueryData();
+
+  QueryData(const QueryData& from);
+
+  inline QueryData& operator=(const QueryData& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const QueryData& default_instance();
+
+  void Swap(QueryData* other);
+
+  // implements Message ----------------------------------------------
+
+  QueryData* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const QueryData& from);
+  void MergeFrom(const QueryData& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.shared.QueryId query_id = 1;
+  inline bool has_query_id() const;
+  inline void clear_query_id();
+  static const int kQueryIdFieldNumber = 1;
+  inline const ::exec::shared::QueryId& query_id() const;
+  inline ::exec::shared::QueryId* mutable_query_id();
+  inline ::exec::shared::QueryId* release_query_id();
+  inline void set_allocated_query_id(::exec::shared::QueryId* query_id);
+
+  // optional int32 row_count = 2;
+  inline bool has_row_count() const;
+  inline void clear_row_count();
+  static const int kRowCountFieldNumber = 2;
+  inline ::google::protobuf::int32 row_count() const;
+  inline void set_row_count(::google::protobuf::int32 value);
+
+  // optional .exec.shared.RecordBatchDef def = 3;
   inline bool has_def() const;
   inline void clear_def();
-  static const int kDefFieldNumber = 10;
+  static const int kDefFieldNumber = 3;
   inline const ::exec::shared::RecordBatchDef& def() const;
   inline ::exec::shared::RecordBatchDef* mutable_def();
   inline ::exec::shared::RecordBatchDef* release_def();
   inline void set_allocated_def(::exec::shared::RecordBatchDef* def);
 
-  // optional bool schema_changed = 11;
-  inline bool has_schema_changed() const;
-  inline void clear_schema_changed();
-  static const int kSchemaChangedFieldNumber = 11;
-  inline bool schema_changed() const;
-  inline void set_schema_changed(bool value);
-
-  // @@protoc_insertion_point(class_scope:exec.shared.QueryResult)
+  // @@protoc_insertion_point(class_scope:exec.shared.QueryData)
  private:
-  inline void set_has_query_state();
-  inline void clear_has_query_state();
   inline void set_has_query_id();
   inline void clear_has_query_id();
-  inline void set_has_is_last_chunk();
-  inline void clear_has_is_last_chunk();
   inline void set_has_row_count();
   inline void clear_has_row_count();
-  inline void set_has_records_scan();
-  inline void clear_has_records_scan();
-  inline void set_has_records_error();
-  inline void clear_has_records_error();
-  inline void set_has_submission_time();
-  inline void clear_has_submission_time();
   inline void set_has_def();
   inline void clear_has_def();
-  inline void set_has_schema_changed();
-  inline void clear_has_schema_changed();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
   ::exec::shared::QueryId* query_id_;
-  int query_state_;
-  ::google::protobuf::int32 row_count_;
-  ::google::protobuf::int64 records_scan_;
-  ::google::protobuf::int64 records_error_;
-  ::google::protobuf::int64 submission_time_;
-  ::google::protobuf::RepeatedPtrField< ::exec::shared::NodeStatus > node_status_;
-  ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError > error_;
   ::exec::shared::RecordBatchDef* def_;
-  bool is_last_chunk_;
-  bool schema_changed_;
+  ::google::protobuf::int32 row_count_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(11 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
 
   friend void  protobuf_AddDesc_UserBitShared_2eproto();
   friend void protobuf_AssignDesc_UserBitShared_2eproto();
   friend void protobuf_ShutdownFile_UserBitShared_2eproto();
 
   void InitAsDefaultInstance();
-  static QueryResult* default_instance_;
+  static QueryData* default_instance_;
 };
 // -------------------------------------------------------------------
 
@@ -2310,12 +2332,12 @@ class OperatorProfile : public ::google::protobuf::Message {
   inline ::google::protobuf::int64 process_nanos() const;
   inline void set_process_nanos(::google::protobuf::int64 value);
 
-  // optional int64 local_memory_allocated = 7;
-  inline bool has_local_memory_allocated() const;
-  inline void clear_local_memory_allocated();
-  static const int kLocalMemoryAllocatedFieldNumber = 7;
-  inline ::google::protobuf::int64 local_memory_allocated() const;
-  inline void set_local_memory_allocated(::google::protobuf::int64 value);
+  // optional int64 peak_local_memory_allocated = 7;
+  inline bool has_peak_local_memory_allocated() const;
+  inline void clear_peak_local_memory_allocated();
+  static const int kPeakLocalMemoryAllocatedFieldNumber = 7;
+  inline ::google::protobuf::int64 peak_local_memory_allocated() const;
+  inline void set_peak_local_memory_allocated(::google::protobuf::int64 value);
 
   // repeated .exec.shared.MetricValue metric = 8;
   inline int metric_size() const;
@@ -2346,8 +2368,8 @@ class OperatorProfile : public ::google::protobuf::Message {
   inline void clear_has_setup_nanos();
   inline void set_has_process_nanos();
   inline void clear_has_process_nanos();
-  inline void set_has_local_memory_allocated();
-  inline void clear_has_local_memory_allocated();
+  inline void set_has_peak_local_memory_allocated();
+  inline void clear_has_peak_local_memory_allocated();
   inline void set_has_wait_nanos();
   inline void clear_has_wait_nanos();
 
@@ -2358,7 +2380,7 @@ class OperatorProfile : public ::google::protobuf::Message {
   ::google::protobuf::int32 operator_type_;
   ::google::protobuf::int64 setup_nanos_;
   ::google::protobuf::int64 process_nanos_;
-  ::google::protobuf::int64 local_memory_allocated_;
+  ::google::protobuf::int64 peak_local_memory_allocated_;
   ::google::protobuf::RepeatedPtrField< ::exec::shared::MetricValue > metric_;
   ::google::protobuf::int64 wait_nanos_;
 
@@ -4039,195 +4061,124 @@ inline void QueryResult::set_allocated_query_id(::exec::shared::QueryId* query_i
   }
 }
 
-// optional bool is_last_chunk = 3;
-inline bool QueryResult::has_is_last_chunk() const {
-  return (_has_bits_[0] & 0x00000004u) != 0;
-}
-inline void QueryResult::set_has_is_last_chunk() {
-  _has_bits_[0] |= 0x00000004u;
-}
-inline void QueryResult::clear_has_is_last_chunk() {
-  _has_bits_[0] &= ~0x00000004u;
-}
-inline void QueryResult::clear_is_last_chunk() {
-  is_last_chunk_ = false;
-  clear_has_is_last_chunk();
-}
-inline bool QueryResult::is_last_chunk() const {
-  return is_last_chunk_;
-}
-inline void QueryResult::set_is_last_chunk(bool value) {
-  set_has_is_last_chunk();
-  is_last_chunk_ = value;
+// repeated .exec.shared.DrillPBError error = 3;
+inline int QueryResult::error_size() const {
+  return error_.size();
 }
-
-// optional int32 row_count = 4;
-inline bool QueryResult::has_row_count() const {
-  return (_has_bits_[0] & 0x00000008u) != 0;
+inline void QueryResult::clear_error() {
+  error_.Clear();
 }
-inline void QueryResult::set_has_row_count() {
-  _has_bits_[0] |= 0x00000008u;
+inline const ::exec::shared::DrillPBError& QueryResult::error(int index) const {
+  return error_.Get(index);
 }
-inline void QueryResult::clear_has_row_count() {
-  _has_bits_[0] &= ~0x00000008u;
+inline ::exec::shared::DrillPBError* QueryResult::mutable_error(int index) {
+  return error_.Mutable(index);
 }
-inline void QueryResult::clear_row_count() {
-  row_count_ = 0;
-  clear_has_row_count();
+inline ::exec::shared::DrillPBError* QueryResult::add_error() {
+  return error_.Add();
 }
-inline ::google::protobuf::int32 QueryResult::row_count() const {
-  return row_count_;
+inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError >&
+QueryResult::error() const {
+  return error_;
 }
-inline void QueryResult::set_row_count(::google::protobuf::int32 value) {
-  set_has_row_count();
-  row_count_ = value;
+inline ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError >*
+QueryResult::mutable_error() {
+  return &error_;
 }
 
-// optional int64 records_scan = 5;
-inline bool QueryResult::has_records_scan() const {
-  return (_has_bits_[0] & 0x00000010u) != 0;
-}
-inline void QueryResult::set_has_records_scan() {
-  _has_bits_[0] |= 0x00000010u;
-}
-inline void QueryResult::clear_has_records_scan() {
-  _has_bits_[0] &= ~0x00000010u;
-}
-inline void QueryResult::clear_records_scan() {
-  records_scan_ = GOOGLE_LONGLONG(0);
-  clear_has_records_scan();
-}
-inline ::google::protobuf::int64 QueryResult::records_scan() const {
-  return records_scan_;
-}
-inline void QueryResult::set_records_scan(::google::protobuf::int64 value) {
-  set_has_records_scan();
-  records_scan_ = value;
-}
+// -------------------------------------------------------------------
 
-// optional int64 records_error = 6;
-inline bool QueryResult::has_records_error() const {
-  return (_has_bits_[0] & 0x00000020u) != 0;
-}
-inline void QueryResult::set_has_records_error() {
-  _has_bits_[0] |= 0x00000020u;
-}
-inline void QueryResult::clear_has_records_error() {
-  _has_bits_[0] &= ~0x00000020u;
-}
-inline void QueryResult::clear_records_error() {
-  records_error_ = GOOGLE_LONGLONG(0);
-  clear_has_records_error();
-}
-inline ::google::protobuf::int64 QueryResult::records_error() const {
-  return records_error_;
-}
-inline void QueryResult::set_records_error(::google::protobuf::int64 value) {
-  set_has_records_error();
-  records_error_ = value;
-}
+// QueryData
 
-// optional int64 submission_time = 7;
-inline bool QueryResult::has_submission_time() const {
-  return (_has_bits_[0] & 0x00000040u) != 0;
-}
-inline void QueryResult::set_has_submission_time() {
-  _has_bits_[0] |= 0x00000040u;
-}
-inline void QueryResult::clear_has_submission_time() {
-  _has_bits_[0] &= ~0x00000040u;
-}
-inline void QueryResult::clear_submission_time() {
-  submission_time_ = GOOGLE_LONGLONG(0);
-  clear_has_submission_time();
-}
-inline ::google::protobuf::int64 QueryResult::submission_time() const {
-  return submission_time_;
-}
-inline void QueryResult::set_submission_time(::google::protobuf::int64 value) {
-  set_has_submission_time();
-  submission_time_ = value;
+// optional .exec.shared.QueryId query_id = 1;
+inline bool QueryData::has_query_id() const {
+  return (_has_bits_[0] & 0x00000001u) != 0;
 }
-
-// repeated .exec.shared.NodeStatus node_status = 8;
-inline int QueryResult::node_status_size() const {
-  return node_status_.size();
+inline void QueryData::set_has_query_id() {
+  _has_bits_[0] |= 0x00000001u;
 }
-inline void QueryResult::clear_node_status() {
-  node_status_.Clear();
+inline void QueryData::clear_has_query_id() {
+  _has_bits_[0] &= ~0x00000001u;
 }
-inline const ::exec::shared::NodeStatus& QueryResult::node_status(int index) const {
-  return node_status_.Get(index);
+inline void QueryData::clear_query_id() {
+  if (query_id_ != NULL) query_id_->::exec::shared::QueryId::Clear();
+  clear_has_query_id();
 }
-inline ::exec::shared::NodeStatus* QueryResult::mutable_node_status(int index) {
-  return node_status_.Mutable(index);
+inline const ::exec::shared::QueryId& QueryData::query_id() const {
+  return query_id_ != NULL ? *query_id_ : *default_instance_->query_id_;
 }
-inline ::exec::shared::NodeStatus* QueryResult::add_node_status() {
-  return node_status_.Add();
+inline ::exec::shared::QueryId* QueryData::mutable_query_id() {
+  set_has_query_id();
+  if (query_id_ == NULL) query_id_ = new ::exec::shared::QueryId;
+  return query_id_;
 }
-inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::NodeStatus >&
-QueryResult::node_status() const {
-  return node_status_;
+inline ::exec::shared::QueryId* QueryData::release_query_id() {
+  clear_has_query_id();
+  ::exec::shared::QueryId* temp = query_id_;
+  query_id_ = NULL;
+  return temp;
 }
-inline ::google::protobuf::RepeatedPtrField< ::exec::shared::NodeStatus >*
-QueryResult::mutable_node_status() {
-  return &node_status_;
+inline void QueryData::set_allocated_query_id(::exec::shared::QueryId* query_id) {
+  delete query_id_;
+  query_id_ = query_id;
+  if (query_id) {
+    set_has_query_id();
+  } else {
+    clear_has_query_id();
+  }
 }
 
-// repeated .exec.shared.DrillPBError error = 9;
-inline int QueryResult::error_size() const {
-  return error_.size();
-}
-inline void QueryResult::clear_error() {
-  error_.Clear();
+// optional int32 row_count = 2;
+inline bool QueryData::has_row_count() const {
+  return (_has_bits_[0] & 0x00000002u) != 0;
 }
-inline const ::exec::shared::DrillPBError& QueryResult::error(int index) const {
-  return error_.Get(index);
+inline void QueryData::set_has_row_count() {
+  _has_bits_[0] |= 0x00000002u;
 }
-inline ::exec::shared::DrillPBError* QueryResult::mutable_error(int index) {
-  return error_.Mutable(index);
+inline void QueryData::clear_has_row_count() {
+  _has_bits_[0] &= ~0x00000002u;
 }
-inline ::exec::shared::DrillPBError* QueryResult::add_error() {
-  return error_.Add();
+inline void QueryData::clear_row_count() {
+  row_count_ = 0;
+  clear_has_row_count();
 }
-inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError >&
-QueryResult::error() const {
-  return error_;
+inline ::google::protobuf::int32 QueryData::row_count() const {
+  return row_count_;
 }
-inline ::google::protobuf::RepeatedPtrField< ::exec::shared::DrillPBError >*
-QueryResult::mutable_error() {
-  return &error_;
+inline void QueryData::set_row_count(::google::protobuf::int32 value) {
+  set_has_row_count();
+  row_count_ = value;
 }
 
-// optional .exec.shared.RecordBatchDef def = 10;
-inline bool QueryResult::has_def() const {
-  return (_has_bits_[0] & 0x00000200u) != 0;
+// optional .exec.shared.RecordBatchDef def = 3;
+inline bool QueryData::has_def() const {
+  return (_has_bits_[0] & 0x00000004u) != 0;
 }
-inline void QueryResult::set_has_def() {
-  _has_bits_[0] |= 0x00000200u;
+inline void QueryData::set_has_def() {
+  _has_bits_[0] |= 0x00000004u;
 }
-inline void QueryResult::clear_has_def() {
-  _has_bits_[0] &= ~0x00000200u;
+inline void QueryData::clear_has_def() {
+  _has_bits_[0] &= ~0x00000004u;
 }
-inline void QueryResult::clear_def() {
+inline void QueryData::clear_def() {
   if (def_ != NULL) def_->::exec::shared::RecordBatchDef::Clear();
   clear_has_def();
 }
-inline const ::exec::shared::RecordBatchDef& QueryResult::def() const {
+inline const ::exec::shared::RecordBatchDef& QueryData::def() const {
   return def_ != NULL ? *def_ : *default_instance_->def_;
 }
-inline ::exec::shared::RecordBatchDef* QueryResult::mutable_def() {
+inline ::exec::shared::RecordBatchDef* QueryData::mutable_def() {
   set_has_def();
   if (def_ == NULL) def_ = new ::exec::shared::RecordBatchDef;
   return def_;
 }
-inline ::exec::shared::RecordBatchDef* QueryResult::release_def() {
+inline ::exec::shared::RecordBatchDef* QueryData::release_def() {
   clear_has_def();
   ::exec::shared::RecordBatchDef* temp = def_;
   def_ = NULL;
   return temp;
 }
-inline void QueryResult::set_allocated_def(::exec::shared::RecordBatchDef* def) {
+inline void QueryData::set_allocated_def(::exec::shared::RecordBatchDef* def) {
   delete def_;
   def_ = def;
   if (def) {
@@ -4237,28 +4188,6 @@ inline void QueryResult::set_allocated_def(::exec::shared::RecordBatchDef* def)
   }
 }
 
-// optional bool schema_changed = 11;
-inline bool QueryResult::has_schema_changed() const {
-  return (_has_bits_[0] & 0x00000400u) != 0;
-}
-inline void QueryResult::set_has_schema_changed() {
-  _has_bits_[0] |= 0x00000400u;
-}
-inline void QueryResult::clear_has_schema_changed() {
-  _has_bits_[0] &= ~0x00000400u;
-}
-inline void QueryResult::clear_schema_changed() {
-  schema_changed_ = false;
-  clear_has_schema_changed();
-}
-inline bool QueryResult::schema_changed() const {
-  return schema_changed_;
-}
-inline void QueryResult::set_schema_changed(bool value) {
-  set_has_schema_changed();
-  schema_changed_ = value;
-}
-
 // -------------------------------------------------------------------
 
 // QueryInfo
@@ -5271,26 +5200,26 @@ inline void OperatorProfile::set_process_nanos(::google::protobuf::int64 value)
   process_nanos_ = value;
 }
 
-// optional int64 local_memory_allocated = 7;
-inline bool OperatorProfile::has_local_memory_allocated() const {
+// optional int64 peak_local_memory_allocated = 7;
+inline bool OperatorProfile::has_peak_local_memory_allocated() const {
   return (_has_bits_[0] & 0x00000020u) != 0;
 }
-inline void OperatorProfile::set_has_local_memory_allocated() {
+inline void OperatorProfile::set_has_peak_local_memory_allocated() {
   _has_bits_[0] |= 0x00000020u;
 }
-inline void OperatorProfile::clear_has_local_memory_allocated() {
+inline void OperatorProfile::clear_has_peak_local_memory_allocated() {
   _has_bits_[0] &= ~0x00000020u;
 }
-inline void OperatorProfile::clear_local_memory_allocated() {
-  local_memory_allocated_ = GOOGLE_LONGLONG(0);
-  clear_has_local_memory_allocated();
+inline void OperatorProfile::clear_peak_local_memory_allocated() {
+  peak_local_memory_allocated_ = GOOGLE_LONGLONG(0);
+  clear_has_peak_local_memory_allocated();
 }
-inline ::google::protobuf::int64 OperatorProfile::local_memory_allocated() const {
-  return local_memory_allocated_;
+inline ::google::protobuf::int64 OperatorProfile::peak_local_memory_allocated() const {
+  return peak_local_memory_allocated_;
 }
-inline void OperatorProfile::set_local_memory_allocated(::google::protobuf::int64 value) {
-  set_has_local_memory_allocated();
-  local_memory_allocated_ = value;
+inline void OperatorProfile::set_peak_local_memory_allocated(::google::protobuf::int64 value) {
+  set_has_peak_local_memory_allocated();
+  peak_local_memory_allocated_ = value;
 }
 
 // repeated .exec.shared.MetricValue metric = 8;


[6/9] drill git commit: DRILL-2673: Update UserServer <==> UserClient RPC to better handle handshake response

Posted by pa...@apache.org.
DRILL-2673: Update UserServer <==> UserClient RPC to better handle handshake response


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

Branch: refs/heads/master
Commit: a218ee3fdd4403c0e1f4478cacf320f0947ea902
Parents: 1d9d82b
Author: vkorukanti <ve...@gmail.com>
Authored: Thu Apr 2 00:23:29 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Apr 3 18:40:52 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/client/DrillClient.java   |   4 +-
 .../exec/rpc/AbstractHandshakeHandler.java      |  33 +-
 .../org/apache/drill/exec/rpc/BasicServer.java  |   2 +-
 .../apache/drill/exec/rpc/user/UserClient.java  |  12 +-
 .../drill/exec/rpc/user/UserRpcConfig.java      |   2 +-
 .../apache/drill/exec/rpc/user/UserServer.java  |  67 ++-
 .../apache/drill/jdbc/DrillConnectionImpl.java  |   2 +-
 .../drill/exec/proto/SchemaUserProtos.java      |  21 +
 .../org/apache/drill/exec/proto/UserProtos.java | 559 ++++++++++++++++++-
 .../exec/proto/beans/BitToUserHandshake.java    |  66 +++
 .../drill/exec/proto/beans/HandshakeStatus.java |  53 ++
 protocol/src/main/protobuf/User.proto           |  10 +
 12 files changed, 789 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/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 650a2eb..9a948fb 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
@@ -135,7 +135,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
    * @throws RpcException
    */
   public void connect() throws RpcException {
-    connect(null, new Properties());
+    connect(null, null);
   }
 
   public void connect(Properties props) throws RpcException {
@@ -365,7 +365,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
 
     @Override
     public void connectionFailed(FailureType type, Throwable t) {
-      getInner().setException(new RpcException(String.format("Failure connecting to server. Failure of type %s.", type.name()), t));
+      getInner().setException(new RpcException(String.format("%s : %s", type.name(), t.getMessage()), t));
     }
 
     private SettableFuture<Void> getInner() {

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
index 9048241..9f426e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -42,23 +42,26 @@ public abstract class AbstractHandshakeHandler<T extends MessageLite> extends Me
 
   @Override
   protected void decode(ChannelHandlerContext ctx, InboundRpcMessage inbound, List<Object> outputs) throws Exception {
-    if (RpcConstants.EXTRA_DEBUGGING) {
-      logger.debug("Received handshake {}", inbound);
-    }
-    this.coordinationId = inbound.coordinationId;
-    ctx.channel().pipeline().remove(this);
-    if (inbound.rpcType != handshakeType.getNumber()) {
-      throw new RpcException(String.format("Handshake failure.  Expected %s[%d] but received number [%d]",
-          handshakeType, handshakeType.getNumber(), inbound.rpcType));
-    }
+    try {
+      if (RpcConstants.EXTRA_DEBUGGING) {
+        logger.debug("Received handshake {}", inbound);
+      }
+      this.coordinationId = inbound.coordinationId;
+      ctx.channel().pipeline().remove(this);
+      if (inbound.rpcType != handshakeType.getNumber()) {
+        throw new RpcException(String.format("Handshake failure.  Expected %s[%d] but received number [%d]",
+            handshakeType, handshakeType.getNumber(), inbound.rpcType));
+      }
 
-    T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
-    consumeHandshake(ctx, msg);
-    inbound.pBody.release();
-    if (inbound.dBody != null) {
-      inbound.dBody.release();
+      T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
+      consumeHandshake(ctx, msg);
+    } finally {
+      // Consuming a handshake may result in exceptions, so make sure to release the message buffers.
+      inbound.pBody.release();
+      if (inbound.dBody != null) {
+        inbound.dBody.release();
+      }
     }
-
   }
 
   protected abstract void consumeHandshake(ChannelHandlerContext ctx, T msg) throws Exception;

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index 0e0398d..3a7032b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -108,7 +108,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
     }
 
     @Override
-    protected final void consumeHandshake(ChannelHandlerContext ctx, T inbound) throws Exception {
+    protected void consumeHandshake(ChannelHandlerContext ctx, T inbound) throws Exception {
       OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId,
           getHandshakeResponse(inbound));
       ctx.writeAndFlush(msg);

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index fd82699..8022c95 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
+import org.apache.drill.exec.proto.UserProtos.HandshakeStatus;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
@@ -42,7 +43,7 @@ import org.apache.drill.exec.rpc.RpcException;
 import com.google.protobuf.MessageLite;
 
 public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHandshake, BitToUserHandshake> {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
 
   private final QueryResultHandler queryResultHandler = new QueryResultHandler();
 
@@ -107,11 +108,12 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
   @Override
   protected void validateHandshake(BitToUserHandshake inbound) throws RpcException {
 //    logger.debug("Handling handshake from bit to user. {}", inbound);
-    if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {
-      throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(),
-          UserRpcConfig.RPC_VERSION));
+    if (inbound.getStatus() != HandshakeStatus.SUCCESS) {
+      final String errMsg = String.format("Status: %s, Error Id: %s, Error message: %s",
+          inbound.getStatus(), inbound.getErrorId(), inbound.getErrorMessage());
+      logger.error(errMsg);
+      throw new RpcException(errMsg);
     }
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
index 9fb8bdb..88592d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
@@ -38,5 +38,5 @@ public class UserRpcConfig {
       .add(RpcType.QUERY_RESULT, QueryResult.class, RpcType.ACK, Ack.class) //bit to user
       .build();
 
-  public static int RPC_VERSION = 4;
+  public static int RPC_VERSION = 5;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 5b2433b..17f189d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -20,21 +20,26 @@ package org.apache.drill.exec.rpc.user;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.EventLoopGroup;
 
 import java.io.IOException;
+import java.util.UUID;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
+import org.apache.drill.exec.proto.UserProtos.HandshakeStatus;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.OutOfMemoryHandler;
+import org.apache.drill.exec.rpc.OutboundRpcMessage;
 import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
 import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.Response;
@@ -149,19 +154,67 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
     return new ServerHandshakeHandler<UserToBitHandshake>(RpcType.HANDSHAKE, UserToBitHandshake.PARSER){
 
       @Override
-      public MessageLite getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
-        logger.trace("Handling handshake from user to bit. {}", inbound);
-        if(inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {
-          throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+      protected void consumeHandshake(ChannelHandlerContext ctx, UserToBitHandshake inbound) throws Exception {
+        BitToUserHandshake handshakeResp = getHandshakeResponse(inbound);
+        OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId, handshakeResp);
+        ctx.writeAndFlush(msg);
+
+        if (handshakeResp.getStatus() != HandshakeStatus.SUCCESS) {
+          // If handling handshake results in an error, throw an exception to terminate the connection.
+          throw new RpcException("Handshake request failed: " + handshakeResp.getErrorMessage());
         }
+      }
 
-        connection.setUser(inbound);
+      @Override
+      public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
+        logger.trace("Handling handshake from user to bit. {}", inbound);
 
-        return BitToUserHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build();
-      }
+        BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
+            .setRpcVersion(UserRpcConfig.RPC_VERSION);
+
+        try {
+          if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {
+            final String errMsg = String.format("Invalid rpc version. Expected %d, actual %d.",
+                UserRpcConfig.RPC_VERSION, inbound.getRpcVersion());
+
+            return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null);
+          }
 
+          connection.setUser(inbound);
+
+          return respBuilder.setStatus(HandshakeStatus.SUCCESS).build();
+        } catch (Exception e) {
+          return handleFailure(respBuilder, HandshakeStatus.UNKNOWN_FAILURE, e.getMessage(), e);
+        }
+      }
     };
+  }
+
+  /**
+   * Complete building the given builder for <i>BitToUserHandshake</i> message with given status and error details.
+   *
+   * @param respBuilder Instance of {@link org.apache.drill.exec.proto.UserProtos.BitToUserHandshake} builder which
+   *                    has RPC version field already set.
+   * @param status  Status of handling handshake request.
+   * @param errMsg  Error message.
+   * @param exception Optional exception.
+   * @return
+   */
+  private static BitToUserHandshake handleFailure(BitToUserHandshake.Builder respBuilder, HandshakeStatus status,
+      String errMsg, Exception exception) {
+    final String errorId = UUID.randomUUID().toString();
+
+    if (exception != null) {
+      logger.error("Error {} in Handling handshake request: {}, {}", errorId, status, errMsg, exception);
+    } else {
+      logger.error("Error {} in Handling handshake request: {}, {}", errorId, status, errMsg);
+    }
 
+    return respBuilder
+        .setStatus(status)
+        .setErrorId(errorId)
+        .setErrorMessage(errMsg)
+        .build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
index e590778..8caeb3f 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -98,7 +98,7 @@ abstract class DrillConnectionImpl extends AvaticaConnection implements DrillCon
         this.client.connect(config.getZookeeperConnectionString(), info);
       }
     } catch (RpcException e) {
-      throw new SQLException("Failure while attempting to connect to Drill.", e);
+      throw new SQLException("Failure while attempting to connect to Drill: " + e.getMessage(), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
index 3b056cf..d587dfc 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserProtos.java
@@ -663,6 +663,12 @@ public final class SchemaUserProtos
             {
                 if(message.hasRpcVersion())
                     output.writeInt32(2, message.getRpcVersion(), false);
+                if(message.hasStatus())
+                    output.writeEnum(3, message.getStatus().getNumber(), false);
+                if(message.hasErrorId())
+                    output.writeString(4, message.getErrorId(), false);
+                if(message.hasErrorMessage())
+                    output.writeString(5, message.getErrorMessage(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserProtos.BitToUserHandshake message)
             {
@@ -705,6 +711,15 @@ public final class SchemaUserProtos
                         case 2:
                             builder.setRpcVersion(input.readInt32());
                             break;
+                        case 3:
+                            builder.setStatus(org.apache.drill.exec.proto.UserProtos.HandshakeStatus.valueOf(input.readEnum()));
+                            break;
+                        case 4:
+                            builder.setErrorId(input.readString());
+                            break;
+                        case 5:
+                            builder.setErrorMessage(input.readString());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -746,6 +761,9 @@ public final class SchemaUserProtos
             switch(number)
             {
                 case 2: return "rpcVersion";
+                case 3: return "status";
+                case 4: return "errorId";
+                case 5: return "errorMessage";
                 default: return null;
             }
         }
@@ -758,6 +776,9 @@ public final class SchemaUserProtos
         static
         {
             fieldMap.put("rpcVersion", 2);
+            fieldMap.put("status", 3);
+            fieldMap.put("errorId", 4);
+            fieldMap.put("errorMessage", 5);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index 0b81926..c072a47 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -305,6 +305,138 @@ public final class UserProtos {
     // @@protoc_insertion_point(enum_scope:exec.user.QueryResultsMode)
   }
 
+  /**
+   * Protobuf enum {@code exec.user.HandshakeStatus}
+   */
+  public enum HandshakeStatus
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SUCCESS = 1;</code>
+     *
+     * <pre>
+     * Handshake is successful (including authentication if any)
+     * </pre>
+     */
+    SUCCESS(0, 1),
+    /**
+     * <code>RPC_VERSION_MISMATCH = 2;</code>
+     *
+     * <pre>
+     * Client and Server RPC versions are different
+     * </pre>
+     */
+    RPC_VERSION_MISMATCH(1, 2),
+    /**
+     * <code>AUTH_FAILED = 3;</code>
+     *
+     * <pre>
+     * User authentication failed
+     * </pre>
+     */
+    AUTH_FAILED(2, 3),
+    /**
+     * <code>UNKNOWN_FAILURE = 4;</code>
+     *
+     * <pre>
+     * Unknown failure, refer to the error message for more details
+     * </pre>
+     */
+    UNKNOWN_FAILURE(3, 4),
+    ;
+
+    /**
+     * <code>SUCCESS = 1;</code>
+     *
+     * <pre>
+     * Handshake is successful (including authentication if any)
+     * </pre>
+     */
+    public static final int SUCCESS_VALUE = 1;
+    /**
+     * <code>RPC_VERSION_MISMATCH = 2;</code>
+     *
+     * <pre>
+     * Client and Server RPC versions are different
+     * </pre>
+     */
+    public static final int RPC_VERSION_MISMATCH_VALUE = 2;
+    /**
+     * <code>AUTH_FAILED = 3;</code>
+     *
+     * <pre>
+     * User authentication failed
+     * </pre>
+     */
+    public static final int AUTH_FAILED_VALUE = 3;
+    /**
+     * <code>UNKNOWN_FAILURE = 4;</code>
+     *
+     * <pre>
+     * Unknown failure, refer to the error message for more details
+     * </pre>
+     */
+    public static final int UNKNOWN_FAILURE_VALUE = 4;
+
+
+    public final int getNumber() { return value; }
+
+    public static HandshakeStatus valueOf(int value) {
+      switch (value) {
+        case 1: return SUCCESS;
+        case 2: return RPC_VERSION_MISMATCH;
+        case 3: return AUTH_FAILED;
+        case 4: return UNKNOWN_FAILURE;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<HandshakeStatus>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<HandshakeStatus>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<HandshakeStatus>() {
+            public HandshakeStatus findValueByNumber(int number) {
+              return HandshakeStatus.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserProtos.getDescriptor().getEnumTypes().get(2);
+    }
+
+    private static final HandshakeStatus[] VALUES = values();
+
+    public static HandshakeStatus valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private HandshakeStatus(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:exec.user.HandshakeStatus)
+  }
+
   public interface PropertyOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -3929,6 +4061,46 @@ public final class UserProtos {
      * <code>optional int32 rpc_version = 2;</code>
      */
     int getRpcVersion();
+
+    // optional .exec.user.HandshakeStatus status = 3;
+    /**
+     * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+     */
+    boolean hasStatus();
+    /**
+     * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+     */
+    org.apache.drill.exec.proto.UserProtos.HandshakeStatus getStatus();
+
+    // optional string errorId = 4;
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    boolean hasErrorId();
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    java.lang.String getErrorId();
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorIdBytes();
+
+    // optional string errorMessage = 5;
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    boolean hasErrorMessage();
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    java.lang.String getErrorMessage();
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorMessageBytes();
   }
   /**
    * Protobuf type {@code exec.user.BitToUserHandshake}
@@ -3986,6 +4158,27 @@ public final class UserProtos {
               rpcVersion_ = input.readInt32();
               break;
             }
+            case 24: {
+              int rawValue = input.readEnum();
+              org.apache.drill.exec.proto.UserProtos.HandshakeStatus value = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(3, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                status_ = value;
+              }
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000004;
+              errorId_ = input.readBytes();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000008;
+              errorMessage_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4042,8 +4235,113 @@ public final class UserProtos {
       return rpcVersion_;
     }
 
+    // optional .exec.user.HandshakeStatus status = 3;
+    public static final int STATUS_FIELD_NUMBER = 3;
+    private org.apache.drill.exec.proto.UserProtos.HandshakeStatus status_;
+    /**
+     * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+     */
+    public boolean hasStatus() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+     */
+    public org.apache.drill.exec.proto.UserProtos.HandshakeStatus getStatus() {
+      return status_;
+    }
+
+    // optional string errorId = 4;
+    public static final int ERRORID_FIELD_NUMBER = 4;
+    private java.lang.Object errorId_;
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    public boolean hasErrorId() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    public java.lang.String getErrorId() {
+      java.lang.Object ref = errorId_;
+      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()) {
+          errorId_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string errorId = 4;</code>
+     */
+    public com.google.protobuf.ByteString
+        getErrorIdBytes() {
+      java.lang.Object ref = errorId_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        errorId_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string errorMessage = 5;
+    public static final int ERRORMESSAGE_FIELD_NUMBER = 5;
+    private java.lang.Object errorMessage_;
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    public boolean hasErrorMessage() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    public java.lang.String getErrorMessage() {
+      java.lang.Object ref = errorMessage_;
+      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()) {
+          errorMessage_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string errorMessage = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getErrorMessageBytes() {
+      java.lang.Object ref = errorMessage_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        errorMessage_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
     private void initFields() {
       rpcVersion_ = 0;
+      status_ = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.SUCCESS;
+      errorId_ = "";
+      errorMessage_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4060,6 +4358,15 @@ public final class UserProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(2, rpcVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(3, status_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(4, getErrorIdBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(5, getErrorMessageBytes());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4073,6 +4380,18 @@ public final class UserProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, rpcVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(3, status_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, getErrorIdBytes());
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getErrorMessageBytes());
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4191,6 +4510,12 @@ public final class UserProtos {
         super.clear();
         rpcVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000001);
+        status_ = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.SUCCESS;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        errorId_ = "";
+        bitField0_ = (bitField0_ & ~0x00000004);
+        errorMessage_ = "";
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -4223,6 +4548,18 @@ public final class UserProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.rpcVersion_ = rpcVersion_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.status_ = status_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.errorId_ = errorId_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.errorMessage_ = errorMessage_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4242,6 +4579,19 @@ public final class UserProtos {
         if (other.hasRpcVersion()) {
           setRpcVersion(other.getRpcVersion());
         }
+        if (other.hasStatus()) {
+          setStatus(other.getStatus());
+        }
+        if (other.hasErrorId()) {
+          bitField0_ |= 0x00000004;
+          errorId_ = other.errorId_;
+          onChanged();
+        }
+        if (other.hasErrorMessage()) {
+          bitField0_ |= 0x00000008;
+          errorMessage_ = other.errorMessage_;
+          onChanged();
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4302,6 +4652,190 @@ public final class UserProtos {
         return this;
       }
 
+      // optional .exec.user.HandshakeStatus status = 3;
+      private org.apache.drill.exec.proto.UserProtos.HandshakeStatus status_ = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.SUCCESS;
+      /**
+       * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+       */
+      public boolean hasStatus() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+       */
+      public org.apache.drill.exec.proto.UserProtos.HandshakeStatus getStatus() {
+        return status_;
+      }
+      /**
+       * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+       */
+      public Builder setStatus(org.apache.drill.exec.proto.UserProtos.HandshakeStatus value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        status_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .exec.user.HandshakeStatus status = 3;</code>
+       */
+      public Builder clearStatus() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        status_ = org.apache.drill.exec.proto.UserProtos.HandshakeStatus.SUCCESS;
+        onChanged();
+        return this;
+      }
+
+      // optional string errorId = 4;
+      private java.lang.Object errorId_ = "";
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public boolean hasErrorId() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public java.lang.String getErrorId() {
+        java.lang.Object ref = errorId_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          errorId_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public com.google.protobuf.ByteString
+          getErrorIdBytes() {
+        java.lang.Object ref = errorId_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          errorId_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public Builder setErrorId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        errorId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public Builder clearErrorId() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        errorId_ = getDefaultInstance().getErrorId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string errorId = 4;</code>
+       */
+      public Builder setErrorIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        errorId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string errorMessage = 5;
+      private java.lang.Object errorMessage_ = "";
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public boolean hasErrorMessage() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public java.lang.String getErrorMessage() {
+        java.lang.Object ref = errorMessage_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          errorMessage_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public com.google.protobuf.ByteString
+          getErrorMessageBytes() {
+        java.lang.Object ref = errorMessage_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          errorMessage_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public Builder setErrorMessage(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        errorMessage_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public Builder clearErrorMessage() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        errorMessage_ = getDefaultInstance().getErrorMessage();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string errorMessage = 5;</code>
+       */
+      public Builder setErrorMessageBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000008;
+        errorMessage_ = value;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.user.BitToUserHandshake)
     }
 
@@ -4367,15 +4901,20 @@ public final class UserProtos {
       "imum_responses\030\002 \001(\005\"q\n\010RunQuery\0221\n\014resu" +
       "lts_mode\030\001 \001(\0162\033.exec.user.QueryResultsM" +
       "ode\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryTyp" +
-      "e\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022\023\n" +
-      "\013rpc_version\030\002 \001(\005*\310\001\n\007RpcType\022\r\n\tHANDSH" +
-      "AKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY" +
-      "\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020" +
-      "\005\022\016\n\nQUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n\022R",
-      "EQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_LIS" +
-      "T\020\t\022\020\n\014QUERY_RESULT\020\n*#\n\020QueryResultsMod" +
-      "e\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.e" +
-      "xec.protoB\nUserProtosH\001"
+      "e\022\014\n\004plan\030\003 \001(\t\"|\n\022BitToUserHandshake\022\023\n" +
+      "\013rpc_version\030\002 \001(\005\022*\n\006status\030\003 \001(\0162\032.exe" +
+      "c.user.HandshakeStatus\022\017\n\007errorId\030\004 \001(\t\022" +
+      "\024\n\014errorMessage\030\005 \001(\t*\310\001\n\007RpcType\022\r\n\tHAN" +
+      "DSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QU",
+      "ERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESUL" +
+      "TS\020\005\022\016\n\nQUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026" +
+      "\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_" +
+      "LIST\020\t\022\020\n\014QUERY_RESULT\020\n*#\n\020QueryResults" +
+      "Mode\022\017\n\013STREAM_FULL\020\001*^\n\017HandshakeStatus" +
+      "\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_VERSION_MISMATCH\020\002\022" +
+      "\017\n\013AUTH_FAILED\020\003\022\023\n\017UNKNOWN_FAILURE\020\004B+\n" +
+      "\033org.apache.drill.exec.protoB\nUserProtos" +
+      "H\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -4417,7 +4956,7 @@ public final class UserProtos {
           internal_static_exec_user_BitToUserHandshake_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_user_BitToUserHandshake_descriptor,
-              new java.lang.String[] { "RpcVersion", });
+              new java.lang.String[] { "RpcVersion", "Status", "ErrorId", "ErrorMessage", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
index 813eb1c..20c6de1 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/BitToUserHandshake.java
@@ -48,6 +48,9 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
 
     
     private int rpcVersion;
+    private HandshakeStatus status;
+    private String errorId;
+    private String errorMessage;
 
     public BitToUserHandshake()
     {
@@ -69,6 +72,45 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
         return this;
     }
 
+    // status
+
+    public HandshakeStatus getStatus()
+    {
+        return status == null ? HandshakeStatus.SUCCESS : status;
+    }
+
+    public BitToUserHandshake setStatus(HandshakeStatus status)
+    {
+        this.status = status;
+        return this;
+    }
+
+    // errorId
+
+    public String getErrorId()
+    {
+        return errorId;
+    }
+
+    public BitToUserHandshake setErrorId(String errorId)
+    {
+        this.errorId = errorId;
+        return this;
+    }
+
+    // errorMessage
+
+    public String getErrorMessage()
+    {
+        return errorMessage;
+    }
+
+    public BitToUserHandshake setErrorMessage(String errorMessage)
+    {
+        this.errorMessage = errorMessage;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -126,6 +168,15 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
                 case 2:
                     message.rpcVersion = input.readInt32();
                     break;
+                case 3:
+                    message.status = HandshakeStatus.valueOf(input.readEnum());
+                    break;
+                case 4:
+                    message.errorId = input.readString();
+                    break;
+                case 5:
+                    message.errorMessage = input.readString();
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -137,6 +188,15 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
     {
         if(message.rpcVersion != 0)
             output.writeInt32(2, message.rpcVersion, false);
+
+        if(message.status != null)
+             output.writeEnum(3, message.status.number, false);
+
+        if(message.errorId != null)
+            output.writeString(4, message.errorId, false);
+
+        if(message.errorMessage != null)
+            output.writeString(5, message.errorMessage, false);
     }
 
     public String getFieldName(int number)
@@ -144,6 +204,9 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
         switch(number)
         {
             case 2: return "rpcVersion";
+            case 3: return "status";
+            case 4: return "errorId";
+            case 5: return "errorMessage";
             default: return null;
         }
     }
@@ -158,6 +221,9 @@ public final class BitToUserHandshake implements Externalizable, Message<BitToUs
     static
     {
         __fieldMap.put("rpcVersion", 2);
+        __fieldMap.put("status", 3);
+        __fieldMap.put("errorId", 4);
+        __fieldMap.put("errorMessage", 5);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/protocol/src/main/java/org/apache/drill/exec/proto/beans/HandshakeStatus.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/HandshakeStatus.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/HandshakeStatus.java
new file mode 100644
index 0000000..3b6a7a1
--- /dev/null
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/HandshakeStatus.java
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+public enum HandshakeStatus implements com.dyuproject.protostuff.EnumLite<HandshakeStatus>
+{
+    SUCCESS(1),
+    RPC_VERSION_MISMATCH(2),
+    AUTH_FAILED(3),
+    UNKNOWN_FAILURE(4);
+    
+    public final int number;
+    
+    private HandshakeStatus (int number)
+    {
+        this.number = number;
+    }
+    
+    public int getNumber()
+    {
+        return number;
+    }
+    
+    public static HandshakeStatus valueOf(int number)
+    {
+        switch(number) 
+        {
+            case 1: return SUCCESS;
+            case 2: return RPC_VERSION_MISMATCH;
+            case 3: return AUTH_FAILED;
+            case 4: return UNKNOWN_FAILURE;
+            default: return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/a218ee3f/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index a083c59..59e22ae 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -63,6 +63,16 @@ enum QueryResultsMode {
   // QUERY_FOR_STATUS = 3; // Client will need to query for status of query.
 }
 
+enum HandshakeStatus {
+  SUCCESS = 1; // Handshake is successful (including authentication if any)
+  RPC_VERSION_MISMATCH = 2; // Client and Server RPC versions are different
+  AUTH_FAILED = 3; // User authentication failed
+  UNKNOWN_FAILURE = 4; // Unknown failure, refer to the error message for more details
+}
+
 message BitToUserHandshake {
   optional int32 rpc_version = 2;
+  optional HandshakeStatus status = 3;
+  optional string errorId = 4;
+  optional string errorMessage = 5;
 }


[5/9] drill git commit: DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData

Posted by pa...@apache.org.
DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData


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

Branch: refs/heads/master
Commit: 1d9d82b001810605e3f94ab3a5517dc0ed739715
Parents: 10be89f
Author: adeneche <ad...@gmail.com>
Authored: Fri Mar 20 12:39:15 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Apr 3 18:27:50 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/hbase/BaseHBaseTest.java   |   10 +-
 .../drill/hbase/TestHBaseCFAsJSONString.java    |    5 +-
 .../apache/drill/exec/fn/hive/TestHiveUDFs.java |   12 +-
 .../drill/exec/fn/hive/TestSampleHiveUDFs.java  |    4 +-
 .../apache/drill/exec/client/DrillClient.java   |   34 +-
 .../exec/client/PrintingResultsListener.java    |   31 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   67 +-
 .../impl/materialize/QueryWritableBatch.java    |    8 +-
 .../impl/materialize/RecordMaterializer.java    |    2 +-
 .../materialize/VectorRecordMaterializer.java   |    8 +-
 .../drill/exec/rpc/user/QueryDataBatch.java     |   62 +
 .../drill/exec/rpc/user/QueryResultBatch.java   |   62 -
 .../drill/exec/rpc/user/QueryResultHandler.java |  261 +--
 .../apache/drill/exec/rpc/user/UserClient.java  |   14 +-
 .../exec/rpc/user/UserResultsListener.java      |   28 +-
 .../drill/exec/rpc/user/UserRpcConfig.java      |    4 +-
 .../apache/drill/exec/rpc/user/UserServer.java  |   16 +-
 .../drill/exec/server/rest/QueryWrapper.java    |   15 +-
 .../apache/drill/exec/work/foreman/Foreman.java |    4 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   38 +-
 .../java/org/apache/drill/DrillTestWrapper.java |   36 +-
 .../java/org/apache/drill/PlanTestBase.java     |    6 +-
 .../java/org/apache/drill/QueryTestUtil.java    |    8 +-
 .../org/apache/drill/SingleRowListener.java     |   43 +-
 .../drill/exec/TestQueriesOnLargeFile.java      |   10 +-
 .../exec/client/DrillClientSystemTest.java      |   10 +-
 .../exec/fn/impl/TestAggregateFunction.java     |    8 +-
 .../drill/exec/fn/impl/TestDateFunctions.java   |    8 +-
 .../drill/exec/fn/impl/TestMultiInputAdd.java   |   11 +-
 .../exec/fn/impl/TestNewAggregateFunctions.java |    8 +-
 .../physical/impl/TestBroadcastExchange.java    |   10 +-
 .../exec/physical/impl/TestCastFunctions.java   |    8 +-
 .../physical/impl/TestCastVarCharToBigInt.java  |   10 +-
 .../physical/impl/TestConvertFunctions.java     |   11 +-
 .../drill/exec/physical/impl/TestDecimal.java   |   38 +-
 .../impl/TestDistributedFragmentRun.java        |   18 +-
 .../physical/impl/TestExtractFunctions.java     |    8 +-
 .../physical/impl/TestHashToRandomExchange.java |    6 +-
 .../exec/physical/impl/TestOptiqPlans.java      |   22 +-
 .../physical/impl/TestReverseImplicitCast.java  |    8 +-
 .../physical/impl/TestSimpleFragmentRun.java    |   10 +-
 .../exec/physical/impl/TestUnionExchange.java   |    6 +-
 .../exec/physical/impl/TopN/TestSimpleTopN.java |    8 +-
 .../exec/physical/impl/join/TestHashJoin.java   |   26 +-
 .../exec/physical/impl/join/TestMergeJoin.java  |   18 +-
 .../impl/join/TestMergeJoinMulCondition.java    |   14 +-
 .../impl/mergereceiver/TestMergingReceiver.java |   14 +-
 .../TestOrderedPartitionExchange.java           |    6 +-
 .../physical/impl/writer/TestParquetWriter.java |    2 +-
 .../exec/physical/impl/writer/TestWriter.java   |   12 +-
 .../impl/xsort/TestSimpleExternalSort.java      |   26 +-
 .../drill/exec/record/vector/TestDateTypes.java |   40 +-
 .../exec/server/TestDrillbitResilience.java     |   10 +-
 .../store/parquet/ParquetRecordReaderTest.java  |   10 +-
 .../store/parquet/ParquetResultListener.java    |   20 +-
 .../store/parquet/TestParquetPhysicalPlan.java  |   18 +-
 .../drill/exec/store/text/TestTextColumn.java   |   10 +-
 .../exec/store/text/TextRecordReaderTest.java   |    6 +-
 .../fn/TestJsonReaderWithSparseFiles.java       |    8 +-
 .../complex/writer/TestComplexToJson.java       |    8 +-
 .../vector/complex/writer/TestJsonReader.java   |   22 +-
 .../exec/work/batch/TestSpoolingBuffer.java     |    6 +-
 .../java/org/apache/drill/jdbc/DrillCursor.java |    4 +-
 .../org/apache/drill/jdbc/DrillResultSet.java   |   43 +-
 .../drill/exec/proto/SchemaUserBitShared.java   |  195 +-
 .../apache/drill/exec/proto/UserBitShared.java  | 1948 ++++++++----------
 .../org/apache/drill/exec/proto/UserProtos.java |   47 +-
 .../drill/exec/proto/beans/QueryData.java       |  211 ++
 .../drill/exec/proto/beans/QueryResult.java     |  194 +-
 .../apache/drill/exec/proto/beans/RpcType.java  |    8 +-
 protocol/src/main/protobuf/User.proto           |    8 +-
 protocol/src/main/protobuf/UserBitShared.proto  |   22 +-
 72 files changed, 1898 insertions(+), 2054 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
index b955d3b..df83c56 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/BaseHBaseTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.hbase.HBaseStoragePlugin;
 import org.apache.drill.exec.store.hbase.HBaseStoragePluginConfig;
@@ -76,22 +76,22 @@ public class BaseHBaseTest extends BaseTestQuery {
 
   protected void runHBasePhysicalVerifyCount(String planFile, String tableName, int expectedRowCount) throws Exception{
     String physicalPlan = getPlanText(planFile, tableName);
-    List<QueryResultBatch> results = testPhysicalWithResults(physicalPlan);
+    List<QueryDataBatch> results = testPhysicalWithResults(physicalPlan);
     printResultAndVerifyRowCount(results, expectedRowCount);
   }
 
-  protected List<QueryResultBatch> runHBaseSQLlWithResults(String sql) throws Exception {
+  protected List<QueryDataBatch> runHBaseSQLlWithResults(String sql) throws Exception {
     sql = canonizeHBaseSQL(sql);
     System.out.println("Running query:\n" + sql);
     return testSqlWithResults(sql);
   }
 
   protected void runHBaseSQLVerifyCount(String sql, int expectedRowCount) throws Exception{
-    List<QueryResultBatch> results = runHBaseSQLlWithResults(sql);
+    List<QueryDataBatch> results = runHBaseSQLlWithResults(sql);
     printResultAndVerifyRowCount(results, expectedRowCount);
   }
 
-  private void printResultAndVerifyRowCount(List<QueryResultBatch> results, int expectedRowCount) throws SchemaChangeException {
+  private void printResultAndVerifyRowCount(List<QueryDataBatch> results, int expectedRowCount) throws SchemaChangeException {
     int rowCount = printResult(results);
     if (expectedRowCount != -1) {
       Assert.assertEquals(expectedRowCount, rowCount);

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseCFAsJSONString.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseCFAsJSONString.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseCFAsJSONString.java
index 6fe1525..7873b80 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseCFAsJSONString.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseCFAsJSONString.java
@@ -21,7 +21,8 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -49,7 +50,7 @@ public class TestHBaseCFAsJSONString extends BaseHBaseTest {
   @Test
   public void testColumnFamiliesAsJSONString() throws Exception {
     setColumnWidths(new int[] {112, 12});
-    List<QueryResultBatch> resultList = runHBaseSQLlWithResults("SELECT f, f2 FROM hbase.`[TABLE_NAME]` tableName LIMIT 1");
+    List<QueryDataBatch> resultList = runHBaseSQLlWithResults("SELECT f, f2 FROM hbase.`[TABLE_NAME]` tableName LIMIT 1");
     printResult(resultList);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
index e134aac..3ce9a6d 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestHiveUDFs.java
@@ -24,10 +24,8 @@ import java.util.List;
 
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.Float4Vector;
-import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
 import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.NullableVar16CharVector;
@@ -44,10 +42,10 @@ public class TestHiveUDFs extends BaseTestQuery {
 
     int numRecords = 0;
     String planString = Resources.toString(Resources.getResource("functions/hive/GenericUDF.json"), Charsets.UTF_8);
-    List<QueryResultBatch> results = testPhysicalWithResults(planString);
+    List<QueryDataBatch> results = testPhysicalWithResults(planString);
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
-    for (QueryResultBatch result : results) {
+    for (QueryDataBatch result : results) {
       batchLoader.load(result.getHeader().getDef(), result.getData());
       if (batchLoader.getRecordCount() <= 0) {
         result.release();
@@ -115,10 +113,10 @@ public class TestHiveUDFs extends BaseTestQuery {
   public void testUDF() throws Throwable {
     int numRecords = 0;
     String planString = Resources.toString(Resources.getResource("functions/hive/UDF.json"), Charsets.UTF_8);
-    List<QueryResultBatch> results = testPhysicalWithResults(planString);
+    List<QueryDataBatch> results = testPhysicalWithResults(planString);
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
-    for (QueryResultBatch result : results) {
+    for (QueryDataBatch result : results) {
       batchLoader.load(result.getHeader().getDef(), result.getData());
       if (batchLoader.getRecordCount() <= 0) {
         result.release();

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestSampleHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestSampleHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestSampleHiveUDFs.java
index 9ef766f..f4b6351 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestSampleHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestSampleHiveUDFs.java
@@ -22,14 +22,14 @@ import static org.junit.Assert.assertTrue;
 import java.util.List;
 
 import org.apache.drill.exec.hive.HiveTestBase;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestSampleHiveUDFs extends HiveTestBase {
 
   private void helper(String query, String expected) throws Exception {
-    List<QueryResultBatch> results = testSqlWithResults(query);
+    List<QueryDataBatch> results = testSqlWithResults(query);
     String actual = getResultString(results, ",");
     assertTrue(String.format("Result:\n%s\ndoes not match:\n%s", actual, expected), expected.equals(actual));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/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 6d4c86c..650a2eb 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
@@ -40,7 +40,6 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos;
 import org.apache.drill.exec.proto.UserProtos.Property;
@@ -54,7 +53,7 @@ import org.apache.drill.exec.rpc.RpcConnectionHandler;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.TransportCheck;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserClient;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 
@@ -66,7 +65,7 @@ import com.google.common.util.concurrent.SettableFuture;
  * String into ByteBuf.
  */
 public class DrillClient implements Closeable, ConnectionThrottle {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
 
   DrillConfig config;
   private UserClient client;
@@ -249,7 +248,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
    * @return a handle for the query result
    * @throws RpcException
    */
-  public List<QueryResultBatch> runQuery(QueryType type, String plan) throws RpcException {
+  public List<QueryDataBatch> runQuery(QueryType type, String plan) throws RpcException {
     UserProtos.RunQuery query = newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build();
     ListHoldingResultsListener listener = new ListHoldingResultsListener(query);
     client.submitQuery(listener, query);
@@ -294,8 +293,8 @@ public class DrillClient implements Closeable, ConnectionThrottle {
   }
 
   private class ListHoldingResultsListener implements UserResultsListener {
-    private Vector<QueryResultBatch> results = new Vector<>();
-    private SettableFuture<List<QueryResultBatch>> future = SettableFuture.create();
+    private Vector<QueryDataBatch> results = new Vector<>();
+    private SettableFuture<List<QueryDataBatch>> future = SettableFuture.create();
     private UserProtos.RunQuery query ;
 
     public ListHoldingResultsListener(UserProtos.RunQuery query) {
@@ -321,6 +320,11 @@ public class DrillClient implements Closeable, ConnectionThrottle {
       }
     }
 
+    @Override
+    public void queryCompleted() {
+      future.set(results);
+    }
+
     private void fail(Exception ex) {
       logger.debug("Submission failed.", ex);
       future.setException(ex);
@@ -328,24 +332,12 @@ public class DrillClient implements Closeable, ConnectionThrottle {
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      logger.debug(
-          "Result arrived:  Query state: {}.  Is last chunk: {}.  Result: {}",
-          result.getHeader().getQueryState(),
-          result.getHeader().getIsLastChunk(),
-          result );
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+      logger.debug("Result arrived:  Result: {}", result );
       results.add(result);
-      if (result.getHeader().getIsLastChunk()) {
-        future.set(results);
-      }
-      else {
-        assert QueryState.PENDING == result.getHeader().getQueryState()
-            : "For non-last chunk, expected query state of PENDING but got "
-              + result.getHeader().getQueryState();
-      }
     }
 
-    public List<QueryResultBatch> getResults() throws RpcException{
+    public List<QueryDataBatch> getResults() throws RpcException{
       try {
         return future.get();
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
index 926e703..98948af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
@@ -20,16 +20,18 @@ package org.apache.drill.exec.client;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.util.VectorUtil;
 
@@ -58,12 +60,21 @@ public class PrintingResultsListener implements UserResultsListener {
   }
 
   @Override
-  public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-    int rows = result.getHeader().getRowCount();
-    if (result.getData() != null) {
-      count.addAndGet(rows);
+  public void queryCompleted() {
+    allocator.close();
+    latch.countDown();
+    System.out.println("Total rows returned: " + count.get());
+  }
+
+  @Override
+  public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+    final QueryData header = result.getHeader();
+    final DrillBuf data = result.getData();
+
+    if (data != null) {
+      count.addAndGet(header.getRowCount());
       try {
-        loader.load(result.getHeader().getDef(), result.getData());
+        loader.load(header.getDef(), data);
       } catch (SchemaChangeException e) {
         submissionFailed(new RpcException(e));
       }
@@ -82,15 +93,7 @@ public class PrintingResultsListener implements UserResultsListener {
       loader.clear();
     }
 
-    boolean isLastChunk = result.getHeader().getIsLastChunk();
     result.release();
-
-    if (isLastChunk) {
-      allocator.close();
-      latch.countDown();
-      System.out.println("Total rows returned: " + count.get());
-    }
-
   }
 
   public int await() throws Exception {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 404c453..8038527 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -31,8 +31,7 @@ import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
 import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
@@ -44,7 +43,7 @@ import org.apache.drill.exec.work.ErrorHelper;
 import com.google.common.base.Preconditions;
 
 public class ScreenCreator implements RootCreator<Screen>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
 
 
 
@@ -67,6 +66,8 @@ public class ScreenCreator implements RootCreator<Screen>{
     final UserClientConnection connection;
     private RecordMaterializer materializer;
 
+    private boolean firstBatch = true;
+
     public enum Metric implements MetricDef {
       BYTES_SENT;
 
@@ -96,67 +97,45 @@ public class ScreenCreator implements RootCreator<Screen>{
       IterOutcome outcome = next(incoming);
       logger.trace("Screen Outcome {}", outcome);
       switch (outcome) {
-      case STOP: {
+      case STOP:
         this.internalStop();
-        boolean verbose = context.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
-        QueryResult header = QueryResult.newBuilder() //
-              .setQueryId(context.getHandle().getQueryId()) //
-              .setRowCount(0) //
-              .setQueryState(QueryState.FAILED)
-              .addError(ErrorHelper.logAndConvertMessageError(context.getIdentity(), "Query stopped.",
-                context.getFailureCause(), logger, verbose))
-              .setDef(RecordBatchDef.getDefaultInstance()) //
-              .setIsLastChunk(true) //
-              .build();
-          QueryWritableBatch batch = new QueryWritableBatch(header);
+        return false;
+      case NONE:
+        if (firstBatch) {
+          // this is the only data message sent to the client and may contain the schema
+          this.internalStop();
+          QueryWritableBatch batch;
+          QueryData header = QueryData.newBuilder() //
+            .setQueryId(context.getHandle().getQueryId()) //
+            .setRowCount(0) //
+            .setDef(RecordBatchDef.getDefaultInstance()) //
+            .build();
+          batch = new QueryWritableBatch(header);
+
           stats.startWait();
           try {
-            connection.sendResult(listener, batch);
+            connection.sendData(listener, batch);
           } finally {
             stats.stopWait();
           }
+          firstBatch = false; // we don't really need to set this. But who knows!
           sendCount.increment();
-
-          return false;
-      }
-      case NONE: {
-        this.internalStop();
-        QueryWritableBatch batch;
-        //TODO: At some point we should make this the last message.
-        //For the moment though, to detect memory leaks, we need to delay sending the
-        //COMPLETED message until the Foreman calls cleanup.
-        QueryResult header = QueryResult.newBuilder() //
-            .setQueryId(context.getHandle().getQueryId()) //
-            .setRowCount(0) //
-            //.setQueryState(QueryState.COMPLETED) //
-            .setDef(RecordBatchDef.getDefaultInstance()) //
-            .setIsLastChunk(true) //
-            .build();
-        batch = new QueryWritableBatch(header);
-        stats.startWait();
-        try {
-          connection.sendResult(listener, batch);
-        } finally {
-          stats.stopWait();
         }
-        sendCount.increment();
 
         return false;
-      }
       case OK_NEW_SCHEMA:
         materializer = new VectorRecordMaterializer(context, incoming);
         //$FALL-THROUGH$
       case OK:
-//        context.getStats().batchesCompleted.inc(1);
-//        context.getStats().recordsCompleted.inc(incoming.getRecordCount());
-        QueryWritableBatch batch = materializer.convertNext(false);
+        QueryWritableBatch batch = materializer.convertNext();
         updateStats(batch);
         stats.startWait();
         try {
-          connection.sendResult(listener, batch);
+          connection.sendData(listener, batch);
         } finally {
           stats.stopWait();
         }
+        firstBatch = false;
         sendCount.increment();
 
         return true;

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
index 2a59e22..44a3489 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
@@ -21,15 +21,15 @@ import io.netty.buffer.ByteBuf;
 
 import java.util.Arrays;
 
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 
 public class QueryWritableBatch {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWritableBatch.class);
 
-  private final QueryResult header;
+  private final QueryData header;
   private final ByteBuf[] buffers;
 
-  public QueryWritableBatch(QueryResult header, ByteBuf... buffers) {
+  public QueryWritableBatch(QueryData header, ByteBuf... buffers) {
     this.header = header;
     this.buffers = buffers;
   }
@@ -46,7 +46,7 @@ public class QueryWritableBatch {
     return n;
   }
 
-  public QueryResult getHeader() {
+  public QueryData getHeader() {
     return header;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
index 221fc34..acb17ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
@@ -20,6 +20,6 @@ package org.apache.drill.exec.physical.impl.materialize;
 
 public interface RecordMaterializer {
 
-  public QueryWritableBatch convertNext(boolean isLast);
+  public QueryWritableBatch convertNext();
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
index cc1b3bf..3933ddd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.materialize;
 
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.WritableBatch;
@@ -41,14 +41,14 @@ public class VectorRecordMaterializer implements RecordMaterializer{
 //    }
   }
 
-  public QueryWritableBatch convertNext(boolean isLast) {
+  public QueryWritableBatch convertNext() {
     //batch.getWritableBatch().getDef().getRecordCount()
     WritableBatch w = batch.getWritableBatch();
 
-    QueryResult header = QueryResult.newBuilder() //
+    QueryData header = QueryData.newBuilder() //
         .setQueryId(queryId) //
         .setRowCount(batch.getRecordCount()) //
-        .setDef(w.getDef()).setIsLastChunk(isLast).build();
+        .setDef(w.getDef()).build();
     QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
     return batch;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java
new file mode 100644
index 0000000..914bd00
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryDataBatch.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.rpc.user;
+
+import io.netty.buffer.DrillBuf;
+
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
+
+public class QueryDataBatch {
+//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultBatch.class);
+
+  private final QueryData header;
+  private final DrillBuf data;
+
+  public QueryDataBatch(QueryData header, DrillBuf data) {
+//    logger.debug("New Result Batch with header {} and data {}", header, data);
+    this.header = header;
+    this.data = data;
+    if (this.data != null) {
+      data.retain();
+    }
+  }
+
+  public QueryData getHeader() {
+    return header;
+  }
+
+  public DrillBuf getData() {
+    return data;
+  }
+
+  public boolean hasData() {
+    return data != null;
+  }
+
+  public void release() {
+    if (data != null) {
+      data.release();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "QueryResultBatch [header=" + header + ", data=" + data + "]";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
deleted file mode 100644
index ab4c9ef..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.rpc.user;
-
-import io.netty.buffer.DrillBuf;
-
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
-
-public class QueryResultBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultBatch.class);
-
-  private final QueryResult header;
-  private final DrillBuf data;
-
-  public QueryResultBatch(QueryResult header, DrillBuf data) {
-//    logger.debug("New Result Batch with header {} and data {}", header, data);
-    this.header = header;
-    this.data = data;
-    if (this.data != null) {
-      data.retain();
-    }
-  }
-
-  public QueryResult getHeader() {
-    return header;
-  }
-
-  public DrillBuf getData() {
-    return data;
-  }
-
-  public boolean hasData() {
-    return data != null;
-  }
-
-  public void release() {
-    if (data != null) {
-      data.release();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "QueryResultBatch [header=" + header + ", data=" + data + "]";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
index c05b127..a1be83b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
@@ -20,17 +20,13 @@ package org.apache.drill.exec.rpc.user;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 
-import javax.annotation.Nullable;
-
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
 import org.apache.drill.exec.rpc.RpcBus;
@@ -65,179 +61,137 @@ public class QueryResultHandler {
   private final ConcurrentMap<QueryId, UserResultsListener> queryIdToResultsListenersMap =
       Maps.newConcurrentMap();
 
-  /**
-   * Any is-last-chunk batch being deferred until the next batch
-   * (normally one with COMPLETED) arrives, per active query.
-   * <ul>
-   *   <li>Last-chunk batch is added (and not passed on) when it arrives.</li>
-   *   <li>Last-chunk batch is removed (and passed on) when next batch arrives
-   *       and has state {@link QueryState.COMPLETED}.</li>
-   *   <li>Last-chunk batch is removed (and not passed on) when next batch
-   *       arrives and has state {@link QueryState.CANCELED} or
-   *       {@link QueryState.FAILED}.</li>
-   * </ul>
-   */
-  private final Map<QueryId, QueryResultBatch> queryIdToDeferredLastChunkBatchesMap =
-      new ConcurrentHashMap<>();
-
-
   public RpcOutcomeListener<QueryId> getWrappedListener(UserResultsListener resultsListener) {
     return new SubmissionListener(resultsListener);
   }
 
   /**
-   * Maps internal low-level API protocol to {@link UserResultsListener}-level
-   * API protocol, deferring sending is-last-chunk batches until (internal)
-   * COMPLETED batch.
+   * Maps internal low-level API protocol to {@link UserResultsListener}-level API protocol.
+   * handles data result messages
    */
-  public void batchArrived( ConnectionThrottle throttle,
-                            ByteBuf pBody, ByteBuf dBody ) throws RpcException {
+  public void resultArrived( ByteBuf pBody ) throws RpcException {
     final QueryResult queryResult = RpcBus.get( pBody, QueryResult.PARSER );
-    // Current batch coming in.  (Not necessarily passed along now or ever.)
-    final QueryResultBatch inputBatch = new QueryResultBatch( queryResult,
-                                                              (DrillBuf) dBody );
 
     final QueryId queryId = queryResult.getQueryId();
-    final QueryState queryState = inputBatch.getHeader().getQueryState();
+    final QueryState queryState = queryResult.getQueryState();
+
+    logger.debug( "resultArrived: queryState: {}, queryId = {}", queryState, queryId );
 
-    logger.debug( "batchArrived: isLastChunk: {}, queryState: {}, queryId = {}",
-                  inputBatch.getHeader().getIsLastChunk(), queryState, queryId );
-    logger.trace( "batchArrived: currentBatch = {}", inputBatch );
+    assert queryResult.hasQueryState() : "received query result without QueryState";
 
-    final boolean isFailureBatch    = QueryState.FAILED    == queryState;
-    final boolean isCompletionBatch = QueryState.COMPLETED == queryState;
-    final boolean isLastChunkBatchToDelay =
-        inputBatch.getHeader().getIsLastChunk() && QueryState.PENDING == queryState;
-    final boolean isTerminalBatch;
+    final boolean isFailureResult    = QueryState.FAILED    == queryState;
+    // CANCELED queries are handled the same way as COMPLETED
+    final boolean isTerminalResult;
     switch ( queryState ) {
       case PENDING:
-         isTerminalBatch = false;
-         break;
+        isTerminalResult = false;
+        break;
       case FAILED:
       case CANCELED:
       case COMPLETED:
-        isTerminalBatch = true;
+        isTerminalResult = true;
         break;
       default:
         logger.error( "Unexpected/unhandled QueryState " + queryState
-                      + " (for query " + queryId +  ")" );
-        isTerminalBatch = false;
+          + " (for query " + queryId +  ")" );
+        isTerminalResult = false;
         break;
     }
-    assert isFailureBatch || inputBatch.getHeader().getErrorCount() == 0
-        : "Error count for the query batch is non-zero but QueryState != FAILED";
 
-    UserResultsListener resultsListener = queryIdToResultsListenersMap.get( queryId );
-    logger.trace( "For QueryId [{}], retrieved results listener {}", queryId,
-                  resultsListener );
-    if ( null == resultsListener ) {
-      // WHO?? didn't get query ID response and set submission listener yet,
-      // so install a buffering listener for now
+    assert isFailureResult || queryResult.getErrorCount() == 0
+      : "Error count for the query batch is non-zero but QueryState != FAILED";
 
-      BufferingResultsListener bl = new BufferingResultsListener();
-      resultsListener = queryIdToResultsListenersMap.putIfAbsent( queryId, bl );
-      // If we had a successful insertion, use that reference.  Otherwise, just
-      // throw away the new buffering listener.
-      if ( null == resultsListener ) {
-        resultsListener = bl;
-      }
-      // TODO:  Is there a more direct way to detect a Query ID in whatever
-      // state this string comparison detects?
-      if ( queryId.toString().equals( "" ) ) {
-        failAll();
-      }
-    }
+    UserResultsListener resultsListener = newUserResultsListener(queryId);
 
     try {
-      if (isFailureBatch) {
+      if (isFailureResult) {
         // Failure case--pass on via submissionFailed(...).
 
+        String message = buildErrorMessage(queryResult);
+        resultsListener.submissionFailed(new RpcException(message));
+        // Note: Listener is removed in finally below.
+      } else if (isTerminalResult) {
+        // A successful completion/canceled case--pass on via resultArrived
+
         try {
-          String message = buildErrorMessage(inputBatch);
-          resultsListener.submissionFailed(new RpcException(message));
+          resultsListener.queryCompleted();
+        } catch ( Exception e ) {
+          resultsListener.submissionFailed(new RpcException(e));
         }
-        finally {
-          inputBatch.release();
-        }
-        // Note: Listener and any delayed batch are removed in finally below.
       } else {
-        // A successful (data, completion, or cancelation) case--pass on via
-        // resultArrived, delaying any last-chunk batches until following
-        // COMPLETED batch and omitting COMPLETED batch.
-
-        // If is last-chunk batch, save until next batch for query (normally a
-        // COMPLETED batch) comes in:
-        if ( isLastChunkBatchToDelay ) {
-          // We have a (non-failure) is-last-chunk batch--defer it until we get
-          // the query's COMPLETED batch.
-
-          QueryResultBatch expectNone;
-          assert null == ( expectNone =
-                           queryIdToDeferredLastChunkBatchesMap.get( queryId ) )
-              : "Already have pending last-batch QueryResultBatch " + expectNone
-                + " (at receiving last-batch QueryResultBatch " + inputBatch
-                + ") for query " + queryId;
-          queryIdToDeferredLastChunkBatchesMap.put( queryId, inputBatch );
-          // Can't release batch now; will release at terminal batch in
-          // finally below.
-        } else {
-          // We have a batch triggering sending out a batch (maybe same one,
-          // maybe deferred one.
-
-          // Batch to send out in response to current batch.
-          final QueryResultBatch outputBatch;
-          if ( isCompletionBatch ) {
-            // We have a COMPLETED batch--we should have a saved is-last-chunk
-            // batch, and we must pass that on now (that we've seen COMPLETED).
-
-            outputBatch = queryIdToDeferredLastChunkBatchesMap.get( queryId );
-            assert null != outputBatch
-                : "No pending last-batch QueryResultsBatch saved, at COMPLETED"
-                + " QueryResultsBatch " + inputBatch + " for query " + queryId;
-          } else {
-            // We have a non--last-chunk PENDING batch or a CANCELED
-            // batch--pass it on.
-            outputBatch = inputBatch;
-          }
-          // Note to release input batch if it's not the batch we're sending out.
-          final boolean releaseInputBatch = outputBatch != inputBatch;
-
-          try {
-            resultsListener.resultArrived( outputBatch, throttle );
-            // That releases outputBatch if successful.
-          } catch ( Exception e ) {
-            outputBatch.release();
-            resultsListener.submissionFailed(new RpcException(e));
-          }
-          finally {
-            if ( releaseInputBatch ) {
-              inputBatch.release();
-            }
-          }
-        }
+        logger.warn("queryState {} was ignored", queryState);
       }
     } finally {
-      if ( isTerminalBatch ) {
-        // Remove and release any deferred is-last-chunk batch:
-        QueryResultBatch anyUnsentLastChunkBatch =
-             queryIdToDeferredLastChunkBatchesMap.remove( queryId );
-        if ( null != anyUnsentLastChunkBatch ) {
-          anyUnsentLastChunkBatch.release();
-        }
-
-       // TODO:  What exactly are we checking for?  How should we really check
+      if ( isTerminalResult ) {
+        // TODO:  What exactly are we checking for?  How should we really check
         // for it?
         if ( (! ( resultsListener instanceof BufferingResultsListener )
-             || ((BufferingResultsListener) resultsListener).output != null ) ) {
+          || ((BufferingResultsListener) resultsListener).output != null ) ) {
           queryIdToResultsListenersMap.remove( queryId, resultsListener );
         }
       }
     }
   }
 
-  protected String buildErrorMessage(QueryResultBatch batch) {
+  /**
+   * Maps internal low-level API protocol to {@link UserResultsListener}-level API protocol.
+   * handles query data messages
+   */
+  public void batchArrived( ConnectionThrottle throttle,
+                            ByteBuf pBody, ByteBuf dBody ) throws RpcException {
+    final QueryData queryData = RpcBus.get( pBody, QueryData.PARSER );
+    // Current batch coming in.
+    final QueryDataBatch batch = new QueryDataBatch( queryData, (DrillBuf) dBody );
+
+    final QueryId queryId = queryData.getQueryId();
+
+    logger.debug( "batchArrived: queryId = {}", queryId );
+    logger.trace( "batchArrived: batch = {}", batch );
+
+    UserResultsListener resultsListener = newUserResultsListener(queryId);
+
+    // A data case--pass on via dataArrived
+
+    try {
+      resultsListener.dataArrived(batch, throttle);
+      // That releases batch if successful.
+    } catch ( Exception e ) {
+      batch.release();
+      resultsListener.submissionFailed(new RpcException(e));
+    }
+  }
+
+  /**
+   * Return {@link UserResultsListener} associated with queryId. Will create a new {@link BufferingResultsListener}
+   * if no listener found.
+   * @param queryId queryId we are getting the listener for
+   * @return {@link UserResultsListener} associated with queryId
+   */
+  private UserResultsListener newUserResultsListener(QueryId queryId) {
+    UserResultsListener resultsListener = queryIdToResultsListenersMap.get( queryId );
+    logger.trace( "For QueryId [{}], retrieved results listener {}", queryId, resultsListener );
+    if ( null == resultsListener ) {
+      // WHO?? didn't get query ID response and set submission listener yet,
+      // so install a buffering listener for now
+
+      BufferingResultsListener bl = new BufferingResultsListener();
+      resultsListener = queryIdToResultsListenersMap.putIfAbsent( queryId, bl );
+      // If we had a successful insertion, use that reference.  Otherwise, just
+      // throw away the new buffering listener.
+      if ( null == resultsListener ) {
+        resultsListener = bl;
+      }
+      // TODO:  Is there a more direct way to detect a Query ID in whatever state this string comparison detects?
+      if ( queryId.toString().isEmpty() ) {
+        failAll();
+      }
+    }
+    return resultsListener;
+  }
+
+  protected String buildErrorMessage(QueryResult result) {
     StringBuilder sb = new StringBuilder();
-    for (UserBitShared.DrillPBError error : batch.getHeader().getErrorList()) {
+    for (UserBitShared.DrillPBError error : result.getErrorList()) {
       sb.append(error.getMessage());
       sb.append("\n");
     }
@@ -252,7 +206,7 @@ public class QueryResultHandler {
 
   private static class BufferingResultsListener implements UserResultsListener {
 
-    private ConcurrentLinkedQueue<QueryResultBatch> results = Queues.newConcurrentLinkedQueue();
+    private ConcurrentLinkedQueue<QueryDataBatch> results = Queues.newConcurrentLinkedQueue();
     private volatile boolean finished = false;
     private volatile RpcException ex;
     private volatile UserResultsListener output;
@@ -261,31 +215,39 @@ public class QueryResultHandler {
     public boolean transferTo(UserResultsListener l) {
       synchronized (this) {
         output = l;
-        boolean last = false;
-        for (QueryResultBatch r : results) {
-          l.resultArrived(r, throttle);
-          last = r.getHeader().getIsLastChunk();
+        for (QueryDataBatch r : results) {
+          l.dataArrived(r, throttle);
         }
         if (ex != null) {
           l.submissionFailed(ex);
           return true;
+        } else if (finished) {
+          l.queryCompleted();
         }
-        return last;
+
+        return finished;
       }
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      this.throttle = throttle;
-      if (result.getHeader().getIsLastChunk()) {
-        finished = true;
+    public void queryCompleted() {
+      finished = true;
+      synchronized (this) {
+        if (output != null) {
+          output.queryCompleted();
+        }
       }
+    }
+
+    @Override
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+      this.throttle = throttle;
 
       synchronized (this) {
         if (output == null) {
           this.results.add(result);
         } else {
-          output.resultArrived(result, throttle);
+          output.dataArrived(result, throttle);
         }
       }
     }
@@ -340,11 +302,10 @@ public class QueryResultHandler {
       if (oldListener != null) {
         logger.debug("Unable to place user results listener, buffering listener was already in place.");
         if (oldListener instanceof BufferingResultsListener) {
-          queryIdToResultsListenersMap.remove(oldListener);
           boolean all = ((BufferingResultsListener) oldListener).transferTo(this.resultsListener);
           // simply remove the buffering listener if we already have the last response.
           if (all) {
-            queryIdToResultsListenersMap.remove(oldListener);
+            queryIdToResultsListenersMap.remove(queryId);
           } else {
             boolean replaced = queryIdToResultsListenersMap.replace(queryId, oldListener, resultsListener);
             if (!replaced) {

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 925154d..fd82699 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
@@ -41,7 +42,7 @@ import org.apache.drill.exec.rpc.RpcException;
 import com.google.protobuf.MessageLite;
 
 public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHandshake, BitToUserHandshake> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
+//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
 
   private final QueryResultHandler queryResultHandler = new QueryResultHandler();
 
@@ -80,8 +81,10 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
       return BitToUserHandshake.getDefaultInstance();
     case RpcType.QUERY_HANDLE_VALUE:
       return QueryId.getDefaultInstance();
-    case RpcType.QUERY_RESULT_VALUE:
-      return QueryResult.getDefaultInstance();
+      case RpcType.QUERY_RESULT_VALUE:
+        return QueryResult.getDefaultInstance();
+    case RpcType.QUERY_DATA_VALUE:
+      return QueryData.getDefaultInstance();
     }
     throw new RpcException(String.format("Unable to deal with RpcType of %d", rpcType));
   }
@@ -89,9 +92,12 @@ public class UserClient extends BasicClientWithConnection<RpcType, UserToBitHand
   @Override
   protected Response handleReponse(ConnectionThrottle throttle, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
     switch (rpcType) {
-    case RpcType.QUERY_RESULT_VALUE:
+    case RpcType.QUERY_DATA_VALUE:
       queryResultHandler.batchArrived(throttle, pBody, dBody);
       return new Response(RpcType.ACK, Ack.getDefaultInstance());
+    case RpcType.QUERY_RESULT_VALUE:
+      queryResultHandler.resultArrived(pBody);
+      return new Response(RpcType.ACK, Ack.getDefaultInstance());
     default:
       throw new RpcException(String.format("Unknown Rpc Type %d. ", rpcType));
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
index 9f83a4f..934a094 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
@@ -22,8 +22,30 @@ import org.apache.drill.exec.rpc.RpcException;
 
 public interface UserResultsListener {
 
-  public abstract void queryIdArrived(QueryId queryId);
-  public abstract void submissionFailed(RpcException ex);
-  public abstract void resultArrived(QueryResultBatch result, ConnectionThrottle throttle);
+  /**
+   * QueryId is available. Called when a query is successfully submitted to the server.
+   * @param queryId sent by the server along {@link org.apache.drill.exec.rpc.Acks.OK Acks.OK}
+   */
+  void queryIdArrived(QueryId queryId);
+
+  /**
+   * The query has failed. Most likely called when the server returns a FAILED query state. Can also be called if
+   * {@link #dataArrived(QueryDataBatch, ConnectionThrottle) dataArrived()} throws an exception
+   * @param ex exception describing the cause of the failure
+   */
+  void submissionFailed(RpcException ex);
+
+  /**
+   * The query has completed (successsful completion or cancellation). The listener will not receive any other
+   * data or result message. Called when the server returns a terminal-non failing- state (COMPLETED or CANCELLED)
+   */
+  void queryCompleted();
+
+  /**
+   * A {@link org.apache.drill.exec.proto.beans.QueryData QueryData} message was received
+   * @param result data batch received
+   * @param throttle connection throttle
+   */
+  void dataArrived(QueryDataBatch result, ConnectionThrottle throttle);
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
index 908d304..9fb8bdb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.rpc.user;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
@@ -33,8 +34,9 @@ public class UserRpcConfig {
       .add(RpcType.HANDSHAKE, UserToBitHandshake.class, RpcType.HANDSHAKE, BitToUserHandshake.class) //user to bit.
       .add(RpcType.RUN_QUERY, RunQuery.class, RpcType.QUERY_HANDLE, QueryId.class) //user to bit
       .add(RpcType.CANCEL_QUERY, QueryId.class, RpcType.ACK, Ack.class) //user to bit
+      .add(RpcType.QUERY_DATA, QueryData.class, RpcType.ACK, Ack.class) //bit to user
       .add(RpcType.QUERY_RESULT, QueryResult.class, RpcType.ACK, Ack.class) //bit to user
       .build();
 
-  public static int RPC_VERSION = 3;
+  public static int RPC_VERSION = 4;
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index c76d324..5b2433b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
@@ -45,7 +46,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
 import com.google.protobuf.MessageLite;
 
 public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnection> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
 
   final UserWorker worker;
   final BufferAllocator alloc;
@@ -117,16 +118,19 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
       return session;
     }
 
-    public void sendResult(RpcOutcomeListener<Ack> listener, QueryWritableBatch result){
+    public void sendResult(RpcOutcomeListener<Ack> listener, QueryResult result, boolean allowInEventThread){
       logger.trace("Sending result to client with {}", result);
-      send(listener, this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, false, result.getBuffers());
+      send(listener, this, RpcType.QUERY_RESULT, result, Ack.class, allowInEventThread);
     }
 
-    public void sendResult(RpcOutcomeListener<Ack> listener, QueryWritableBatch result, boolean allowInEventThread){
-      logger.trace("Sending result to client with {}", result);
-      send(listener, this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, allowInEventThread, result.getBuffers());
+    public void sendData(RpcOutcomeListener<Ack> listener, QueryWritableBatch result){
+      sendData(listener, result, false);
     }
 
+    public void sendData(RpcOutcomeListener<Ack> listener, QueryWritableBatch result, boolean allowInEventThread){
+      logger.trace("Sending data to client with {}", result);
+      send(listener, this, RpcType.QUERY_DATA, result.getHeader(), Ack.class, allowInEventThread, result.getBuffers());
+    }
     @Override
     public BufferAllocator getAllocator() {
       return alloc;

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
index 8996a69..fbbf0b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
@@ -39,7 +39,7 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -49,8 +49,7 @@ import parquet.Preconditions;
 
 @XmlRootElement
 public class QueryWrapper {
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWrapper.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWrapper.class);
 
   private String query;
   private String queryType;
@@ -137,7 +136,12 @@ public class QueryWrapper {
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+    public void queryCompleted() {
+      latch.countDown();
+    }
+
+    @Override
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
       try {
         final int rows = result.getHeader().getRowCount();
         if (result.hasData()) {
@@ -162,9 +166,6 @@ public class QueryWrapper {
         throw new RuntimeException(e);
       } finally {
         result.release();
-        if (result.getHeader().getIsLastChunk()) {
-          latch.countDown();
-        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/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 285b75a..23ef0d3 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
@@ -46,7 +46,6 @@ import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.ExternalSort;
-import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.planner.fragment.Fragment;
 import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
@@ -612,7 +611,6 @@ public class Foreman implements Runnable {
        * Construct the response based on the latest resultState. The builder shouldn't fail.
        */
       final QueryResult.Builder resultBuilder = QueryResult.newBuilder()
-          .setIsLastChunk(resultState != QueryState.COMPLETED) // TODO(DRILL-2498) temporary
           .setQueryId(queryId)
           .setQueryState(resultState);
       if (resultException != null) {
@@ -629,7 +627,7 @@ public class Foreman implements Runnable {
        */
       try {
         // send whatever result we ended up with
-        initiatingClient.sendResult(responseListener, new QueryWritableBatch(resultBuilder.build()), true);
+        initiatingClient.sendResult(responseListener, resultBuilder.build(), true);
       } catch(Exception e) {
         addException(e);
         logger.warn("Exception sending result to client", resultException);

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 64cf2ec..c602a01 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -38,7 +38,7 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -188,19 +188,19 @@ public class BaseTestQuery extends ExecTest {
     listener.waitForCompletion();
   }
 
-  protected static List<QueryResultBatch> testSqlWithResults(String sql) throws Exception{
+  protected static List<QueryDataBatch> testSqlWithResults(String sql) throws Exception{
     return testRunAndReturn(QueryType.SQL, sql);
   }
 
-  protected static List<QueryResultBatch> testLogicalWithResults(String logical) throws Exception{
+  protected static List<QueryDataBatch> testLogicalWithResults(String logical) throws Exception{
     return testRunAndReturn(QueryType.LOGICAL, logical);
   }
 
-  protected static List<QueryResultBatch> testPhysicalWithResults(String physical) throws Exception{
+  protected static List<QueryDataBatch> testPhysicalWithResults(String physical) throws Exception{
     return testRunAndReturn(QueryType.PHYSICAL, physical);
   }
 
-  public static List<QueryResultBatch>  testRunAndReturn(QueryType type, String query) throws Exception{
+  public static List<QueryDataBatch>  testRunAndReturn(QueryType type, String query) throws Exception{
     query = QueryTestUtil.normalizeQuery(query);
     return client.runQuery(type, query);
   }
@@ -221,9 +221,9 @@ public class BaseTestQuery extends ExecTest {
     query = String.format(query, args);
     logger.debug("Running query:\n--------------\n"+query);
     for (int i = 0; i < interation; i++) {
-      List<QueryResultBatch> results = client.runQuery(QueryType.SQL, query);
-      for (QueryResultBatch queryResultBatch : results) {
-        queryResultBatch.release();
+      List<QueryDataBatch> results = client.runQuery(QueryType.SQL, query);
+      for (QueryDataBatch queryDataBatch : results) {
+        queryDataBatch.release();
       }
     }
   }
@@ -252,7 +252,7 @@ public class BaseTestQuery extends ExecTest {
     testPhysical(getFile(file));
   }
 
-  protected static List<QueryResultBatch> testPhysicalFromFileWithResults(String file) throws Exception {
+  protected static List<QueryDataBatch> testPhysicalFromFileWithResults(String file) throws Exception {
     return testRunAndReturn(QueryType.PHYSICAL, getFile(file));
   }
 
@@ -285,16 +285,18 @@ public class BaseTestQuery extends ExecTest {
     }
 
     @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+    public void queryCompleted() {
+      System.out.println("Query completed successfully with row count: " + count.get());
+      latch.countDown();
+    }
+
+    @Override
+    public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
       int rows = result.getHeader().getRowCount();
       if (result.getData() != null) {
         count.addAndGet(rows);
       }
       result.release();
-      if (result.getHeader().getIsLastChunk()) {
-        System.out.println("Query completed successfully with row count: " + count.get());
-        latch.countDown();
-      }
     }
 
     @Override
@@ -317,10 +319,10 @@ public class BaseTestQuery extends ExecTest {
     this.columnWidths = columnWidths;
   }
 
-  protected int printResult(List<QueryResultBatch> results) throws SchemaChangeException {
+  protected int printResult(List<QueryDataBatch> results) throws SchemaChangeException {
     int rowCount = 0;
     RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
-    for(QueryResultBatch result : results) {
+    for(QueryDataBatch result : results) {
       rowCount += result.getHeader().getRowCount();
       loader.load(result.getHeader().getDef(), result.getData());
       if (loader.getRecordCount() <= 0) {
@@ -334,12 +336,12 @@ public class BaseTestQuery extends ExecTest {
     return rowCount;
   }
 
-  protected static String getResultString(List<QueryResultBatch> results, String delimiter)
+  protected static String getResultString(List<QueryDataBatch> results, String delimiter)
       throws SchemaChangeException {
     StringBuilder formattedResults = new StringBuilder();
     boolean includeHeader = true;
     RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
-    for(QueryResultBatch result : results) {
+    for(QueryDataBatch result : results) {
       loader.load(result.getHeader().getDef(), result.getData());
       if (loader.getRecordCount() <= 0) {
         continue;

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java b/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
index 75a91b3..d05c896 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.record.HyperVectorWrapper;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.hadoop.io.Text;
 
@@ -166,13 +166,13 @@ public class DrillTestWrapper {
     }
   }
 
-  private Map<String, HyperVectorValueIterator> addToHyperVectorMap(List<QueryResultBatch> records, RecordBatchLoader loader,
+  private Map<String, HyperVectorValueIterator> addToHyperVectorMap(List<QueryDataBatch> records, RecordBatchLoader loader,
                                                                       BatchSchema schema) throws SchemaChangeException, UnsupportedEncodingException {
     // TODO - this does not handle schema changes
     Map<String, HyperVectorValueIterator> combinedVectors = new HashMap();
 
     long totalRecords = 0;
-    QueryResultBatch batch;
+    QueryDataBatch batch;
     int size = records.size();
     for (int i = 0; i < size; i++) {
       batch = records.get(i);
@@ -213,13 +213,13 @@ public class DrillTestWrapper {
    * @throws SchemaChangeException
    * @throws UnsupportedEncodingException
    */
-   private Map<String, List> addToCombinedVectorResults(List<QueryResultBatch> records, RecordBatchLoader loader,
+   private Map<String, List> addToCombinedVectorResults(List<QueryDataBatch> records, RecordBatchLoader loader,
                                                          BatchSchema schema) throws SchemaChangeException, UnsupportedEncodingException {
     // TODO - this does not handle schema changes
     Map<String, List> combinedVectors = new HashMap();
 
     long totalRecords = 0;
-    QueryResultBatch batch;
+    QueryDataBatch batch;
     int size = records.size();
     for (int i = 0; i < size; i++) {
       batch = records.get(0);
@@ -268,14 +268,14 @@ public class DrillTestWrapper {
     BatchSchema schema = null;
 
     BaseTestQuery.test(testOptionSettingQueries);
-    List<QueryResultBatch> expected = BaseTestQuery.testRunAndReturn(queryType, query);
+    List<QueryDataBatch> expected = BaseTestQuery.testRunAndReturn(queryType, query);
 
     addTypeInfoIfMissing(expected.get(0), testBuilder);
 
     List<Map> expectedRecords = new ArrayList<>();
     addToMaterializedResults(expectedRecords, expected, loader, schema);
 
-    List<QueryResultBatch> results = new ArrayList();
+    List<QueryDataBatch> results = new ArrayList();
     List<Map> actualRecords = new ArrayList<>();
     // If baseline data was not provided to the test builder directly, we must run a query for the baseline, this includes
     // the cases where the baseline is stored in a file.
@@ -313,13 +313,13 @@ public class DrillTestWrapper {
     BatchSchema schema = null;
 
     BaseTestQuery.test(testOptionSettingQueries);
-    List<QueryResultBatch> results = BaseTestQuery.testRunAndReturn(queryType, query);
+    List<QueryDataBatch> results = BaseTestQuery.testRunAndReturn(queryType, query);
     // To avoid extra work for test writers, types can optionally be inferred from the test query
     addTypeInfoIfMissing(results.get(0), testBuilder);
 
     Map<String, List> actualSuperVectors = addToCombinedVectorResults(results, loader, schema);
 
-    List<QueryResultBatch> expected = null;
+    List<QueryDataBatch> expected = null;
     Map<String, List> expectedSuperVectors = null;
 
     // If baseline data was not provided to the test builder directly, we must run a query for the baseline, this includes
@@ -353,14 +353,14 @@ public class DrillTestWrapper {
     BatchSchema schema = null;
 
     BaseTestQuery.test(testOptionSettingQueries);
-    List<QueryResultBatch> results = BaseTestQuery.testRunAndReturn(queryType, query);
+    List<QueryDataBatch> results = BaseTestQuery.testRunAndReturn(queryType, query);
     // To avoid extra work for test writers, types can optionally be inferred from the test query
     addTypeInfoIfMissing(results.get(0), testBuilder);
 
     Map<String, HyperVectorValueIterator> actualSuperVectors = addToHyperVectorMap(results, loader, schema);
 
     BaseTestQuery.test(baselineOptionSettingQueries);
-    List<QueryResultBatch> expected = BaseTestQuery.testRunAndReturn(baselineQueryType, testBuilder.getValidationQuery());
+    List<QueryDataBatch> expected = BaseTestQuery.testRunAndReturn(baselineQueryType, testBuilder.getValidationQuery());
 
     Map<String, HyperVectorValueIterator> expectedSuperVectors = addToHyperVectorMap(expected, loader, schema);
 
@@ -368,7 +368,7 @@ public class DrillTestWrapper {
     cleanupBatches(results, expected);
   }
 
-  private void addTypeInfoIfMissing(QueryResultBatch batch, TestBuilder testBuilder) {
+  private void addTypeInfoIfMissing(QueryDataBatch batch, TestBuilder testBuilder) {
     if (! testBuilder.typeInfoSet()) {
       Map<SchemaPath, TypeProtos.MajorType> typeMap = getTypeMapFromBatch(batch);
       testBuilder.baselineTypes(typeMap);
@@ -376,7 +376,7 @@ public class DrillTestWrapper {
 
   }
 
-  private Map<SchemaPath, TypeProtos.MajorType> getTypeMapFromBatch(QueryResultBatch batch) {
+  private Map<SchemaPath, TypeProtos.MajorType> getTypeMapFromBatch(QueryDataBatch batch) {
     Map<SchemaPath, TypeProtos.MajorType> typeMap = new HashMap();
     for (int i = 0; i < batch.getHeader().getDef().getFieldCount(); i++) {
       typeMap.put(MaterializedField.create(batch.getHeader().getDef().getField(i)).getPath(),
@@ -385,18 +385,18 @@ public class DrillTestWrapper {
     return typeMap;
   }
 
-  private void cleanupBatches(List<QueryResultBatch>... results) {
-    for (List<QueryResultBatch> resultList : results ) {
-      for (QueryResultBatch result : resultList) {
+  private void cleanupBatches(List<QueryDataBatch>... results) {
+    for (List<QueryDataBatch> resultList : results ) {
+      for (QueryDataBatch result : resultList) {
         result.release();
       }
     }
   }
 
-  protected void addToMaterializedResults(List<Map> materializedRecords,  List<QueryResultBatch> records, RecordBatchLoader loader,
+  protected void addToMaterializedResults(List<Map> materializedRecords,  List<QueryDataBatch> records, RecordBatchLoader loader,
                                           BatchSchema schema) throws SchemaChangeException, UnsupportedEncodingException {
     long totalRecords = 0;
-    QueryResultBatch batch;
+    QueryDataBatch batch;
     int size = records.size();
     for (int i = 0; i < size; i++) {
       batch = records.get(0);

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index 80b4d13..4744978 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -28,7 +28,7 @@ import java.util.regex.Pattern;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.eigenbase.sql.SqlExplain.Depth;
@@ -288,12 +288,12 @@ public class PlanTestBase extends BaseTestQuery {
    */
   protected static String getPlanInString(String sql, String columnName)
       throws Exception {
-    List<QueryResultBatch> results = testSqlWithResults(sql);
+    List<QueryDataBatch> results = testSqlWithResults(sql);
 
     RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
     StringBuilder builder = new StringBuilder();
 
-    for (QueryResultBatch b : results) {
+    for (QueryDataBatch b : results) {
       if (!b.hasData()) {
         continue;
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
index 3d19229..82f1752 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.client.PrintingResultsListener;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.util.VectorUtil;
@@ -59,11 +59,11 @@ public class QueryTestUtil {
     final DrillClient drillClient = new DrillClient(drillConfig, remoteServiceSet.getCoordinator());
     drillClient.connect();
 
-    final List<QueryResultBatch> results = drillClient.runQuery(
+    final List<QueryDataBatch> results = drillClient.runQuery(
         QueryType.SQL, String.format("alter session set `%s` = %d",
             ExecConstants.MAX_WIDTH_PER_NODE_KEY, maxWidth));
-    for (QueryResultBatch queryResultBatch : results) {
-      queryResultBatch.release();
+    for (QueryDataBatch queryDataBatch : results) {
+      queryDataBatch.release();
     }
 
     return drillClient;

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java b/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
index 07cb833..5703bf9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/SingleRowListener.java
@@ -24,17 +24,17 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryResult;
+import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
 
 /**
  * Result listener that is set up to receive a single row. Useful for queries
- * such with a count(*) or limit 1. The abstract method {@link #rowArrived(QueryResultBatch)} provides
+ * such with a count(*) or limit 1. The abstract method {@link #rowArrived(QueryDataBatch)} provides
  * the means for a derived class to get the expected record's data.
  */
 public abstract class SingleRowListener implements UserResultsListener {
@@ -51,14 +51,26 @@ public abstract class SingleRowListener implements UserResultsListener {
   @Override
   public void submissionFailed(final RpcException ex) {
     exception = ex;
+    synchronized(errorList) {
+      errorList.add(ex.getRemoteError());
+    }
     latch.countDown();
   }
 
   @Override
-  public void resultArrived(final QueryResultBatch result, final ConnectionThrottle throttle) {
-    final QueryResult queryResult = result.getHeader();
+  public void queryCompleted() {
+    try {
+      cleanup();
+    } finally {
+      latch.countDown();
+    }
+  }
+
+  @Override
+  public void dataArrived(final QueryDataBatch result, final ConnectionThrottle throttle) {
+    final QueryData queryData = result.getHeader();
     if (result.hasData()) {
-      final int nRows = this.nRows.addAndGet(queryResult.getRowCount());
+      final int nRows = this.nRows.addAndGet(queryData.getRowCount());
       if (nRows > 1) {
         throw new IllegalStateException("Expected exactly one row, but got " + nRows);
       }
@@ -66,22 +78,7 @@ public abstract class SingleRowListener implements UserResultsListener {
       rowArrived(result);
     }
 
-    // TODO this appears to never be set
-    if (queryResult.hasQueryState()) {
-      queryState = queryResult.getQueryState();
-    }
-
-    synchronized(errorList) {
-      errorList.addAll(queryResult.getErrorList());
-    }
-
-    final boolean isLastChunk = queryResult.getIsLastChunk();
     result.release();
-
-    if (isLastChunk) {
-      cleanup();
-      latch.countDown();
-    }
   }
 
   /**
@@ -110,9 +107,9 @@ public abstract class SingleRowListener implements UserResultsListener {
    * <p>Derived classes provide whatever implementation they require here to access
    * the record's data.
    *
-   * @param queryResultBatch result batch holding the row
+   * @param queryDataBatch result batch holding the row
    */
-  protected abstract void rowArrived(QueryResultBatch queryResultBatch);
+  protected abstract void rowArrived(QueryDataBatch queryDataBatch);
 
   /**
    * Wait for the completion of this query; receiving a record or an error will both cause the

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/TestQueriesOnLargeFile.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestQueriesOnLargeFile.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestQueriesOnLargeFile.java
index 67b102d..0a24073 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestQueriesOnLargeFile.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestQueriesOnLargeFile.java
@@ -27,7 +27,7 @@ import java.util.List;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -72,12 +72,12 @@ public class TestQueriesOnLargeFile extends BaseTestQuery {
 
   @Test
   public void testRead() throws Exception {
-    List<QueryResultBatch> results = testSqlWithResults(
+    List<QueryDataBatch> results = testSqlWithResults(
         String.format("SELECT count(*) FROM dfs_test.`default`.`%s`", dataFile.getPath()));
 
     RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
 
-    for(QueryResultBatch batch : results) {
+    for(QueryDataBatch batch : results) {
       batchLoader.load(batch.getHeader().getDef(), batch.getData());
 
       if (batchLoader.getRecordCount() <= 0) {
@@ -96,10 +96,10 @@ public class TestQueriesOnLargeFile extends BaseTestQuery {
   public void testMergingReceiver() throws Exception {
     String plan = Files.toString(FileUtils.getResourceAsFile("/largefiles/merging_receiver_large_data.json"),
         Charsets.UTF_8).replace("#{TEST_FILE}", escapeJsonString(dataFile.getPath()));
-    List<QueryResultBatch> results = testPhysicalWithResults(plan);
+    List<QueryDataBatch> results = testPhysicalWithResults(plan);
 
     int recordsInOutput = 0;
-    for(QueryResultBatch batch : results) {
+    for(QueryDataBatch batch : results) {
       recordsInOutput += batch.getHeader().getDef().getRecordCount();
       batch.release();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index 98919ec..df03c7d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -21,7 +21,7 @@ import java.util.List;
 
 import org.apache.drill.exec.DrillSystemTestBase;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -53,8 +53,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
     startCluster(1);
     DrillClient client = new DrillClient();
     client.connect();
-    List<QueryResultBatch> results = client.runQuery(QueryType.LOGICAL, plan);
-    for (QueryResultBatch result : results) {
+    List<QueryDataBatch> results = client.runQuery(QueryType.LOGICAL, plan);
+    for (QueryDataBatch result : results) {
       System.out.println(result);
       result.release();
     }
@@ -66,8 +66,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
     startCluster(2);
     DrillClient client = new DrillClient();
     client.connect();
-    List<QueryResultBatch> results = client.runQuery(QueryType.LOGICAL, plan);
-    for (QueryResultBatch result : results) {
+    List<QueryDataBatch> results = client.runQuery(QueryType.LOGICAL, plan);
+    for (QueryDataBatch result : results) {
       System.out.println(result);
       result.release();
     }


[9/9] drill git commit: DRILL-2573: C++ Client - Separate QueryResult into QueryResult and QueryData

Posted by pa...@apache.org.
DRILL-2573: C++ Client - Separate QueryResult into QueryResult and QueryData


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

Branch: refs/heads/master
Commit: 4f213570f29a30c8609afacba0ca01cc33cdc7d0
Parents: 6a8e0ec
Author: Parth Chandra <pc...@maprtech.com>
Authored: Fri Mar 27 11:21:07 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Apr 3 18:40:53 2015 -0700

----------------------------------------------------------------------
 .../native/client/example/querySubmitter.cpp    |  20 +-
 .../client/src/clientlib/drillClientImpl.cpp    | 372 ++++----
 .../client/src/clientlib/drillClientImpl.hpp    |  19 +-
 .../native/client/src/clientlib/recordBatch.cpp |   4 +-
 .../native/client/src/include/drill/common.hpp  |   2 +-
 .../client/src/include/drill/drillClient.hpp    |   4 +-
 .../client/src/include/drill/recordBatch.hpp    |   9 +-
 .../native/client/src/protobuf/BitData.pb.cc    | 289 ++++---
 contrib/native/client/src/protobuf/BitData.pb.h | 256 +++---
 contrib/native/client/src/protobuf/User.pb.cc   |  13 +-
 contrib/native/client/src/protobuf/User.pb.h    |   7 +-
 .../client/src/protobuf/UserBitShared.pb.cc     | 855 +++++++++----------
 .../client/src/protobuf/UserBitShared.pb.h      | 477 +++++------
 13 files changed, 1171 insertions(+), 1156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/example/querySubmitter.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/example/querySubmitter.cpp b/contrib/native/client/example/querySubmitter.cpp
index bef64bf..85e89e0 100644
--- a/contrib/native/client/example/querySubmitter.cpp
+++ b/contrib/native/client/example/querySubmitter.cpp
@@ -71,15 +71,18 @@ Drill::status_t QueryResultsListener(void* ctx, Drill::RecordBatch* b, Drill::Dr
     // or
     // (received query state message passed by `err` and b is NULL)
     if(!err){
-        assert(b!=NULL);
-        b->print(std::cout, 0); // print all rows
-        std::cout << "DATA RECEIVED ..." << std::endl;
-        delete b; // we're done with this batch, we can delete it
-        if(bTestCancel){
-            return Drill::QRY_FAILURE;
+        if(b!=NULL){
+            b->print(std::cout, 0); // print all rows
+            std::cout << "DATA RECEIVED ..." << std::endl;
+            delete b; // we're done with this batch, we can delete it
+            if(bTestCancel){
+                return Drill::QRY_FAILURE;
+            }else{
+                return Drill::QRY_SUCCESS ;
+            }
         }else{
-            return Drill::QRY_SUCCESS ;
-        }
+            std::cout << "Query Complete." << std::endl;
+		}
     }else{
         assert(b==NULL);
         switch(err->status) {
@@ -392,6 +395,7 @@ int main(int argc, char* argv[]) {
                 }
                 client.freeQueryIterator(&pRecIter);
             }
+            client.waitForResults();
         }else{
             if(bSyncSend){
                 for(queryInpIter = queryInputs.begin(); queryInpIter != queryInputs.end(); queryInpIter++) {

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index 71f960e..dce5bdc 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -53,7 +53,7 @@ static std::map<exec::shared::QueryResult_QueryState, status_t> QUERYSTATE_TO_ST
     (exec::shared::QueryResult_QueryState_COMPLETED, QRY_COMPLETED)
     (exec::shared::QueryResult_QueryState_CANCELED, QRY_CANCELED)
     (exec::shared::QueryResult_QueryState_FAILED, QRY_FAILED)
-    (exec::shared::QueryResult_QueryState_UNKNOWN_QUERY, QRY_UNKNOWN_QUERY);
+    ;
 
 RpcEncoder DrillClientImpl::s_encoder;
 RpcDecoder DrillClientImpl::s_decoder;
@@ -325,7 +325,7 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
         return ret;
     }
     if(m_handshakeVersion != u2b.rpc_version()) {
-        DRILL_LOG(LOG_TRACE) << "Invalid rpc version.  Expected << "
+        DRILL_LOG(LOG_TRACE) << "Invalid rpc version.  Expected "
             << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;
         return handleConnError(CONN_HANDSHAKE_FAILED,
                 getMessage(ERR_CONN_NOHSHAKE, DRILL_RPC_VERSION, m_handshakeVersion));
@@ -510,7 +510,59 @@ status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
 status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer, InBoundRpcMessage& msg ){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     status_t ret=QRY_SUCCESS;
-	exec::shared::QueryId qid;
+    exec::shared::QueryId qid;
+    sendAck(msg, true);
+    {
+        boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
+        exec::shared::QueryResult qr;
+
+        DRILL_LOG(LOG_DEBUG) << "Processing Query Result " << std::endl;
+        qr.ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
+        DRILL_LOG(LOG_TRACE) << qr.DebugString() << std::endl;
+        
+        qid.CopyFrom(qr.query_id());
+        
+        if (qr.has_query_state() &&
+                qr.query_state() != exec::shared::QueryResult_QueryState_RUNNING &&
+                qr.query_state() != exec::shared::QueryResult_QueryState_PENDING) {
+            pDrillClientQueryResult=findQueryResult(qid);
+            //Queries that have been cancelled or whose resources are freed before completion 
+            //do not have a DrillClientQueryResult object. We need not handle the terminal message 
+            //in that case since all it does is to free resources (and they have already been freed)
+            if(pDrillClientQueryResult!=NULL){
+                //Validate the RPC message
+                std::string valErr;
+                if( (ret=validateResultMessage(msg, qr, valErr)) != QRY_SUCCESS){
+                    delete allocatedBuffer;
+                    DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: ERR_QRY_INVRPC." << std::endl;
+                    return handleQryError(ret, getMessage(ERR_QRY_INVRPC, valErr.c_str()), pDrillClientQueryResult);
+                }
+                ret=processQueryStatusResult(&qr, pDrillClientQueryResult);
+            }else{
+                // We've received the final message for a query that has been cancelled
+                // or for which the resources have been freed. We no longer need to listen
+                // for more incoming messages for such a query.
+                DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult:" << debugPrintQid(qid)<< " completed."<< std::endl;
+                m_pendingRequests--;
+                DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: pending requests is " << m_pendingRequests<< std::endl;
+                ret=QRY_CANCELED;
+            }
+            delete allocatedBuffer;
+            return ret;
+        }else{
+            // Normal query results come back with query_state not set.
+            // Actually this is not strictly true. The query state is set to
+            // 0(i.e. PENDING), but protobuf thinks this means the value is not set.
+            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: Query State was not set.\n";
+        }
+    }
+    return ret;
+}
+
+status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer, InBoundRpcMessage& msg ){
+    DrillClientQueryResult* pDrillClientQueryResult=NULL;
+    status_t ret=QRY_SUCCESS;
+    exec::shared::QueryId qid;
     // Be a good client and send ack as early as possible.
     // Drillbit pushed the query result to the client, the client should send ack
     // whenever it receives the message
@@ -518,62 +570,34 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
     RecordBatch* pRecordBatch=NULL;
     {
         boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-        exec::shared::QueryResult* qr = new exec::shared::QueryResult; //Record Batch will own this object and free it up.
+        exec::shared::QueryData* qr = new exec::shared::QueryData; //Record Batch will own this object and free it up.
 
-        DRILL_LOG(LOG_DEBUG) << "Processing Query Result " << std::endl;
+        DRILL_LOG(LOG_DEBUG) << "Processing Query Data " << std::endl;
         qr->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
         DRILL_LOG(LOG_TRACE) << qr->DebugString() << std::endl;
 
-        DRILL_LOG(LOG_DEBUG) << "Searching for Query Id - " << debugPrintQid(qr->query_id()) << std::endl;
-
         qid.CopyFrom(qr->query_id());
-        std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::iterator it;
-        DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: m_queryResults size: " << m_queryResults.size() << std::endl;
-        if(m_queryResults.size() != 0){
-            for(it=m_queryResults.begin(); it!=m_queryResults.end(); it++){
-                DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: m_QueryResult ids: [" << it->first->part1() << ":"
-                    << it->first->part2() << "]\n";
-            }
-        }
         if(qid.part1()==0){
-            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: QID=0. Ignore and return QRY_SUCCESS." << std::endl;
-            return QRY_SUCCESS;
-        }
-        it=this->m_queryResults.find(&qid);
-        if(it!=this->m_queryResults.end()){
-            pDrillClientQueryResult=(*it).second;
-        }else{
-            ret=processCancelledQueryResult(qid, qr);
-            DRILL_LOG(LOG_TRACE) << "Cleaning up resource allocated for canceled quyery." << std::endl;
-            delete qr;
+            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: QID=0. Ignore and return QRY_SUCCESS." << std::endl;
             delete allocatedBuffer;
-            return ret;
+            return QRY_SUCCESS;
         }
-        DRILL_LOG(LOG_DEBUG) << "Drill Client Query Result Query Id - " <<
-            debugPrintQid(*pDrillClientQueryResult->m_pQueryId) << std::endl;
 
-        // Drillbit may send a query state change message which does not contain any
-        // record batch.
-        if (qr->has_query_state() &&
-                qr->query_state() != exec::shared::QueryResult_QueryState_RUNNING &&
-                qr->query_state() != exec::shared::QueryResult_QueryState_PENDING) {
-            ret=processQueryStatusResult(qr, pDrillClientQueryResult);
-            delete allocatedBuffer;
+        pDrillClientQueryResult=findQueryResult(qid);
+        if(pDrillClientQueryResult==NULL){
+            DRILL_LOG(LOG_TRACE) << "Cleaning up resources allocated for canceled query (" 
+                                 << debugPrintQid(qid) << ")." << std::endl;
             delete qr;
+            delete allocatedBuffer;
             return ret;
-        }else{
-            // Normal query results come back with query_state not set.
-            // Actually this is not strictly true. The query state is set to
-            // 0(i.e. PENDING), but protobuf thinks this means the value is not set.
-            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: Query State was not set.\n";
         }
-
+        
         //Validate the RPC message
         std::string valErr;
-        if( (ret=validateMessage(msg, *qr, valErr)) != QRY_SUCCESS){
+        if( (ret=validateDataMessage(msg, *qr, valErr)) != QRY_SUCCESS){
             delete allocatedBuffer;
             delete qr;
-            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: ERR_QRY_INVRPC.\n";
+            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: ERR_QRY_INVRPC.\n";
             pDrillClientQueryResult->setQueryStatus(ret);
             return handleQryError(ret, getMessage(ERR_QRY_INVRPC, valErr.c_str()), pDrillClientQueryResult);
         }
@@ -590,16 +614,13 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
             << pRecordBatch->getNumRecords()  << std::endl;
         DRILL_LOG(LOG_DEBUG) << debugPrintQid(qr->query_id())<<"recordBatch.numFields "
             << pRecordBatch->getNumFields()  << std::endl;
-        DRILL_LOG(LOG_DEBUG) << debugPrintQid(qr->query_id())<<"recordBatch.isLastChunk "
-            << pRecordBatch->isLastChunk()  << std::endl;
 
         ret=pDrillClientQueryResult->setupColumnDefs(qr);
         if(ret==QRY_SUCCESS_WITH_INFO){
             pRecordBatch->schemaChanged(true);
         }
 
-        pDrillClientQueryResult->m_bIsQueryPending=true;
-        pDrillClientQueryResult->m_bIsLastChunk=qr->is_last_chunk();
+        pDrillClientQueryResult->setIsQueryPending(true);
         pfnQueryResultsListener pResultsListener=pDrillClientQueryResult->m_pResultsListener;
         if(pDrillClientQueryResult->m_bIsLastChunk){
             DRILL_LOG(LOG_DEBUG) << debugPrintQid(*pDrillClientQueryResult->m_pQueryId)
@@ -619,7 +640,7 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
         sendCancel(&qid);
         // Do not decrement pending requests here. We have sent a cancel and we may still receive results that are
         // pushed on the wire before the cancel is processed.
-        pDrillClientQueryResult->m_bIsQueryPending=false;
+        pDrillClientQueryResult->setIsQueryPending(false);
         DRILL_LOG(LOG_DEBUG) << "Client app cancelled query." << std::endl;
         pDrillClientQueryResult->setQueryStatus(ret);
         clearMapEntries(pDrillClientQueryResult);
@@ -628,42 +649,6 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
     return ret;
 }
 
-status_t DrillClientImpl::processCancelledQueryResult(exec::shared::QueryId& qid, exec::shared::QueryResult* qr){
-    status_t ret=QRY_SUCCESS;
-    // look in cancelled queries
-    DRILL_LOG(LOG_DEBUG) << "Query Id - " << debugPrintQid(qr->query_id()) << " has been cancelled." << std::endl;
-    std::set<exec::shared::QueryId*, compareQueryId>::iterator it2;
-    exec::shared::QueryId* pQid=NULL;//
-    it2=this->m_cancelledQueries.find(&qid);
-    if(it2!=this->m_cancelledQueries.end()){
-        pQid=(*it2);
-        if(qr->has_query_state()){
-            ret = QUERYSTATE_TO_STATUS_MAP[qr->query_state()];
-            if(qr->query_state()==exec::shared::QueryResult_QueryState_COMPLETED
-                    || qr->query_state()==exec::shared::QueryResult_QueryState_CANCELED
-                    || qr->query_state()==exec::shared::QueryResult_QueryState_FAILED) {
-                this->m_pendingRequests--;
-                this->m_cancelledQueries.erase(it2);
-                delete pQid;
-                DRILL_LOG(LOG_DEBUG) << "Query Id - " << debugPrintQid(qr->query_id()) << " completed." << std::endl;
-                DRILL_LOG(LOG_DEBUG) << "Pending requests - " << this->m_pendingRequests << std::endl;
-            }
-        }
-    }else{
-        status_t ret=QRY_FAILED;
-        if(qr->has_query_state() && qr->query_state()==exec::shared::QueryResult_QueryState_COMPLETED){
-            ret = QUERYSTATE_TO_STATUS_MAP[qr->query_state()];
-        }else if(!qr->has_query_state() && qr->row_count()==0){
-            ret=QRY_SUCCESS;
-        }else{
-            //ret= handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_OUTOFORDER), NULL);
-            DRILL_LOG(LOG_DEBUG) << "Pending requests - " << getMessage(ERR_QRY_OUTOFORDER) << std::endl;
-            ret= QRY_SUCCESS;
-        }
-    }
-    return ret;
-}
-
 status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg ){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     DRILL_LOG(LOG_DEBUG) << "Processing Query Handle with coordination id:" << msg.m_coord_id << std::endl;
@@ -698,51 +683,72 @@ status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, InB
     return ret;
 }
 
+DrillClientQueryResult* DrillClientImpl::findQueryResult(exec::shared::QueryId& qid){
+    DrillClientQueryResult* pDrillClientQueryResult=NULL;
+    DRILL_LOG(LOG_DEBUG) << "Searching for Query Id - " << debugPrintQid(qid) << std::endl;
+    std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::iterator it;
+    DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: m_queryResults size: " << m_queryResults.size() << std::endl;
+    if(m_queryResults.size() != 0){
+        for(it=m_queryResults.begin(); it!=m_queryResults.end(); it++){
+            DRILL_LOG(LOG_TRACE) << "DrillClientImpl::findQueryResult: m_QueryResult ids: [" << it->first->part1() << ":"
+                << it->first->part2() << "]\n";
+        }
+    }
+    it=this->m_queryResults.find(&qid);
+    if(it!=this->m_queryResults.end()){
+        pDrillClientQueryResult=(*it).second;
+        DRILL_LOG(LOG_DEBUG) << "Drill Client Query Result Query Id - " <<
+            debugPrintQid(*pDrillClientQueryResult->m_pQueryId) << std::endl;
+    }
+    return pDrillClientQueryResult;
+}
+
 status_t DrillClientImpl::processQueryStatusResult(exec::shared::QueryResult* qr,
         DrillClientQueryResult* pDrillClientQueryResult){
-        status_t ret = QUERYSTATE_TO_STATUS_MAP[qr->query_state()];
+    status_t ret = QUERYSTATE_TO_STATUS_MAP[qr->query_state()];
+    if(pDrillClientQueryResult!=NULL){
         pDrillClientQueryResult->setQueryStatus(ret);
         pDrillClientQueryResult->setQueryState(qr->query_state());
-        switch(qr->query_state()) {
-            case exec::shared::QueryResult_QueryState_FAILED:
-            case exec::shared::QueryResult_QueryState_UNKNOWN_QUERY:
-                {
-                    // get the error message from protobuf and handle errors
-                    ret=handleQryError(ret, qr->error(0), pDrillClientQueryResult);
-                }
-                break;
-                // m_pendingRequests should be decremented when the query is
-                // completed
-            case exec::shared::QueryResult_QueryState_CANCELED:
-                {
-                    ret=handleTerminatedQryState(ret,
-                            getMessage(ERR_QRY_CANCELED),
-                            pDrillClientQueryResult);
-                    m_pendingRequests--;
-                }
-                break;
-            case exec::shared::QueryResult_QueryState_COMPLETED:
-                {
-                    //Not clean to call the handleTerminatedQryState method
-                    //because it signals an error to the listener.
-                    //The ODBC driver expects this though and the sync API
-                    //handles this (luckily).
-                    ret=handleTerminatedQryState(ret,
-                            getMessage(ERR_QRY_COMPLETED),
-                            pDrillClientQueryResult);
-                    m_pendingRequests--;
-                }
-                break;
-            default:
-                {
-                    DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryResult: Unknown Query State.\n";
-                    ret=handleQryError(QRY_INTERNAL_ERROR,
-                            getMessage(ERR_QRY_UNKQRYSTATE),
-                            pDrillClientQueryResult);
-                }
-                break;
-        }
-        return ret;
+    }
+    switch(qr->query_state()) {
+        case exec::shared::QueryResult_QueryState_FAILED:
+            {
+                // get the error message from protobuf and handle errors
+                ret=handleQryError(ret, qr->error(0), pDrillClientQueryResult);
+            }
+            break;
+            // m_pendingRequests should be decremented when the query is
+            // completed
+        case exec::shared::QueryResult_QueryState_CANCELED:
+            {
+                ret=handleTerminatedQryState(ret,
+                        getMessage(ERR_QRY_CANCELED),
+                        pDrillClientQueryResult);
+                m_pendingRequests--;
+            }
+            break;
+        case exec::shared::QueryResult_QueryState_COMPLETED:
+            {
+                //Not clean to call the handleTerminatedQryState method
+                //because it signals an error to the listener.
+                //The ODBC driver expects this though and the sync API
+                //handles this (luckily).
+                ret=handleTerminatedQryState(ret,
+                        getMessage(ERR_QRY_COMPLETED),
+                        pDrillClientQueryResult);
+                m_pendingRequests--;
+            }
+            break;
+        default:
+            {
+                DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryStatusResult: Unknown Query State.\n";
+                ret=handleQryError(QRY_INTERNAL_ERROR,
+                        getMessage(ERR_QRY_UNKQRYSTATE),
+                        pDrillClientQueryResult);
+            }
+            break;
+    }
+    return ret;
 }
 
 void DrillClientImpl::handleReadTimeout(const boost::system::error_code & err){
@@ -809,6 +815,14 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
                 }
                 return;
             }
+        }else if(!error && msg.m_rpc_type==exec::user::QUERY_DATA){
+            if(processQueryData(allocatedBuffer, msg)!=QRY_SUCCESS){
+                if(m_pendingRequests!=0){
+                    boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
+                    getNextResult();
+                }
+                return;
+            }
         }else if(!error && msg.m_rpc_type==exec::user::QUERY_HANDLE){
             if(processQueryId(allocatedBuffer, msg)!=QRY_SUCCESS){
                 if(m_pendingRequests!=0){
@@ -820,6 +834,7 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
         }else if(!error && msg.m_rpc_type==exec::user::ACK){
             // Cancel requests will result in an ACK sent back.
             // Consume silently
+            delete allocatedBuffer;
             if(m_pendingRequests!=0){
                 boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
                 getNextResult();
@@ -830,6 +845,7 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
             if(error){
                 // We have a socket read error, but we do not know which query this is for.
                 // Signal ALL pending queries that they should stop waiting.
+                delete allocatedBuffer;
                 DRILL_LOG(LOG_TRACE) << "read error: " << error << std::endl;
                 handleQryError(QRY_COMM_ERROR, getMessage(ERR_QRY_COMMERR, error.message().c_str()), NULL);
                 return;
@@ -851,10 +867,11 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
                         DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake response from server. Ignore.\n";
                     }
                 }else{
-                DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
+                    DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
                         << "QueryResult returned " << msg.m_rpc_type << std::endl;
-                handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
+                    handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
                 }
+                delete allocatedBuffer;
                 return;
             }
         }
@@ -874,21 +891,25 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
     return;
 }
 
-status_t DrillClientImpl::validateMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valErr){
+status_t DrillClientImpl::validateDataMessage(InBoundRpcMessage& msg, exec::shared::QueryData& qd, std::string& valErr){
     if(msg.m_mode == exec::rpc::RESPONSE_FAILURE){
         valErr=getMessage(ERR_QRY_RESPFAIL);
         return QRY_FAILURE;
     }
-    if(qr.query_state()== exec::shared::QueryResult_QueryState_UNKNOWN_QUERY){
-        valErr=getMessage(ERR_QRY_UNKQRY);
+    if(qd.def().carries_two_byte_selection_vector() == true){
+        valErr=getMessage(ERR_QRY_SELVEC2);
         return QRY_FAILURE;
     }
-    if(qr.query_state()== exec::shared::QueryResult_QueryState_CANCELED){
-        valErr=getMessage(ERR_QRY_CANCELED);
+    return QRY_SUCCESS;
+}
+
+status_t DrillClientImpl::validateResultMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valErr){
+    if(msg.m_mode == exec::rpc::RESPONSE_FAILURE){
+        valErr=getMessage(ERR_QRY_RESPFAIL);
         return QRY_FAILURE;
     }
-    if(qr.def().carries_two_byte_selection_vector() == true){
-        valErr=getMessage(ERR_QRY_SELVEC2);
+    if(qr.query_state()==exec::shared::QueryResult_QueryState_CANCELED){
+        valErr=getMessage(ERR_QRY_CANCELED);
         return QRY_FAILURE;
     }
     return QRY_SUCCESS;
@@ -948,7 +969,9 @@ status_t DrillClientImpl::handleTerminatedQryState(
         std::string msg,
         DrillClientQueryResult* pQueryResult){
     assert(pQueryResult!=NULL);
-    if(status!=QRY_COMPLETED){
+    if(status==QRY_COMPLETED){
+        pQueryResult->signalComplete();
+    }else{
         // set query error only if queries did not complete successfully
         DrillClientError* pErr = new DrillClientError(status, DrillClientError::QRY_ERROR_START+status, msg);
         pQueryResult->signalError(pErr);
@@ -957,18 +980,6 @@ status_t DrillClientImpl::handleTerminatedQryState(
 }
 
 
-void DrillClientImpl::clearCancelledEntries(){
-
-    std::map<int, DrillClientQueryResult*>::iterator iter;
-    boost::lock_guard<boost::mutex> lock(m_dcMutex);
-
-    if(!m_cancelledQueries.empty()){
-        std::set<exec::shared::QueryId*, compareQueryId>::iterator it;
-        m_cancelledQueries.erase(m_cancelledQueries.begin(), m_cancelledQueries.end());
-    }
-}
-
-
 void DrillClientImpl::clearMapEntries(DrillClientQueryResult* pQueryResult){
     std::map<int, DrillClientQueryResult*>::iterator iter;
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
@@ -981,13 +992,6 @@ void DrillClientImpl::clearMapEntries(DrillClientQueryResult* pQueryResult){
         }
     }
     if(!m_queryResults.empty()){
-        // Save the query id and state and free when the query is complete
-        if(pQueryResult->getQueryState()!=exec::shared::QueryResult_QueryState_COMPLETED
-                && pQueryResult->getQueryState()!=exec::shared::QueryResult_QueryState_FAILED){
-            exec::shared::QueryId* pQueryId=new exec::shared::QueryId();
-            pQueryId->CopyFrom(pQueryResult->getQueryId());
-            m_cancelledQueries.insert(pQueryId);
-        }
         std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::iterator it;
         for(it=m_queryResults.begin(); it!=m_queryResults.end(); it++) {
             if(pQueryResult==(DrillClientQueryResult*)it->second){
@@ -1017,7 +1021,7 @@ void DrillClientImpl::sendCancel(exec::shared::QueryId* pQueryId){
 
 // This COPIES the FieldMetadata definition for the record batch.  ColumnDefs held by this
 // class are used by the async callbacks.
-status_t DrillClientQueryResult::setupColumnDefs(exec::shared::QueryResult* pQueryResult) {
+status_t DrillClientQueryResult::setupColumnDefs(exec::shared::QueryData* pQueryData) {
     bool hasSchemaChanged=false;
     bool isFirstIter=false;
     boost::lock_guard<boost::mutex> schLock(this->m_schemaMutex);
@@ -1035,11 +1039,11 @@ status_t DrillClientQueryResult::setupColumnDefs(exec::shared::QueryResult* pQue
         }
     }
     m_columnDefs->clear();
-    size_t numFields=pQueryResult->def().field_size();
+    size_t numFields=pQueryData->def().field_size();
     if (numFields > 0){
         for(size_t i=0; i<numFields; i++){
             Drill::FieldMetadata* fmd= new Drill::FieldMetadata;
-            fmd->set(pQueryResult->def().field(i));
+            fmd->set(pQueryData->def().field(i));
             this->m_columnDefs->push_back(fmd);
 
             //Look for changes in the vector and trigger a Schema change event if necessary.
@@ -1077,19 +1081,21 @@ status_t DrillClientQueryResult::defaultQueryResultsListener(void* ctx,
     DRILL_LOG(LOG_TRACE) << "Query result listener called" << std::endl;
     //check if the query has been canceled. IF so then return FAILURE. Caller will send cancel to the server.
     if(this->m_bCancel){
-        delete b;
+        if(b!=NULL) delete b;
         return QRY_FAILURE;
     }
     if (!err) {
         // signal the cond var
         {
-            #ifdef DEBUG
-            DRILL_LOG(LOG_DEBUG)<<debugPrintQid(b->getQueryResult()->query_id())
-                << "Query result listener saved result to queue." << std::endl;
-            #endif
-            boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
-            this->m_recordBatches.push(b);
-            this->m_bHasData=true;
+            if(b!=NULL){
+#ifdef DEBUG
+                DRILL_LOG(LOG_DEBUG)<<debugPrintQid(b->getQueryResult()->query_id())
+                    << "Query result listener saved result to queue." << std::endl;
+#endif
+                boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
+                this->m_recordBatches.push(b);
+                this->m_bHasData=true;
+            }
         }
         m_cv.notify_one();
     }else{
@@ -1100,11 +1106,11 @@ status_t DrillClientQueryResult::defaultQueryResultsListener(void* ctx,
 
 RecordBatch*  DrillClientQueryResult::peekNext(){
     RecordBatch* pRecordBatch=NULL;
+    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     //if no more data, return NULL;
     if(!m_bIsQueryPending) return NULL;
-    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     DRILL_LOG(LOG_TRACE) << "Synchronous read waiting for data." << std::endl;
-    while(!this->m_bHasData && !m_bHasError) {
+    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending) {
         this->m_cv.wait(cvLock);
     }
     // READ but not remove first element from queue
@@ -1114,6 +1120,7 @@ RecordBatch*  DrillClientQueryResult::peekNext(){
 
 RecordBatch*  DrillClientQueryResult::getNext() {
     RecordBatch* pRecordBatch=NULL;
+    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     //if no more data, return NULL;
     if(!m_bIsQueryPending){
         DRILL_LOG(LOG_TRACE) << "Query is done." << std::endl;
@@ -1123,9 +1130,8 @@ RecordBatch*  DrillClientQueryResult::getNext() {
         return NULL;
     }
 
-    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     DRILL_LOG(LOG_TRACE) << "Synchronous read waiting for data." << std::endl;
-    while(!this->m_bHasData && !m_bHasError){
+    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending){
         this->m_cv.wait(cvLock);
     }
     // remove first element from queue
@@ -1133,16 +1139,16 @@ RecordBatch*  DrillClientQueryResult::getNext() {
     this->m_recordBatches.pop();
     this->m_bHasData=!this->m_recordBatches.empty();
     // if vector is empty, set m_bHasDataPending to false;
-    m_bIsQueryPending=!(this->m_recordBatches.empty()&&m_bIsLastChunk);
+    m_bIsQueryPending=!(this->m_recordBatches.empty()&&m_queryState==exec::shared::QueryResult_QueryState_COMPLETED);
     return pRecordBatch;
 }
 
 // Blocks until data is available
 void DrillClientQueryResult::waitForData() {
+    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     //if no more data, return NULL;
     if(!m_bIsQueryPending) return;
-    boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
-    while(!this->m_bHasData && !m_bHasError) {
+    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending) {
         this->m_cv.wait(cvLock);
     }
 }
@@ -1164,9 +1170,9 @@ void DrillClientQueryResult::signalError(DrillClientError* pErr){
         }else{
             defaultQueryResultsListener(this, NULL, pErr);
         }
-        m_bIsQueryPending=false;
         {
             boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
+            m_bIsQueryPending=false;
             m_bHasData=false;
             m_bHasError=true;
         }
@@ -1176,6 +1182,24 @@ void DrillClientQueryResult::signalError(DrillClientError* pErr){
     return;
 }
 
+void DrillClientQueryResult::signalComplete(){
+    pfnQueryResultsListener pResultsListener=this->m_pResultsListener;
+    if(pResultsListener!=NULL){
+        pResultsListener(this, NULL, NULL);
+    }else{
+        defaultQueryResultsListener(this, NULL, NULL);
+    }
+    {
+        boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
+        m_bIsQueryPending=false;
+        m_bIsQueryPending=!(this->m_recordBatches.empty()&&m_queryState==exec::shared::QueryResult_QueryState_COMPLETED);
+        m_bHasError=false;
+    }
+    //Signal the cv in case there is a client waiting for data already.
+    m_cv.notify_one();
+    return;
+}
+
 void DrillClientQueryResult::clearAndDestroy(){
     //free memory allocated for FieldMetadata objects saved in m_columnDefs;
     if(!m_columnDefs->empty()){
@@ -1207,7 +1231,7 @@ void DrillClientQueryResult::clearAndDestroy(){
     }
     if(m_pError!=NULL){
         delete m_pError; m_pError=NULL;
-}
+    }
 }
 
 char ZookeeperImpl::s_drillRoot[]="/drill/";

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/clientlib/drillClientImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.hpp b/contrib/native/client/src/clientlib/drillClientImpl.hpp
index 33f81db..95fe922 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.hpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.hpp
@@ -127,14 +127,19 @@ class DrillClientQueryResult{
 
     void setQueryState(exec::shared::QueryResult_QueryState s){ m_queryState = s;}
     exec::shared::QueryResult_QueryState getQueryState(){ return m_queryState;}
+    void setIsQueryPending(bool isPending){
+        boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
+        m_bIsQueryPending=isPending;
+    }
 
     private:
-    status_t setupColumnDefs(exec::shared::QueryResult* pQueryResult);
+    status_t setupColumnDefs(exec::shared::QueryData* pQueryData);
     status_t defaultQueryResultsListener(void* ctx, RecordBatch* b, DrillClientError* err);
     // Construct a DrillClientError object, set the appropriate state and signal any listeners, condition variables.
     // Also used when a query is cancelled or when a query completed response is received.
     // Error object is now owned by the DrillClientQueryResult object.
     void signalError(DrillClientError* pErr);
+    void signalComplete();
     void clearAndDestroy();
 
 
@@ -212,7 +217,6 @@ class DrillClientImpl{
                 this->m_pWork = NULL;
             }
 
-            clearCancelledEntries();
             m_deadlineTimer.cancel();
             m_io_service.stop();
             boost::system::error_code ignorederr;
@@ -272,13 +276,16 @@ class DrillClientImpl{
                 InBoundRpcMessage& msg,
                 boost::system::error_code& error);
         status_t processQueryResult(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg);
+        status_t processQueryData(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg);
         status_t processCancelledQueryResult( exec::shared::QueryId& qid, exec::shared::QueryResult* qr);
         status_t processQueryId(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg );
+        DrillClientQueryResult* findQueryResult(exec::shared::QueryId& qid);
         status_t processQueryStatusResult( exec::shared::QueryResult* qr,
                 DrillClientQueryResult* pDrillClientQueryResult);
         void handleReadTimeout(const boost::system::error_code & err);
         void handleRead(ByteBuf_t _buf, const boost::system::error_code & err, size_t bytes_transferred) ;
-        status_t validateMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valError);
+        status_t validateDataMessage(InBoundRpcMessage& msg, exec::shared::QueryData& qd, std::string& valError);
+        status_t validateResultMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valError);
         connectionStatus_t handleConnError(connectionStatus_t status, std::string msg);
         status_t handleQryError(status_t status, std::string msg, DrillClientQueryResult* pQueryResult);
         status_t handleQryError(status_t status,
@@ -291,7 +298,6 @@ class DrillClientImpl{
                 DrillClientQueryResult* pQueryResult);
         void broadcastError(DrillClientError* pErr);
         void clearMapEntries(DrillClientQueryResult* pQueryResult);
-        void clearCancelledEntries();
         void sendAck(InBoundRpcMessage& msg, bool isOk);
         void sendCancel(exec::shared::QueryId* pQueryId);
 
@@ -335,11 +341,6 @@ class DrillClientImpl{
 
         // Map of query id to query result for currently executing queries
         std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId> m_queryResults;
-        //
-        // State for every Query id whose queries have result data pending but which
-        // have been cancelled and whose resources have been released by the client application.
-        // The entry is cleared when the state changes to completed or failed.
-        std::set<exec::shared::QueryId*, compareQueryId> m_cancelledQueries;
 
 };
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/clientlib/recordBatch.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/recordBatch.cpp b/contrib/native/client/src/clientlib/recordBatch.cpp
index 44140b2..c6c033b 100644
--- a/contrib/native/client/src/clientlib/recordBatch.cpp
+++ b/contrib/native/client/src/clientlib/recordBatch.cpp
@@ -306,7 +306,7 @@ ret_t FieldBatch::loadNull(size_t nRecords){
     return RET_SUCCESS;
 }
     
-RecordBatch::RecordBatch(exec::shared::QueryResult* pResult, AllocatedBufferPtr r, ByteBuf_t b)
+RecordBatch::RecordBatch(exec::shared::QueryData* pResult, AllocatedBufferPtr r, ByteBuf_t b)
     :m_fieldDefs(new(std::vector<Drill::FieldMetadata*>)){
         m_pQueryResult=pResult;
         m_pRecordBatchDef=&pResult->def();
@@ -398,7 +398,7 @@ size_t RecordBatch::getNumFields(){
 }
 
 bool RecordBatch::isLastChunk(){
-    return m_pQueryResult->is_last_chunk(); 
+    return false;
 }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/include/drill/common.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/common.hpp b/contrib/native/client/src/include/drill/common.hpp
index 6560692..72b9a98 100644
--- a/contrib/native/client/src/include/drill/common.hpp
+++ b/contrib/native/client/src/include/drill/common.hpp
@@ -34,7 +34,7 @@
 #include <vector>
 #include <boost/shared_ptr.hpp>
 
-#define DRILL_RPC_VERSION 3
+#define DRILL_RPC_VERSION 4
 
 #define LENGTH_PREFIX_MAX_LENGTH 5
 #define LEN_PREFIX_BUFLEN LENGTH_PREFIX_MAX_LENGTH

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/include/drill/drillClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index 9289df3..c288c70 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -167,7 +167,9 @@ typedef void* QueryHandle_t;
  * Query Results listener callback. This function is called for every record batch after it has
  * been received and decoded. The listener function should return a status.
  * If the listener returns failure, the query will be canceled.
- *
+ * The listener is also called one last time when the query is completed or gets an error. In that
+ * case the RecordBatch Parameter is NULL. The DrillClientError parameter is NULL is there was no
+ * error oterwise it will have a valid DrillClientError object.
  * DrillClientQueryResult will hold a listener & listener contxt for the call back function
  */
 typedef status_t (*pfnQueryResultsListener)(QueryHandle_t ctx, RecordBatch* b, DrillClientError* err);

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/include/drill/recordBatch.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/recordBatch.hpp b/contrib/native/client/src/include/drill/recordBatch.hpp
index 92a4c3ad..12cbad4 100644
--- a/contrib/native/client/src/include/drill/recordBatch.hpp
+++ b/contrib/native/client/src/include/drill/recordBatch.hpp
@@ -56,6 +56,7 @@ namespace exec{
         class SerializedField;
         class RecordBatchDef;
         class QueryResult;
+        class QueryData;
     };
 };
 
@@ -863,7 +864,7 @@ class DECLSPEC_DRILL_CLIENT RecordBatch{
         //m_allocatedBuffer is the memory block allocated to hold the incoming RPC message. Record Batches operate on
         //slices of the allocated buffer. The first slice (the first Field Batch), begins at m_buffer. Data in the
         //allocated buffer before m_buffer is mostly the RPC header, and the QueryResult object.
-        RecordBatch(exec::shared::QueryResult* pResult, AllocatedBufferPtr r, ByteBuf_t b);
+        RecordBatch(exec::shared::QueryData* pResult, AllocatedBufferPtr r, ByteBuf_t b);
 
         ~RecordBatch();
 
@@ -876,7 +877,7 @@ class DECLSPEC_DRILL_CLIENT RecordBatch{
         size_t getNumRecords(){ return m_numRecords;}
         std::vector<FieldBatch*>& getFields(){ return m_fields;}
         size_t getNumFields();
-        bool isLastChunk();
+        DEPRECATED bool isLastChunk();
 
         boost::shared_ptr<std::vector<Drill::FieldMetadata*> > getColumnDefs(){ return m_fieldDefs;}
 
@@ -902,10 +903,10 @@ class DECLSPEC_DRILL_CLIENT RecordBatch{
         bool hasSchemaChanged(){ return m_bHasSchemaChanged;}
 
         #ifdef DEBUG
-        const exec::shared::QueryResult* getQueryResult(){ return this->m_pQueryResult;}
+        const exec::shared::QueryData* getQueryResult(){ return this->m_pQueryResult;}
         #endif
     private:
-        const exec::shared::QueryResult* m_pQueryResult;
+        const exec::shared::QueryData* m_pQueryResult;
         const exec::shared::RecordBatchDef* m_pRecordBatchDef;
         AllocatedBufferPtr m_allocatedBuffer;
         ByteBuf_t m_buffer;

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/BitData.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/BitData.pb.cc b/contrib/native/client/src/protobuf/BitData.pb.cc
index ef4f99d..57bceff 100644
--- a/contrib/native/client/src/protobuf/BitData.pb.cc
+++ b/contrib/native/client/src/protobuf/BitData.pb.cc
@@ -43,10 +43,9 @@ void protobuf_AssignDesc_BitData_2eproto() {
       "BitData.proto");
   GOOGLE_CHECK(file != NULL);
   BitClientHandshake_descriptor_ = file->message_type(0);
-  static const int BitClientHandshake_offsets_[3] = {
+  static const int BitClientHandshake_offsets_[2] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitClientHandshake, rpc_version_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitClientHandshake, channel_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitClientHandshake, handle_),
   };
   BitClientHandshake_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -75,8 +74,10 @@ void protobuf_AssignDesc_BitData_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(BitServerHandshake));
   FragmentRecordBatch_descriptor_ = file->message_type(2);
-  static const int FragmentRecordBatch_offsets_[6] = {
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, handle_),
+  static const int FragmentRecordBatch_offsets_[8] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, query_id_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, receiving_major_fragment_id_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, receiving_minor_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, sending_major_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, sending_minor_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentRecordBatch, def_),
@@ -138,20 +139,21 @@ void protobuf_AddDesc_BitData_2eproto() {
   ::google::protobuf::DescriptorPool::InternalAddGeneratedFile(
     "\n\rBitData.proto\022\rexec.bit.data\032\025Executio"
     "nProtos.proto\032\022Coordination.proto\032\023UserB"
-    "itShared.proto\"\207\001\n\022BitClientHandshake\022\023\n"
-    "\013rpc_version\030\001 \001(\005\0222\n\007channel\030\002 \001(\0162\027.ex"
-    "ec.shared.RpcChannel:\010BIT_DATA\022(\n\006handle"
-    "\030\003 \001(\0132\030.exec.bit.FragmentHandle\")\n\022BitS"
-    "erverHandshake\022\023\n\013rpc_version\030\001 \001(\005\"\342\001\n\023"
-    "FragmentRecordBatch\022(\n\006handle\030\001 \001(\0132\030.ex"
-    "ec.bit.FragmentHandle\022!\n\031sending_major_f"
-    "ragment_id\030\002 \001(\005\022!\n\031sending_minor_fragme"
-    "nt_id\030\003 \001(\005\022(\n\003def\030\004 \001(\0132\033.exec.shared.R"
-    "ecordBatchDef\022\023\n\013isLastBatch\030\005 \001(\010\022\034\n\ris"
-    "OutOfMemory\030\006 \001(\010:\005false*D\n\007RpcType\022\r\n\tH"
-    "ANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n\020REQ_"
-    "RECORD_BATCH\020\003B(\n\033org.apache.drill.exec."
-    "protoB\007BitDataH\001", 616);
+    "itShared.proto\"]\n\022BitClientHandshake\022\023\n\013"
+    "rpc_version\030\001 \001(\005\0222\n\007channel\030\002 \001(\0162\027.exe"
+    "c.shared.RpcChannel:\010BIT_DATA\")\n\022BitServ"
+    "erHandshake\022\023\n\013rpc_version\030\001 \001(\005\"\252\002\n\023Fra"
+    "gmentRecordBatch\022&\n\010query_id\030\001 \001(\0132\024.exe"
+    "c.shared.QueryId\022#\n\033receiving_major_frag"
+    "ment_id\030\002 \001(\005\022#\n\033receiving_minor_fragmen"
+    "t_id\030\003 \003(\005\022!\n\031sending_major_fragment_id\030"
+    "\004 \001(\005\022!\n\031sending_minor_fragment_id\030\005 \001(\005"
+    "\022(\n\003def\030\006 \001(\0132\033.exec.shared.RecordBatchD"
+    "ef\022\023\n\013isLastBatch\030\007 \001(\010\022\034\n\risOutOfMemory"
+    "\030\010 \001(\010:\005false*D\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022"
+    "\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n\020REQ_RECORD_BATC"
+    "H\020\003B(\n\033org.apache.drill.exec.protoB\007BitD"
+    "ataH\001", 645);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "BitData.proto", &protobuf_RegisterTypes);
   BitClientHandshake::default_instance_ = new BitClientHandshake();
@@ -191,7 +193,6 @@ bool RpcType_IsValid(int value) {
 #ifndef _MSC_VER
 const int BitClientHandshake::kRpcVersionFieldNumber;
 const int BitClientHandshake::kChannelFieldNumber;
-const int BitClientHandshake::kHandleFieldNumber;
 #endif  // !_MSC_VER
 
 BitClientHandshake::BitClientHandshake()
@@ -200,7 +201,6 @@ BitClientHandshake::BitClientHandshake()
 }
 
 void BitClientHandshake::InitAsDefaultInstance() {
-  handle_ = const_cast< ::exec::bit::FragmentHandle*>(&::exec::bit::FragmentHandle::default_instance());
 }
 
 BitClientHandshake::BitClientHandshake(const BitClientHandshake& from)
@@ -213,7 +213,6 @@ void BitClientHandshake::SharedCtor() {
   _cached_size_ = 0;
   rpc_version_ = 0;
   channel_ = 1;
-  handle_ = NULL;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -223,7 +222,6 @@ BitClientHandshake::~BitClientHandshake() {
 
 void BitClientHandshake::SharedDtor() {
   if (this != default_instance_) {
-    delete handle_;
   }
 }
 
@@ -252,9 +250,6 @@ void BitClientHandshake::Clear() {
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
     rpc_version_ = 0;
     channel_ = 1;
-    if (has_handle()) {
-      if (handle_ != NULL) handle_->::exec::bit::FragmentHandle::Clear();
-    }
   }
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
@@ -298,20 +293,6 @@ bool BitClientHandshake::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(26)) goto parse_handle;
-        break;
-      }
-
-      // optional .exec.bit.FragmentHandle handle = 3;
-      case 3: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
-         parse_handle:
-          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-               input, mutable_handle()));
-        } else {
-          goto handle_uninterpreted;
-        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -345,12 +326,6 @@ void BitClientHandshake::SerializeWithCachedSizes(
       2, this->channel(), output);
   }
 
-  // optional .exec.bit.FragmentHandle handle = 3;
-  if (has_handle()) {
-    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      3, this->handle(), output);
-  }
-
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -370,13 +345,6 @@ void BitClientHandshake::SerializeWithCachedSizes(
       2, this->channel(), target);
   }
 
-  // optional .exec.bit.FragmentHandle handle = 3;
-  if (has_handle()) {
-    target = ::google::protobuf::internal::WireFormatLite::
-      WriteMessageNoVirtualToArray(
-        3, this->handle(), target);
-  }
-
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -401,13 +369,6 @@ int BitClientHandshake::ByteSize() const {
         ::google::protobuf::internal::WireFormatLite::EnumSize(this->channel());
     }
 
-    // optional .exec.bit.FragmentHandle handle = 3;
-    if (has_handle()) {
-      total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-          this->handle());
-    }
-
   }
   if (!unknown_fields().empty()) {
     total_size +=
@@ -441,9 +402,6 @@ void BitClientHandshake::MergeFrom(const BitClientHandshake& from) {
     if (from.has_channel()) {
       set_channel(from.channel());
     }
-    if (from.has_handle()) {
-      mutable_handle()->::exec::bit::FragmentHandle::MergeFrom(from.handle());
-    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -469,7 +427,6 @@ void BitClientHandshake::Swap(BitClientHandshake* other) {
   if (other != this) {
     std::swap(rpc_version_, other->rpc_version_);
     std::swap(channel_, other->channel_);
-    std::swap(handle_, other->handle_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
@@ -696,7 +653,9 @@ void BitServerHandshake::Swap(BitServerHandshake* other) {
 // ===================================================================
 
 #ifndef _MSC_VER
-const int FragmentRecordBatch::kHandleFieldNumber;
+const int FragmentRecordBatch::kQueryIdFieldNumber;
+const int FragmentRecordBatch::kReceivingMajorFragmentIdFieldNumber;
+const int FragmentRecordBatch::kReceivingMinorFragmentIdFieldNumber;
 const int FragmentRecordBatch::kSendingMajorFragmentIdFieldNumber;
 const int FragmentRecordBatch::kSendingMinorFragmentIdFieldNumber;
 const int FragmentRecordBatch::kDefFieldNumber;
@@ -710,7 +669,7 @@ FragmentRecordBatch::FragmentRecordBatch()
 }
 
 void FragmentRecordBatch::InitAsDefaultInstance() {
-  handle_ = const_cast< ::exec::bit::FragmentHandle*>(&::exec::bit::FragmentHandle::default_instance());
+  query_id_ = const_cast< ::exec::shared::QueryId*>(&::exec::shared::QueryId::default_instance());
   def_ = const_cast< ::exec::shared::RecordBatchDef*>(&::exec::shared::RecordBatchDef::default_instance());
 }
 
@@ -722,7 +681,8 @@ FragmentRecordBatch::FragmentRecordBatch(const FragmentRecordBatch& from)
 
 void FragmentRecordBatch::SharedCtor() {
   _cached_size_ = 0;
-  handle_ = NULL;
+  query_id_ = NULL;
+  receiving_major_fragment_id_ = 0;
   sending_major_fragment_id_ = 0;
   sending_minor_fragment_id_ = 0;
   def_ = NULL;
@@ -737,7 +697,7 @@ FragmentRecordBatch::~FragmentRecordBatch() {
 
 void FragmentRecordBatch::SharedDtor() {
   if (this != default_instance_) {
-    delete handle_;
+    delete query_id_;
     delete def_;
   }
 }
@@ -765,9 +725,10 @@ FragmentRecordBatch* FragmentRecordBatch::New() const {
 
 void FragmentRecordBatch::Clear() {
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    if (has_handle()) {
-      if (handle_ != NULL) handle_->::exec::bit::FragmentHandle::Clear();
+    if (has_query_id()) {
+      if (query_id_ != NULL) query_id_->::exec::shared::QueryId::Clear();
     }
+    receiving_major_fragment_id_ = 0;
     sending_major_fragment_id_ = 0;
     sending_minor_fragment_id_ = 0;
     if (has_def()) {
@@ -776,6 +737,7 @@ void FragmentRecordBatch::Clear() {
     islastbatch_ = false;
     isoutofmemory_ = false;
   }
+  receiving_minor_fragment_id_.Clear();
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
 }
@@ -786,23 +748,61 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
   ::google::protobuf::uint32 tag;
   while ((tag = input->ReadTag()) != 0) {
     switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
-      // optional .exec.bit.FragmentHandle handle = 1;
+      // optional .exec.shared.QueryId query_id = 1;
       case 1: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
           DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-               input, mutable_handle()));
+               input, mutable_query_id()));
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(16)) goto parse_sending_major_fragment_id;
+        if (input->ExpectTag(16)) goto parse_receiving_major_fragment_id;
         break;
       }
 
-      // optional int32 sending_major_fragment_id = 2;
+      // optional int32 receiving_major_fragment_id = 2;
       case 2: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_receiving_major_fragment_id:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &receiving_major_fragment_id_)));
+          set_has_receiving_major_fragment_id();
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(24)) goto parse_receiving_minor_fragment_id;
+        break;
+      }
+
+      // repeated int32 receiving_minor_fragment_id = 3;
+      case 3: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_receiving_minor_fragment_id:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadRepeatedPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 1, 24, input, this->mutable_receiving_minor_fragment_id())));
+        } else if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag)
+                   == ::google::protobuf::internal::WireFormatLite::
+                      WIRETYPE_LENGTH_DELIMITED) {
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPackedPrimitiveNoInline<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, this->mutable_receiving_minor_fragment_id())));
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(24)) goto parse_receiving_minor_fragment_id;
+        if (input->ExpectTag(32)) goto parse_sending_major_fragment_id;
+        break;
+      }
+
+      // optional int32 sending_major_fragment_id = 4;
+      case 4: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
          parse_sending_major_fragment_id:
           DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
                    ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
@@ -811,12 +811,12 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(24)) goto parse_sending_minor_fragment_id;
+        if (input->ExpectTag(40)) goto parse_sending_minor_fragment_id;
         break;
       }
 
-      // optional int32 sending_minor_fragment_id = 3;
-      case 3: {
+      // optional int32 sending_minor_fragment_id = 5;
+      case 5: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
          parse_sending_minor_fragment_id:
@@ -827,12 +827,12 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(34)) goto parse_def;
+        if (input->ExpectTag(50)) goto parse_def;
         break;
       }
 
-      // optional .exec.shared.RecordBatchDef def = 4;
-      case 4: {
+      // optional .exec.shared.RecordBatchDef def = 6;
+      case 6: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
          parse_def:
@@ -841,12 +841,12 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(40)) goto parse_isLastBatch;
+        if (input->ExpectTag(56)) goto parse_isLastBatch;
         break;
       }
 
-      // optional bool isLastBatch = 5;
-      case 5: {
+      // optional bool isLastBatch = 7;
+      case 7: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
          parse_isLastBatch:
@@ -857,12 +857,12 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(48)) goto parse_isOutOfMemory;
+        if (input->ExpectTag(64)) goto parse_isOutOfMemory;
         break;
       }
 
-      // optional bool isOutOfMemory = 6 [default = false];
-      case 6: {
+      // optional bool isOutOfMemory = 8 [default = false];
+      case 8: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
          parse_isOutOfMemory:
@@ -895,36 +895,47 @@ bool FragmentRecordBatch::MergePartialFromCodedStream(
 
 void FragmentRecordBatch::SerializeWithCachedSizes(
     ::google::protobuf::io::CodedOutputStream* output) const {
-  // optional .exec.bit.FragmentHandle handle = 1;
-  if (has_handle()) {
+  // optional .exec.shared.QueryId query_id = 1;
+  if (has_query_id()) {
     ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      1, this->handle(), output);
+      1, this->query_id(), output);
   }
 
-  // optional int32 sending_major_fragment_id = 2;
+  // optional int32 receiving_major_fragment_id = 2;
+  if (has_receiving_major_fragment_id()) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(2, this->receiving_major_fragment_id(), output);
+  }
+
+  // repeated int32 receiving_minor_fragment_id = 3;
+  for (int i = 0; i < this->receiving_minor_fragment_id_size(); i++) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(
+      3, this->receiving_minor_fragment_id(i), output);
+  }
+
+  // optional int32 sending_major_fragment_id = 4;
   if (has_sending_major_fragment_id()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt32(2, this->sending_major_fragment_id(), output);
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(4, this->sending_major_fragment_id(), output);
   }
 
-  // optional int32 sending_minor_fragment_id = 3;
+  // optional int32 sending_minor_fragment_id = 5;
   if (has_sending_minor_fragment_id()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt32(3, this->sending_minor_fragment_id(), output);
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(5, this->sending_minor_fragment_id(), output);
   }
 
-  // optional .exec.shared.RecordBatchDef def = 4;
+  // optional .exec.shared.RecordBatchDef def = 6;
   if (has_def()) {
     ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      4, this->def(), output);
+      6, this->def(), output);
   }
 
-  // optional bool isLastBatch = 5;
+  // optional bool isLastBatch = 7;
   if (has_islastbatch()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(5, this->islastbatch(), output);
+    ::google::protobuf::internal::WireFormatLite::WriteBool(7, this->islastbatch(), output);
   }
 
-  // optional bool isOutOfMemory = 6 [default = false];
+  // optional bool isOutOfMemory = 8 [default = false];
   if (has_isoutofmemory()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(6, this->isoutofmemory(), output);
+    ::google::protobuf::internal::WireFormatLite::WriteBool(8, this->isoutofmemory(), output);
   }
 
   if (!unknown_fields().empty()) {
@@ -935,38 +946,49 @@ void FragmentRecordBatch::SerializeWithCachedSizes(
 
 ::google::protobuf::uint8* FragmentRecordBatch::SerializeWithCachedSizesToArray(
     ::google::protobuf::uint8* target) const {
-  // optional .exec.bit.FragmentHandle handle = 1;
-  if (has_handle()) {
+  // optional .exec.shared.QueryId query_id = 1;
+  if (has_query_id()) {
     target = ::google::protobuf::internal::WireFormatLite::
       WriteMessageNoVirtualToArray(
-        1, this->handle(), target);
+        1, this->query_id(), target);
+  }
+
+  // optional int32 receiving_major_fragment_id = 2;
+  if (has_receiving_major_fragment_id()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(2, this->receiving_major_fragment_id(), target);
+  }
+
+  // repeated int32 receiving_minor_fragment_id = 3;
+  for (int i = 0; i < this->receiving_minor_fragment_id_size(); i++) {
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteInt32ToArray(3, this->receiving_minor_fragment_id(i), target);
   }
 
-  // optional int32 sending_major_fragment_id = 2;
+  // optional int32 sending_major_fragment_id = 4;
   if (has_sending_major_fragment_id()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(2, this->sending_major_fragment_id(), target);
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(4, this->sending_major_fragment_id(), target);
   }
 
-  // optional int32 sending_minor_fragment_id = 3;
+  // optional int32 sending_minor_fragment_id = 5;
   if (has_sending_minor_fragment_id()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(3, this->sending_minor_fragment_id(), target);
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(5, this->sending_minor_fragment_id(), target);
   }
 
-  // optional .exec.shared.RecordBatchDef def = 4;
+  // optional .exec.shared.RecordBatchDef def = 6;
   if (has_def()) {
     target = ::google::protobuf::internal::WireFormatLite::
       WriteMessageNoVirtualToArray(
-        4, this->def(), target);
+        6, this->def(), target);
   }
 
-  // optional bool isLastBatch = 5;
+  // optional bool isLastBatch = 7;
   if (has_islastbatch()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(5, this->islastbatch(), target);
+    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(7, this->islastbatch(), target);
   }
 
-  // optional bool isOutOfMemory = 6 [default = false];
+  // optional bool isOutOfMemory = 8 [default = false];
   if (has_isoutofmemory()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(6, this->isoutofmemory(), target);
+    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(8, this->isoutofmemory(), target);
   }
 
   if (!unknown_fields().empty()) {
@@ -980,45 +1002,62 @@ int FragmentRecordBatch::ByteSize() const {
   int total_size = 0;
 
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    // optional .exec.bit.FragmentHandle handle = 1;
-    if (has_handle()) {
+    // optional .exec.shared.QueryId query_id = 1;
+    if (has_query_id()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-          this->handle());
+          this->query_id());
     }
 
-    // optional int32 sending_major_fragment_id = 2;
+    // optional int32 receiving_major_fragment_id = 2;
+    if (has_receiving_major_fragment_id()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::Int32Size(
+          this->receiving_major_fragment_id());
+    }
+
+    // optional int32 sending_major_fragment_id = 4;
     if (has_sending_major_fragment_id()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::Int32Size(
           this->sending_major_fragment_id());
     }
 
-    // optional int32 sending_minor_fragment_id = 3;
+    // optional int32 sending_minor_fragment_id = 5;
     if (has_sending_minor_fragment_id()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::Int32Size(
           this->sending_minor_fragment_id());
     }
 
-    // optional .exec.shared.RecordBatchDef def = 4;
+    // optional .exec.shared.RecordBatchDef def = 6;
     if (has_def()) {
       total_size += 1 +
         ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
           this->def());
     }
 
-    // optional bool isLastBatch = 5;
+    // optional bool isLastBatch = 7;
     if (has_islastbatch()) {
       total_size += 1 + 1;
     }
 
-    // optional bool isOutOfMemory = 6 [default = false];
+    // optional bool isOutOfMemory = 8 [default = false];
     if (has_isoutofmemory()) {
       total_size += 1 + 1;
     }
 
   }
+  // repeated int32 receiving_minor_fragment_id = 3;
+  {
+    int data_size = 0;
+    for (int i = 0; i < this->receiving_minor_fragment_id_size(); i++) {
+      data_size += ::google::protobuf::internal::WireFormatLite::
+        Int32Size(this->receiving_minor_fragment_id(i));
+    }
+    total_size += 1 * this->receiving_minor_fragment_id_size() + data_size;
+  }
+
   if (!unknown_fields().empty()) {
     total_size +=
       ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
@@ -1044,9 +1083,13 @@ void FragmentRecordBatch::MergeFrom(const ::google::protobuf::Message& from) {
 
 void FragmentRecordBatch::MergeFrom(const FragmentRecordBatch& from) {
   GOOGLE_CHECK_NE(&from, this);
+  receiving_minor_fragment_id_.MergeFrom(from.receiving_minor_fragment_id_);
   if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    if (from.has_handle()) {
-      mutable_handle()->::exec::bit::FragmentHandle::MergeFrom(from.handle());
+    if (from.has_query_id()) {
+      mutable_query_id()->::exec::shared::QueryId::MergeFrom(from.query_id());
+    }
+    if (from.has_receiving_major_fragment_id()) {
+      set_receiving_major_fragment_id(from.receiving_major_fragment_id());
     }
     if (from.has_sending_major_fragment_id()) {
       set_sending_major_fragment_id(from.sending_major_fragment_id());
@@ -1086,7 +1129,9 @@ bool FragmentRecordBatch::IsInitialized() const {
 
 void FragmentRecordBatch::Swap(FragmentRecordBatch* other) {
   if (other != this) {
-    std::swap(handle_, other->handle_);
+    std::swap(query_id_, other->query_id_);
+    std::swap(receiving_major_fragment_id_, other->receiving_major_fragment_id_);
+    receiving_minor_fragment_id_.Swap(&other->receiving_minor_fragment_id_);
     std::swap(sending_major_fragment_id_, other->sending_major_fragment_id_);
     std::swap(sending_minor_fragment_id_, other->sending_minor_fragment_id_);
     std::swap(def_, other->def_);

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/BitData.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/BitData.pb.h b/contrib/native/client/src/protobuf/BitData.pb.h
index f1f9353..806d7f7 100644
--- a/contrib/native/client/src/protobuf/BitData.pb.h
+++ b/contrib/native/client/src/protobuf/BitData.pb.h
@@ -134,32 +134,20 @@ class BitClientHandshake : public ::google::protobuf::Message {
   inline ::exec::shared::RpcChannel channel() const;
   inline void set_channel(::exec::shared::RpcChannel value);
 
-  // optional .exec.bit.FragmentHandle handle = 3;
-  inline bool has_handle() const;
-  inline void clear_handle();
-  static const int kHandleFieldNumber = 3;
-  inline const ::exec::bit::FragmentHandle& handle() const;
-  inline ::exec::bit::FragmentHandle* mutable_handle();
-  inline ::exec::bit::FragmentHandle* release_handle();
-  inline void set_allocated_handle(::exec::bit::FragmentHandle* handle);
-
   // @@protoc_insertion_point(class_scope:exec.bit.data.BitClientHandshake)
  private:
   inline void set_has_rpc_version();
   inline void clear_has_rpc_version();
   inline void set_has_channel();
   inline void clear_has_channel();
-  inline void set_has_handle();
-  inline void clear_has_handle();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
   ::google::protobuf::int32 rpc_version_;
   int channel_;
-  ::exec::bit::FragmentHandle* handle_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(2 + 31) / 32];
 
   friend void  protobuf_AddDesc_BitData_2eproto();
   friend void protobuf_AssignDesc_BitData_2eproto();
@@ -306,56 +294,77 @@ class FragmentRecordBatch : public ::google::protobuf::Message {
 
   // accessors -------------------------------------------------------
 
-  // optional .exec.bit.FragmentHandle handle = 1;
-  inline bool has_handle() const;
-  inline void clear_handle();
-  static const int kHandleFieldNumber = 1;
-  inline const ::exec::bit::FragmentHandle& handle() const;
-  inline ::exec::bit::FragmentHandle* mutable_handle();
-  inline ::exec::bit::FragmentHandle* release_handle();
-  inline void set_allocated_handle(::exec::bit::FragmentHandle* handle);
-
-  // optional int32 sending_major_fragment_id = 2;
+  // optional .exec.shared.QueryId query_id = 1;
+  inline bool has_query_id() const;
+  inline void clear_query_id();
+  static const int kQueryIdFieldNumber = 1;
+  inline const ::exec::shared::QueryId& query_id() const;
+  inline ::exec::shared::QueryId* mutable_query_id();
+  inline ::exec::shared::QueryId* release_query_id();
+  inline void set_allocated_query_id(::exec::shared::QueryId* query_id);
+
+  // optional int32 receiving_major_fragment_id = 2;
+  inline bool has_receiving_major_fragment_id() const;
+  inline void clear_receiving_major_fragment_id();
+  static const int kReceivingMajorFragmentIdFieldNumber = 2;
+  inline ::google::protobuf::int32 receiving_major_fragment_id() const;
+  inline void set_receiving_major_fragment_id(::google::protobuf::int32 value);
+
+  // repeated int32 receiving_minor_fragment_id = 3;
+  inline int receiving_minor_fragment_id_size() const;
+  inline void clear_receiving_minor_fragment_id();
+  static const int kReceivingMinorFragmentIdFieldNumber = 3;
+  inline ::google::protobuf::int32 receiving_minor_fragment_id(int index) const;
+  inline void set_receiving_minor_fragment_id(int index, ::google::protobuf::int32 value);
+  inline void add_receiving_minor_fragment_id(::google::protobuf::int32 value);
+  inline const ::google::protobuf::RepeatedField< ::google::protobuf::int32 >&
+      receiving_minor_fragment_id() const;
+  inline ::google::protobuf::RepeatedField< ::google::protobuf::int32 >*
+      mutable_receiving_minor_fragment_id();
+
+  // optional int32 sending_major_fragment_id = 4;
   inline bool has_sending_major_fragment_id() const;
   inline void clear_sending_major_fragment_id();
-  static const int kSendingMajorFragmentIdFieldNumber = 2;
+  static const int kSendingMajorFragmentIdFieldNumber = 4;
   inline ::google::protobuf::int32 sending_major_fragment_id() const;
   inline void set_sending_major_fragment_id(::google::protobuf::int32 value);
 
-  // optional int32 sending_minor_fragment_id = 3;
+  // optional int32 sending_minor_fragment_id = 5;
   inline bool has_sending_minor_fragment_id() const;
   inline void clear_sending_minor_fragment_id();
-  static const int kSendingMinorFragmentIdFieldNumber = 3;
+  static const int kSendingMinorFragmentIdFieldNumber = 5;
   inline ::google::protobuf::int32 sending_minor_fragment_id() const;
   inline void set_sending_minor_fragment_id(::google::protobuf::int32 value);
 
-  // optional .exec.shared.RecordBatchDef def = 4;
+  // optional .exec.shared.RecordBatchDef def = 6;
   inline bool has_def() const;
   inline void clear_def();
-  static const int kDefFieldNumber = 4;
+  static const int kDefFieldNumber = 6;
   inline const ::exec::shared::RecordBatchDef& def() const;
   inline ::exec::shared::RecordBatchDef* mutable_def();
   inline ::exec::shared::RecordBatchDef* release_def();
   inline void set_allocated_def(::exec::shared::RecordBatchDef* def);
 
-  // optional bool isLastBatch = 5;
+  // optional bool isLastBatch = 7;
   inline bool has_islastbatch() const;
   inline void clear_islastbatch();
-  static const int kIsLastBatchFieldNumber = 5;
+  static const int kIsLastBatchFieldNumber = 7;
   inline bool islastbatch() const;
   inline void set_islastbatch(bool value);
 
-  // optional bool isOutOfMemory = 6 [default = false];
+  // optional bool isOutOfMemory = 8 [default = false];
   inline bool has_isoutofmemory() const;
   inline void clear_isoutofmemory();
-  static const int kIsOutOfMemoryFieldNumber = 6;
+  static const int kIsOutOfMemoryFieldNumber = 8;
   inline bool isoutofmemory() const;
   inline void set_isoutofmemory(bool value);
 
   // @@protoc_insertion_point(class_scope:exec.bit.data.FragmentRecordBatch)
  private:
-  inline void set_has_handle();
-  inline void clear_has_handle();
+  inline void set_has_query_id();
+  inline void clear_has_query_id();
+  inline void set_has_receiving_major_fragment_id();
+  inline void clear_has_receiving_major_fragment_id();
   inline void set_has_sending_major_fragment_id();
   inline void clear_has_sending_major_fragment_id();
   inline void set_has_sending_minor_fragment_id();
@@ -369,15 +378,17 @@ class FragmentRecordBatch : public ::google::protobuf::Message {
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
-  ::exec::bit::FragmentHandle* handle_;
+  ::exec::shared::QueryId* query_id_;
+  ::google::protobuf::RepeatedField< ::google::protobuf::int32 > receiving_minor_fragment_id_;
+  ::google::protobuf::int32 receiving_major_fragment_id_;
   ::google::protobuf::int32 sending_major_fragment_id_;
-  ::google::protobuf::int32 sending_minor_fragment_id_;
   ::exec::shared::RecordBatchDef* def_;
+  ::google::protobuf::int32 sending_minor_fragment_id_;
   bool islastbatch_;
   bool isoutofmemory_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(6 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(8 + 31) / 32];
 
   friend void  protobuf_AddDesc_BitData_2eproto();
   friend void protobuf_AssignDesc_BitData_2eproto();
@@ -438,44 +449,6 @@ inline void BitClientHandshake::set_channel(::exec::shared::RpcChannel value) {
   channel_ = value;
 }
 
-// optional .exec.bit.FragmentHandle handle = 3;
-inline bool BitClientHandshake::has_handle() const {
-  return (_has_bits_[0] & 0x00000004u) != 0;
-}
-inline void BitClientHandshake::set_has_handle() {
-  _has_bits_[0] |= 0x00000004u;
-}
-inline void BitClientHandshake::clear_has_handle() {
-  _has_bits_[0] &= ~0x00000004u;
-}
-inline void BitClientHandshake::clear_handle() {
-  if (handle_ != NULL) handle_->::exec::bit::FragmentHandle::Clear();
-  clear_has_handle();
-}
-inline const ::exec::bit::FragmentHandle& BitClientHandshake::handle() const {
-  return handle_ != NULL ? *handle_ : *default_instance_->handle_;
-}
-inline ::exec::bit::FragmentHandle* BitClientHandshake::mutable_handle() {
-  set_has_handle();
-  if (handle_ == NULL) handle_ = new ::exec::bit::FragmentHandle;
-  return handle_;
-}
-inline ::exec::bit::FragmentHandle* BitClientHandshake::release_handle() {
-  clear_has_handle();
-  ::exec::bit::FragmentHandle* temp = handle_;
-  handle_ = NULL;
-  return temp;
-}
-inline void BitClientHandshake::set_allocated_handle(::exec::bit::FragmentHandle* handle) {
-  delete handle_;
-  handle_ = handle;
-  if (handle) {
-    set_has_handle();
-  } else {
-    clear_has_handle();
-  }
-}
-
 // -------------------------------------------------------------------
 
 // BitServerHandshake
@@ -506,54 +479,101 @@ inline void BitServerHandshake::set_rpc_version(::google::protobuf::int32 value)
 
 // FragmentRecordBatch
 
-// optional .exec.bit.FragmentHandle handle = 1;
-inline bool FragmentRecordBatch::has_handle() const {
+// optional .exec.shared.QueryId query_id = 1;
+inline bool FragmentRecordBatch::has_query_id() const {
   return (_has_bits_[0] & 0x00000001u) != 0;
 }
-inline void FragmentRecordBatch::set_has_handle() {
+inline void FragmentRecordBatch::set_has_query_id() {
   _has_bits_[0] |= 0x00000001u;
 }
-inline void FragmentRecordBatch::clear_has_handle() {
+inline void FragmentRecordBatch::clear_has_query_id() {
   _has_bits_[0] &= ~0x00000001u;
 }
-inline void FragmentRecordBatch::clear_handle() {
-  if (handle_ != NULL) handle_->::exec::bit::FragmentHandle::Clear();
-  clear_has_handle();
+inline void FragmentRecordBatch::clear_query_id() {
+  if (query_id_ != NULL) query_id_->::exec::shared::QueryId::Clear();
+  clear_has_query_id();
 }
-inline const ::exec::bit::FragmentHandle& FragmentRecordBatch::handle() const {
-  return handle_ != NULL ? *handle_ : *default_instance_->handle_;
+inline const ::exec::shared::QueryId& FragmentRecordBatch::query_id() const {
+  return query_id_ != NULL ? *query_id_ : *default_instance_->query_id_;
 }
-inline ::exec::bit::FragmentHandle* FragmentRecordBatch::mutable_handle() {
-  set_has_handle();
-  if (handle_ == NULL) handle_ = new ::exec::bit::FragmentHandle;
-  return handle_;
+inline ::exec::shared::QueryId* FragmentRecordBatch::mutable_query_id() {
+  set_has_query_id();
+  if (query_id_ == NULL) query_id_ = new ::exec::shared::QueryId;
+  return query_id_;
 }
-inline ::exec::bit::FragmentHandle* FragmentRecordBatch::release_handle() {
-  clear_has_handle();
-  ::exec::bit::FragmentHandle* temp = handle_;
-  handle_ = NULL;
+inline ::exec::shared::QueryId* FragmentRecordBatch::release_query_id() {
+  clear_has_query_id();
+  ::exec::shared::QueryId* temp = query_id_;
+  query_id_ = NULL;
   return temp;
 }
-inline void FragmentRecordBatch::set_allocated_handle(::exec::bit::FragmentHandle* handle) {
-  delete handle_;
-  handle_ = handle;
-  if (handle) {
-    set_has_handle();
+inline void FragmentRecordBatch::set_allocated_query_id(::exec::shared::QueryId* query_id) {
+  delete query_id_;
+  query_id_ = query_id;
+  if (query_id) {
+    set_has_query_id();
   } else {
-    clear_has_handle();
+    clear_has_query_id();
   }
 }
 
-// optional int32 sending_major_fragment_id = 2;
-inline bool FragmentRecordBatch::has_sending_major_fragment_id() const {
+// optional int32 receiving_major_fragment_id = 2;
+inline bool FragmentRecordBatch::has_receiving_major_fragment_id() const {
   return (_has_bits_[0] & 0x00000002u) != 0;
 }
-inline void FragmentRecordBatch::set_has_sending_major_fragment_id() {
+inline void FragmentRecordBatch::set_has_receiving_major_fragment_id() {
   _has_bits_[0] |= 0x00000002u;
 }
-inline void FragmentRecordBatch::clear_has_sending_major_fragment_id() {
+inline void FragmentRecordBatch::clear_has_receiving_major_fragment_id() {
   _has_bits_[0] &= ~0x00000002u;
 }
+inline void FragmentRecordBatch::clear_receiving_major_fragment_id() {
+  receiving_major_fragment_id_ = 0;
+  clear_has_receiving_major_fragment_id();
+}
+inline ::google::protobuf::int32 FragmentRecordBatch::receiving_major_fragment_id() const {
+  return receiving_major_fragment_id_;
+}
+inline void FragmentRecordBatch::set_receiving_major_fragment_id(::google::protobuf::int32 value) {
+  set_has_receiving_major_fragment_id();
+  receiving_major_fragment_id_ = value;
+}
+
+// repeated int32 receiving_minor_fragment_id = 3;
+inline int FragmentRecordBatch::receiving_minor_fragment_id_size() const {
+  return receiving_minor_fragment_id_.size();
+}
+inline void FragmentRecordBatch::clear_receiving_minor_fragment_id() {
+  receiving_minor_fragment_id_.Clear();
+}
+inline ::google::protobuf::int32 FragmentRecordBatch::receiving_minor_fragment_id(int index) const {
+  return receiving_minor_fragment_id_.Get(index);
+}
+inline void FragmentRecordBatch::set_receiving_minor_fragment_id(int index, ::google::protobuf::int32 value) {
+  receiving_minor_fragment_id_.Set(index, value);
+}
+inline void FragmentRecordBatch::add_receiving_minor_fragment_id(::google::protobuf::int32 value) {
+  receiving_minor_fragment_id_.Add(value);
+}
+inline const ::google::protobuf::RepeatedField< ::google::protobuf::int32 >&
+FragmentRecordBatch::receiving_minor_fragment_id() const {
+  return receiving_minor_fragment_id_;
+}
+inline ::google::protobuf::RepeatedField< ::google::protobuf::int32 >*
+FragmentRecordBatch::mutable_receiving_minor_fragment_id() {
+  return &receiving_minor_fragment_id_;
+}
+
+// optional int32 sending_major_fragment_id = 4;
+inline bool FragmentRecordBatch::has_sending_major_fragment_id() const {
+  return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void FragmentRecordBatch::set_has_sending_major_fragment_id() {
+  _has_bits_[0] |= 0x00000008u;
+}
+inline void FragmentRecordBatch::clear_has_sending_major_fragment_id() {
+  _has_bits_[0] &= ~0x00000008u;
+}
 inline void FragmentRecordBatch::clear_sending_major_fragment_id() {
   sending_major_fragment_id_ = 0;
   clear_has_sending_major_fragment_id();
@@ -566,15 +586,15 @@ inline void FragmentRecordBatch::set_sending_major_fragment_id(::google::protobu
   sending_major_fragment_id_ = value;
 }
 
-// optional int32 sending_minor_fragment_id = 3;
+// optional int32 sending_minor_fragment_id = 5;
 inline bool FragmentRecordBatch::has_sending_minor_fragment_id() const {
-  return (_has_bits_[0] & 0x00000004u) != 0;
+  return (_has_bits_[0] & 0x00000010u) != 0;
 }
 inline void FragmentRecordBatch::set_has_sending_minor_fragment_id() {
-  _has_bits_[0] |= 0x00000004u;
+  _has_bits_[0] |= 0x00000010u;
 }
 inline void FragmentRecordBatch::clear_has_sending_minor_fragment_id() {
-  _has_bits_[0] &= ~0x00000004u;
+  _has_bits_[0] &= ~0x00000010u;
 }
 inline void FragmentRecordBatch::clear_sending_minor_fragment_id() {
   sending_minor_fragment_id_ = 0;
@@ -588,15 +608,15 @@ inline void FragmentRecordBatch::set_sending_minor_fragment_id(::google::protobu
   sending_minor_fragment_id_ = value;
 }
 
-// optional .exec.shared.RecordBatchDef def = 4;
+// optional .exec.shared.RecordBatchDef def = 6;
 inline bool FragmentRecordBatch::has_def() const {
-  return (_has_bits_[0] & 0x00000008u) != 0;
+  return (_has_bits_[0] & 0x00000020u) != 0;
 }
 inline void FragmentRecordBatch::set_has_def() {
-  _has_bits_[0] |= 0x00000008u;
+  _has_bits_[0] |= 0x00000020u;
 }
 inline void FragmentRecordBatch::clear_has_def() {
-  _has_bits_[0] &= ~0x00000008u;
+  _has_bits_[0] &= ~0x00000020u;
 }
 inline void FragmentRecordBatch::clear_def() {
   if (def_ != NULL) def_->::exec::shared::RecordBatchDef::Clear();
@@ -626,15 +646,15 @@ inline void FragmentRecordBatch::set_allocated_def(::exec::shared::RecordBatchDe
   }
 }
 
-// optional bool isLastBatch = 5;
+// optional bool isLastBatch = 7;
 inline bool FragmentRecordBatch::has_islastbatch() const {
-  return (_has_bits_[0] & 0x00000010u) != 0;
+  return (_has_bits_[0] & 0x00000040u) != 0;
 }
 inline void FragmentRecordBatch::set_has_islastbatch() {
-  _has_bits_[0] |= 0x00000010u;
+  _has_bits_[0] |= 0x00000040u;
 }
 inline void FragmentRecordBatch::clear_has_islastbatch() {
-  _has_bits_[0] &= ~0x00000010u;
+  _has_bits_[0] &= ~0x00000040u;
 }
 inline void FragmentRecordBatch::clear_islastbatch() {
   islastbatch_ = false;
@@ -648,15 +668,15 @@ inline void FragmentRecordBatch::set_islastbatch(bool value) {
   islastbatch_ = value;
 }
 
-// optional bool isOutOfMemory = 6 [default = false];
+// optional bool isOutOfMemory = 8 [default = false];
 inline bool FragmentRecordBatch::has_isoutofmemory() const {
-  return (_has_bits_[0] & 0x00000020u) != 0;
+  return (_has_bits_[0] & 0x00000080u) != 0;
 }
 inline void FragmentRecordBatch::set_has_isoutofmemory() {
-  _has_bits_[0] |= 0x00000020u;
+  _has_bits_[0] |= 0x00000080u;
 }
 inline void FragmentRecordBatch::clear_has_isoutofmemory() {
-  _has_bits_[0] &= ~0x00000020u;
+  _has_bits_[0] &= ~0x00000080u;
 }
 inline void FragmentRecordBatch::clear_isoutofmemory() {
   isoutofmemory_ = false;

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/User.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/User.pb.cc b/contrib/native/client/src/protobuf/User.pb.cc
index d85c81b..360becb 100644
--- a/contrib/native/client/src/protobuf/User.pb.cc
+++ b/contrib/native/client/src/protobuf/User.pb.cc
@@ -220,14 +220,14 @@ void protobuf_AddDesc_User_2eproto() {
     "lts_mode\030\001 \001(\0162\033.exec.user.QueryResultsM"
     "ode\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryTyp"
     "e\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022\023\n"
-    "\013rpc_version\030\002 \001(\005*\270\001\n\007RpcType\022\r\n\tHANDSH"
+    "\013rpc_version\030\002 \001(\005*\310\001\n\007RpcType\022\r\n\tHANDSH"
     "AKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY"
     "\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020"
-    "\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n"
-    "\022REQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_L"
-    "IST\020\t*#\n\020QueryResultsMode\022\017\n\013STREAM_FULL"
-    "\020\001B+\n\033org.apache.drill.exec.protoB\nUserP"
-    "rotosH\001", 927);
+    "\005\022\016\n\nQUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n\022R"
+    "EQ_META_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_LIS"
+    "T\020\t\022\020\n\014QUERY_RESULT\020\n*#\n\020QueryResultsMod"
+    "e\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.e"
+    "xec.protoB\nUserProtosH\001", 943);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "User.proto", &protobuf_RegisterTypes);
   Property::default_instance_ = new Property();
@@ -267,6 +267,7 @@ bool RpcType_IsValid(int value) {
     case 7:
     case 8:
     case 9:
+    case 10:
       return true;
     default:
       return false;

http://git-wip-us.apache.org/repos/asf/drill/blob/4f213570/contrib/native/client/src/protobuf/User.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/User.pb.h b/contrib/native/client/src/protobuf/User.pb.h
index eca199d..69daf50 100644
--- a/contrib/native/client/src/protobuf/User.pb.h
+++ b/contrib/native/client/src/protobuf/User.pb.h
@@ -51,14 +51,15 @@ enum RpcType {
   RUN_QUERY = 3,
   CANCEL_QUERY = 4,
   REQUEST_RESULTS = 5,
-  QUERY_RESULT = 6,
+  QUERY_DATA = 6,
   QUERY_HANDLE = 7,
   REQ_META_FUNCTIONS = 8,
-  RESP_FUNCTION_LIST = 9
+  RESP_FUNCTION_LIST = 9,
+  QUERY_RESULT = 10
 };
 bool RpcType_IsValid(int value);
 const RpcType RpcType_MIN = HANDSHAKE;
-const RpcType RpcType_MAX = RESP_FUNCTION_LIST;
+const RpcType RpcType_MAX = QUERY_RESULT;
 const int RpcType_ARRAYSIZE = RpcType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* RpcType_descriptor();


[2/9] drill git commit: DRILL-2498: Separate QueryResult into two messages QueryResult and QueryData

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/1d9d82b0/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 9327c42..9a9d196 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -9798,132 +9798,38 @@ public final class UserBitShared {
      */
     org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getQueryIdOrBuilder();
 
-    // optional bool is_last_chunk = 3;
+    // repeated .exec.shared.DrillPBError error = 3;
     /**
-     * <code>optional bool is_last_chunk = 3;</code>
-     */
-    boolean hasIsLastChunk();
-    /**
-     * <code>optional bool is_last_chunk = 3;</code>
-     */
-    boolean getIsLastChunk();
-
-    // optional int32 row_count = 4;
-    /**
-     * <code>optional int32 row_count = 4;</code>
-     */
-    boolean hasRowCount();
-    /**
-     * <code>optional int32 row_count = 4;</code>
-     */
-    int getRowCount();
-
-    // optional int64 records_scan = 5;
-    /**
-     * <code>optional int64 records_scan = 5;</code>
-     */
-    boolean hasRecordsScan();
-    /**
-     * <code>optional int64 records_scan = 5;</code>
-     */
-    long getRecordsScan();
-
-    // optional int64 records_error = 6;
-    /**
-     * <code>optional int64 records_error = 6;</code>
-     */
-    boolean hasRecordsError();
-    /**
-     * <code>optional int64 records_error = 6;</code>
-     */
-    long getRecordsError();
-
-    // optional int64 submission_time = 7;
-    /**
-     * <code>optional int64 submission_time = 7;</code>
-     */
-    boolean hasSubmissionTime();
-    /**
-     * <code>optional int64 submission_time = 7;</code>
-     */
-    long getSubmissionTime();
-
-    // repeated .exec.shared.NodeStatus node_status = 8;
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus> 
-        getNodeStatusList();
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.NodeStatus getNodeStatus(int index);
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    int getNodeStatusCount();
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder> 
-        getNodeStatusOrBuilderList();
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder getNodeStatusOrBuilder(
-        int index);
-
-    // repeated .exec.shared.DrillPBError error = 9;
-    /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> 
         getErrorList();
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.DrillPBError getError(int index);
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     int getErrorCount();
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
         getErrorOrBuilderList();
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder(
         int index);
-
-    // optional .exec.shared.RecordBatchDef def = 10;
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    boolean hasDef();
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef();
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder();
-
-    // optional bool schema_changed = 11;
-    /**
-     * <code>optional bool schema_changed = 11;</code>
-     */
-    boolean hasSchemaChanged();
-    /**
-     * <code>optional bool schema_changed = 11;</code>
-     */
-    boolean getSchemaChanged();
   }
   /**
    * Protobuf type {@code exec.shared.QueryResult}
+   *
+   * <pre>
+   *
+   * Used by the server to report informations about the query state to the client
+   * </pre>
    */
   public static final class QueryResult extends
       com.google.protobuf.GeneratedMessage
@@ -9997,65 +9903,14 @@ public final class UserBitShared {
               bitField0_ |= 0x00000002;
               break;
             }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              isLastChunk_ = input.readBool();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              rowCount_ = input.readInt32();
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000010;
-              recordsScan_ = input.readInt64();
-              break;
-            }
-            case 48: {
-              bitField0_ |= 0x00000020;
-              recordsError_ = input.readInt64();
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000040;
-              submissionTime_ = input.readInt64();
-              break;
-            }
-            case 66: {
-              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-                nodeStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.NodeStatus>();
-                mutable_bitField0_ |= 0x00000080;
-              }
-              nodeStatus_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.NodeStatus.PARSER, extensionRegistry));
-              break;
-            }
-            case 74: {
-              if (!((mutable_bitField0_ & 0x00000100) == 0x00000100)) {
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
                 error_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.DrillPBError>();
-                mutable_bitField0_ |= 0x00000100;
+                mutable_bitField0_ |= 0x00000004;
               }
               error_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.DrillPBError.PARSER, extensionRegistry));
               break;
             }
-            case 82: {
-              org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000080) == 0x00000080)) {
-                subBuilder = def_.toBuilder();
-              }
-              def_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(def_);
-                def_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000080;
-              break;
-            }
-            case 88: {
-              bitField0_ |= 0x00000100;
-              schemaChanged_ = input.readBool();
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -10064,10 +9919,7 @@ public final class UserBitShared {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
-          nodeStatus_ = java.util.Collections.unmodifiableList(nodeStatus_);
-        }
-        if (((mutable_bitField0_ & 0x00000100) == 0x00000100)) {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
           error_ = java.util.Collections.unmodifiableList(error_);
         }
         this.unknownFields = unknownFields.build();
@@ -10282,208 +10134,46 @@ public final class UserBitShared {
       return queryId_;
     }
 
-    // optional bool is_last_chunk = 3;
-    public static final int IS_LAST_CHUNK_FIELD_NUMBER = 3;
-    private boolean isLastChunk_;
-    /**
-     * <code>optional bool is_last_chunk = 3;</code>
-     */
-    public boolean hasIsLastChunk() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional bool is_last_chunk = 3;</code>
-     */
-    public boolean getIsLastChunk() {
-      return isLastChunk_;
-    }
-
-    // optional int32 row_count = 4;
-    public static final int ROW_COUNT_FIELD_NUMBER = 4;
-    private int rowCount_;
-    /**
-     * <code>optional int32 row_count = 4;</code>
-     */
-    public boolean hasRowCount() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional int32 row_count = 4;</code>
-     */
-    public int getRowCount() {
-      return rowCount_;
-    }
-
-    // optional int64 records_scan = 5;
-    public static final int RECORDS_SCAN_FIELD_NUMBER = 5;
-    private long recordsScan_;
-    /**
-     * <code>optional int64 records_scan = 5;</code>
-     */
-    public boolean hasRecordsScan() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional int64 records_scan = 5;</code>
-     */
-    public long getRecordsScan() {
-      return recordsScan_;
-    }
-
-    // optional int64 records_error = 6;
-    public static final int RECORDS_ERROR_FIELD_NUMBER = 6;
-    private long recordsError_;
-    /**
-     * <code>optional int64 records_error = 6;</code>
-     */
-    public boolean hasRecordsError() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
-    }
-    /**
-     * <code>optional int64 records_error = 6;</code>
-     */
-    public long getRecordsError() {
-      return recordsError_;
-    }
-
-    // optional int64 submission_time = 7;
-    public static final int SUBMISSION_TIME_FIELD_NUMBER = 7;
-    private long submissionTime_;
-    /**
-     * <code>optional int64 submission_time = 7;</code>
-     */
-    public boolean hasSubmissionTime() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional int64 submission_time = 7;</code>
-     */
-    public long getSubmissionTime() {
-      return submissionTime_;
-    }
-
-    // repeated .exec.shared.NodeStatus node_status = 8;
-    public static final int NODE_STATUS_FIELD_NUMBER = 8;
-    private java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus> nodeStatus_;
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    public java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus> getNodeStatusList() {
-      return nodeStatus_;
-    }
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder> 
-        getNodeStatusOrBuilderList() {
-      return nodeStatus_;
-    }
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    public int getNodeStatusCount() {
-      return nodeStatus_.size();
-    }
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.NodeStatus getNodeStatus(int index) {
-      return nodeStatus_.get(index);
-    }
-    /**
-     * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder getNodeStatusOrBuilder(
-        int index) {
-      return nodeStatus_.get(index);
-    }
-
-    // repeated .exec.shared.DrillPBError error = 9;
-    public static final int ERROR_FIELD_NUMBER = 9;
+    // repeated .exec.shared.DrillPBError error = 3;
+    public static final int ERROR_FIELD_NUMBER = 3;
     private java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> error_;
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     public java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> getErrorList() {
       return error_;
     }
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
         getErrorOrBuilderList() {
       return error_;
     }
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     public int getErrorCount() {
       return error_.size();
     }
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError(int index) {
       return error_.get(index);
     }
     /**
-     * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+     * <code>repeated .exec.shared.DrillPBError error = 3;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder(
         int index) {
       return error_.get(index);
     }
 
-    // optional .exec.shared.RecordBatchDef def = 10;
-    public static final int DEF_FIELD_NUMBER = 10;
-    private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_;
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    public boolean hasDef() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
-      return def_;
-    }
-    /**
-     * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
-      return def_;
-    }
-
-    // optional bool schema_changed = 11;
-    public static final int SCHEMA_CHANGED_FIELD_NUMBER = 11;
-    private boolean schemaChanged_;
-    /**
-     * <code>optional bool schema_changed = 11;</code>
-     */
-    public boolean hasSchemaChanged() {
-      return ((bitField0_ & 0x00000100) == 0x00000100);
-    }
-    /**
-     * <code>optional bool schema_changed = 11;</code>
-     */
-    public boolean getSchemaChanged() {
-      return schemaChanged_;
-    }
-
     private void initFields() {
       queryState_ = org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState.PENDING;
       queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
-      isLastChunk_ = false;
-      rowCount_ = 0;
-      recordsScan_ = 0L;
-      recordsError_ = 0L;
-      submissionTime_ = 0L;
-      nodeStatus_ = java.util.Collections.emptyList();
       error_ = java.util.Collections.emptyList();
-      def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
-      schemaChanged_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -10503,32 +10193,8 @@ public final class UserBitShared {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeMessage(2, queryId_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBool(3, isLastChunk_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt32(4, rowCount_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeInt64(5, recordsScan_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeInt64(6, recordsError_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeInt64(7, submissionTime_);
-      }
-      for (int i = 0; i < nodeStatus_.size(); i++) {
-        output.writeMessage(8, nodeStatus_.get(i));
-      }
       for (int i = 0; i < error_.size(); i++) {
-        output.writeMessage(9, error_.get(i));
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeMessage(10, def_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        output.writeBool(11, schemaChanged_);
+        output.writeMessage(3, error_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -10547,41 +10213,9 @@ public final class UserBitShared {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(2, queryId_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(3, isLastChunk_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(4, rowCount_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(5, recordsScan_);
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(6, recordsError_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(7, submissionTime_);
-      }
-      for (int i = 0; i < nodeStatus_.size(); i++) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(8, nodeStatus_.get(i));
-      }
       for (int i = 0; i < error_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(9, error_.get(i));
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(10, def_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(11, schemaChanged_);
+          .computeMessageSize(3, error_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -10663,6 +10297,11 @@ public final class UserBitShared {
     }
     /**
      * Protobuf type {@code exec.shared.QueryResult}
+     *
+     * <pre>
+     *
+     * Used by the server to report informations about the query state to the client
+     * </pre>
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
@@ -10692,9 +10331,7 @@ public final class UserBitShared {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getQueryIdFieldBuilder();
-          getNodeStatusFieldBuilder();
           getErrorFieldBuilder();
-          getDefFieldBuilder();
         }
       }
       private static Builder create() {
@@ -10711,36 +10348,12 @@ public final class UserBitShared {
           queryIdBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
-        isLastChunk_ = false;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        rowCount_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        recordsScan_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        recordsError_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000020);
-        submissionTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        if (nodeStatusBuilder_ == null) {
-          nodeStatus_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000080);
-        } else {
-          nodeStatusBuilder_.clear();
-        }
         if (errorBuilder_ == null) {
           error_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000100);
+          bitField0_ = (bitField0_ & ~0x00000004);
         } else {
           errorBuilder_.clear();
         }
-        if (defBuilder_ == null) {
-          def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
-        } else {
-          defBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000200);
-        schemaChanged_ = false;
-        bitField0_ = (bitField0_ & ~0x00000400);
         return this;
       }
 
@@ -10781,56 +10394,15 @@ public final class UserBitShared {
         } else {
           result.queryId_ = queryIdBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
+        if (errorBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            error_ = java.util.Collections.unmodifiableList(error_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.error_ = error_;
+        } else {
+          result.error_ = errorBuilder_.build();
         }
-        result.isLastChunk_ = isLastChunk_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.rowCount_ = rowCount_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.recordsScan_ = recordsScan_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.recordsError_ = recordsError_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.submissionTime_ = submissionTime_;
-        if (nodeStatusBuilder_ == null) {
-          if (((bitField0_ & 0x00000080) == 0x00000080)) {
-            nodeStatus_ = java.util.Collections.unmodifiableList(nodeStatus_);
-            bitField0_ = (bitField0_ & ~0x00000080);
-          }
-          result.nodeStatus_ = nodeStatus_;
-        } else {
-          result.nodeStatus_ = nodeStatusBuilder_.build();
-        }
-        if (errorBuilder_ == null) {
-          if (((bitField0_ & 0x00000100) == 0x00000100)) {
-            error_ = java.util.Collections.unmodifiableList(error_);
-            bitField0_ = (bitField0_ & ~0x00000100);
-          }
-          result.error_ = error_;
-        } else {
-          result.error_ = errorBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        if (defBuilder_ == null) {
-          result.def_ = def_;
-        } else {
-          result.def_ = defBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
-          to_bitField0_ |= 0x00000100;
-        }
-        result.schemaChanged_ = schemaChanged_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -10853,52 +10425,11 @@ public final class UserBitShared {
         if (other.hasQueryId()) {
           mergeQueryId(other.getQueryId());
         }
-        if (other.hasIsLastChunk()) {
-          setIsLastChunk(other.getIsLastChunk());
-        }
-        if (other.hasRowCount()) {
-          setRowCount(other.getRowCount());
-        }
-        if (other.hasRecordsScan()) {
-          setRecordsScan(other.getRecordsScan());
-        }
-        if (other.hasRecordsError()) {
-          setRecordsError(other.getRecordsError());
-        }
-        if (other.hasSubmissionTime()) {
-          setSubmissionTime(other.getSubmissionTime());
-        }
-        if (nodeStatusBuilder_ == null) {
-          if (!other.nodeStatus_.isEmpty()) {
-            if (nodeStatus_.isEmpty()) {
-              nodeStatus_ = other.nodeStatus_;
-              bitField0_ = (bitField0_ & ~0x00000080);
-            } else {
-              ensureNodeStatusIsMutable();
-              nodeStatus_.addAll(other.nodeStatus_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.nodeStatus_.isEmpty()) {
-            if (nodeStatusBuilder_.isEmpty()) {
-              nodeStatusBuilder_.dispose();
-              nodeStatusBuilder_ = null;
-              nodeStatus_ = other.nodeStatus_;
-              bitField0_ = (bitField0_ & ~0x00000080);
-              nodeStatusBuilder_ = 
-                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
-                   getNodeStatusFieldBuilder() : null;
-            } else {
-              nodeStatusBuilder_.addAllMessages(other.nodeStatus_);
-            }
-          }
-        }
         if (errorBuilder_ == null) {
           if (!other.error_.isEmpty()) {
             if (error_.isEmpty()) {
               error_ = other.error_;
-              bitField0_ = (bitField0_ & ~0x00000100);
+              bitField0_ = (bitField0_ & ~0x00000004);
             } else {
               ensureErrorIsMutable();
               error_.addAll(other.error_);
@@ -10911,7 +10442,7 @@ public final class UserBitShared {
               errorBuilder_.dispose();
               errorBuilder_ = null;
               error_ = other.error_;
-              bitField0_ = (bitField0_ & ~0x00000100);
+              bitField0_ = (bitField0_ & ~0x00000004);
               errorBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getErrorFieldBuilder() : null;
@@ -10920,12 +10451,6 @@ public final class UserBitShared {
             }
           }
         }
-        if (other.hasDef()) {
-          mergeDef(other.getDef());
-        }
-        if (other.hasSchemaChanged()) {
-          setSchemaChanged(other.getSchemaChanged());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -11106,663 +10631,931 @@ public final class UserBitShared {
         return queryIdBuilder_;
       }
 
-      // optional bool is_last_chunk = 3;
-      private boolean isLastChunk_ ;
-      /**
-       * <code>optional bool is_last_chunk = 3;</code>
-       */
-      public boolean hasIsLastChunk() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional bool is_last_chunk = 3;</code>
-       */
-      public boolean getIsLastChunk() {
-        return isLastChunk_;
-      }
-      /**
-       * <code>optional bool is_last_chunk = 3;</code>
-       */
-      public Builder setIsLastChunk(boolean value) {
-        bitField0_ |= 0x00000004;
-        isLastChunk_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool is_last_chunk = 3;</code>
-       */
-      public Builder clearIsLastChunk() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        isLastChunk_ = false;
-        onChanged();
-        return this;
-      }
-
-      // optional int32 row_count = 4;
-      private int rowCount_ ;
-      /**
-       * <code>optional int32 row_count = 4;</code>
-       */
-      public boolean hasRowCount() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int32 row_count = 4;</code>
-       */
-      public int getRowCount() {
-        return rowCount_;
-      }
-      /**
-       * <code>optional int32 row_count = 4;</code>
-       */
-      public Builder setRowCount(int value) {
-        bitField0_ |= 0x00000008;
-        rowCount_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 row_count = 4;</code>
-       */
-      public Builder clearRowCount() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        rowCount_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 records_scan = 5;
-      private long recordsScan_ ;
-      /**
-       * <code>optional int64 records_scan = 5;</code>
-       */
-      public boolean hasRecordsScan() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional int64 records_scan = 5;</code>
-       */
-      public long getRecordsScan() {
-        return recordsScan_;
-      }
-      /**
-       * <code>optional int64 records_scan = 5;</code>
-       */
-      public Builder setRecordsScan(long value) {
-        bitField0_ |= 0x00000010;
-        recordsScan_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 records_scan = 5;</code>
-       */
-      public Builder clearRecordsScan() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        recordsScan_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 records_error = 6;
-      private long recordsError_ ;
-      /**
-       * <code>optional int64 records_error = 6;</code>
-       */
-      public boolean hasRecordsError() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional int64 records_error = 6;</code>
-       */
-      public long getRecordsError() {
-        return recordsError_;
-      }
-      /**
-       * <code>optional int64 records_error = 6;</code>
-       */
-      public Builder setRecordsError(long value) {
-        bitField0_ |= 0x00000020;
-        recordsError_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 records_error = 6;</code>
-       */
-      public Builder clearRecordsError() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        recordsError_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 submission_time = 7;
-      private long submissionTime_ ;
-      /**
-       * <code>optional int64 submission_time = 7;</code>
-       */
-      public boolean hasSubmissionTime() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
-      }
-      /**
-       * <code>optional int64 submission_time = 7;</code>
-       */
-      public long getSubmissionTime() {
-        return submissionTime_;
-      }
-      /**
-       * <code>optional int64 submission_time = 7;</code>
-       */
-      public Builder setSubmissionTime(long value) {
-        bitField0_ |= 0x00000040;
-        submissionTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 submission_time = 7;</code>
-       */
-      public Builder clearSubmissionTime() {
-        bitField0_ = (bitField0_ & ~0x00000040);
-        submissionTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // repeated .exec.shared.NodeStatus node_status = 8;
-      private java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus> nodeStatus_ =
+      // repeated .exec.shared.DrillPBError error = 3;
+      private java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> error_ =
         java.util.Collections.emptyList();
-      private void ensureNodeStatusIsMutable() {
-        if (!((bitField0_ & 0x00000080) == 0x00000080)) {
-          nodeStatus_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.NodeStatus>(nodeStatus_);
-          bitField0_ |= 0x00000080;
+      private void ensureErrorIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          error_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.DrillPBError>(error_);
+          bitField0_ |= 0x00000004;
          }
       }
 
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.NodeStatus, org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder, org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder> nodeStatusBuilder_;
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
 
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus> getNodeStatusList() {
-        if (nodeStatusBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(nodeStatus_);
+      public java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> getErrorList() {
+        if (errorBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(error_);
         } else {
-          return nodeStatusBuilder_.getMessageList();
+          return errorBuilder_.getMessageList();
         }
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public int getNodeStatusCount() {
-        if (nodeStatusBuilder_ == null) {
-          return nodeStatus_.size();
+      public int getErrorCount() {
+        if (errorBuilder_ == null) {
+          return error_.size();
         } else {
-          return nodeStatusBuilder_.getCount();
+          return errorBuilder_.getCount();
         }
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.NodeStatus getNodeStatus(int index) {
-        if (nodeStatusBuilder_ == null) {
-          return nodeStatus_.get(index);
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError(int index) {
+        if (errorBuilder_ == null) {
+          return error_.get(index);
         } else {
-          return nodeStatusBuilder_.getMessage(index);
+          return errorBuilder_.getMessage(index);
         }
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder setNodeStatus(
-          int index, org.apache.drill.exec.proto.UserBitShared.NodeStatus value) {
-        if (nodeStatusBuilder_ == null) {
+      public Builder setError(
+          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+        if (errorBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureNodeStatusIsMutable();
-          nodeStatus_.set(index, value);
+          ensureErrorIsMutable();
+          error_.set(index, value);
           onChanged();
         } else {
-          nodeStatusBuilder_.setMessage(index, value);
+          errorBuilder_.setMessage(index, value);
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder setNodeStatus(
-          int index, org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder builderForValue) {
-        if (nodeStatusBuilder_ == null) {
-          ensureNodeStatusIsMutable();
-          nodeStatus_.set(index, builderForValue.build());
+      public Builder setError(
+          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
+        if (errorBuilder_ == null) {
+          ensureErrorIsMutable();
+          error_.set(index, builderForValue.build());
           onChanged();
         } else {
-          nodeStatusBuilder_.setMessage(index, builderForValue.build());
+          errorBuilder_.setMessage(index, builderForValue.build());
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder addNodeStatus(org.apache.drill.exec.proto.UserBitShared.NodeStatus value) {
-        if (nodeStatusBuilder_ == null) {
+      public Builder addError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+        if (errorBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureNodeStatusIsMutable();
-          nodeStatus_.add(value);
+          ensureErrorIsMutable();
+          error_.add(value);
           onChanged();
         } else {
-          nodeStatusBuilder_.addMessage(value);
+          errorBuilder_.addMessage(value);
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder addNodeStatus(
-          int index, org.apache.drill.exec.proto.UserBitShared.NodeStatus value) {
-        if (nodeStatusBuilder_ == null) {
+      public Builder addError(
+          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+        if (errorBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureNodeStatusIsMutable();
-          nodeStatus_.add(index, value);
+          ensureErrorIsMutable();
+          error_.add(index, value);
           onChanged();
         } else {
-          nodeStatusBuilder_.addMessage(index, value);
+          errorBuilder_.addMessage(index, value);
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder addNodeStatus(
-          org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder builderForValue) {
-        if (nodeStatusBuilder_ == null) {
-          ensureNodeStatusIsMutable();
-          nodeStatus_.add(builderForValue.build());
+      public Builder addError(
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
+        if (errorBuilder_ == null) {
+          ensureErrorIsMutable();
+          error_.add(builderForValue.build());
           onChanged();
         } else {
-          nodeStatusBuilder_.addMessage(builderForValue.build());
+          errorBuilder_.addMessage(builderForValue.build());
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder addNodeStatus(
-          int index, org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder builderForValue) {
-        if (nodeStatusBuilder_ == null) {
-          ensureNodeStatusIsMutable();
-          nodeStatus_.add(index, builderForValue.build());
+      public Builder addError(
+          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
+        if (errorBuilder_ == null) {
+          ensureErrorIsMutable();
+          error_.add(index, builderForValue.build());
           onChanged();
         } else {
-          nodeStatusBuilder_.addMessage(index, builderForValue.build());
+          errorBuilder_.addMessage(index, builderForValue.build());
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder addAllNodeStatus(
-          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.NodeStatus> values) {
-        if (nodeStatusBuilder_ == null) {
-          ensureNodeStatusIsMutable();
-          super.addAll(values, nodeStatus_);
+      public Builder addAllError(
+          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBError> values) {
+        if (errorBuilder_ == null) {
+          ensureErrorIsMutable();
+          super.addAll(values, error_);
           onChanged();
         } else {
-          nodeStatusBuilder_.addAllMessages(values);
+          errorBuilder_.addAllMessages(values);
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder clearNodeStatus() {
-        if (nodeStatusBuilder_ == null) {
-          nodeStatus_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000080);
+      public Builder clearError() {
+        if (errorBuilder_ == null) {
+          error_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
           onChanged();
         } else {
-          nodeStatusBuilder_.clear();
+          errorBuilder_.clear();
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public Builder removeNodeStatus(int index) {
-        if (nodeStatusBuilder_ == null) {
-          ensureNodeStatusIsMutable();
-          nodeStatus_.remove(index);
+      public Builder removeError(int index) {
+        if (errorBuilder_ == null) {
+          ensureErrorIsMutable();
+          error_.remove(index);
           onChanged();
         } else {
-          nodeStatusBuilder_.remove(index);
+          errorBuilder_.remove(index);
         }
         return this;
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder getNodeStatusBuilder(
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder(
           int index) {
-        return getNodeStatusFieldBuilder().getBuilder(index);
+        return getErrorFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder getNodeStatusOrBuilder(
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder(
           int index) {
-        if (nodeStatusBuilder_ == null) {
-          return nodeStatus_.get(index);  } else {
-          return nodeStatusBuilder_.getMessageOrBuilder(index);
+        if (errorBuilder_ == null) {
+          return error_.get(index);  } else {
+          return errorBuilder_.getMessageOrBuilder(index);
         }
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder> 
-           getNodeStatusOrBuilderList() {
-        if (nodeStatusBuilder_ != null) {
-          return nodeStatusBuilder_.getMessageOrBuilderList();
+      public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
+           getErrorOrBuilderList() {
+        if (errorBuilder_ != null) {
+          return errorBuilder_.getMessageOrBuilderList();
         } else {
-          return java.util.Collections.unmodifiableList(nodeStatus_);
+          return java.util.Collections.unmodifiableList(error_);
         }
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder addNodeStatusBuilder() {
-        return getNodeStatusFieldBuilder().addBuilder(
-            org.apache.drill.exec.proto.UserBitShared.NodeStatus.getDefaultInstance());
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder addErrorBuilder() {
+        return getErrorFieldBuilder().addBuilder(
+            org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder addNodeStatusBuilder(
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder addErrorBuilder(
           int index) {
-        return getNodeStatusFieldBuilder().addBuilder(
-            index, org.apache.drill.exec.proto.UserBitShared.NodeStatus.getDefaultInstance());
+        return getErrorFieldBuilder().addBuilder(
+            index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.NodeStatus node_status = 8;</code>
+       * <code>repeated .exec.shared.DrillPBError error = 3;</code>
        */
-      public java.util.List<org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder> 
-           getNodeStatusBuilderList() {
-        return getNodeStatusFieldBuilder().getBuilderList();
+      public java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder> 
+           getErrorBuilderList() {
+        return getErrorFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.NodeStatus, org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder, org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder> 
-          getNodeStatusFieldBuilder() {
-        if (nodeStatusBuilder_ == null) {
-          nodeStatusBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.drill.exec.proto.UserBitShared.NodeStatus, org.apache.drill.exec.proto.UserBitShared.NodeStatus.Builder, org.apache.drill.exec.proto.UserBitShared.NodeStatusOrBuilder>(
-                  nodeStatus_,
-                  ((bitField0_ & 0x00000080) == 0x00000080),
-                  getParentForChildren(),
-                  isClean());
-          nodeStatus_ = null;
-        }
-        return nodeStatusBuilder_;
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
+          getErrorFieldBuilder() {
+        if (errorBuilder_ == null) {
+          errorBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
+                  error_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          error_ = null;
+        }
+        return errorBuilder_;
       }
 
-      // repeated .exec.shared.DrillPBError error = 9;
-      private java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> error_ =
-        java.util.Collections.emptyList();
-      private void ensureErrorIsMutable() {
-        if (!((bitField0_ & 0x00000100) == 0x00000100)) {
-          error_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.DrillPBError>(error_);
-          bitField0_ |= 0x00000100;
-         }
+      // @@protoc_insertion_point(builder_scope:exec.shared.QueryResult)
+    }
+
+    static {
+      defaultInstance = new QueryResult(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:exec.shared.QueryResult)
+  }
+
+  public interface QueryDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .exec.shared.QueryId query_id = 1;
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    boolean hasQueryId();
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.QueryId getQueryId();
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getQueryIdOrBuilder();
+
+    // optional int32 row_count = 2;
+    /**
+     * <code>optional int32 row_count = 2;</code>
+     */
+    boolean hasRowCount();
+    /**
+     * <code>optional int32 row_count = 2;</code>
+     */
+    int getRowCount();
+
+    // optional .exec.shared.RecordBatchDef def = 3;
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    boolean hasDef();
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef();
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder();
+  }
+  /**
+   * Protobuf type {@code exec.shared.QueryData}
+   *
+   * <pre>
+   *
+   * Used by the server when sending query result data batches to the client
+   * </pre>
+   */
+  public static final class QueryData extends
+      com.google.protobuf.GeneratedMessage
+      implements QueryDataOrBuilder {
+    // Use QueryData.newBuilder() to construct.
+    private QueryData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private QueryData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final QueryData defaultInstance;
+    public static QueryData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public QueryData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private QueryData(
+        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 10: {
+              org.apache.drill.exec.proto.UserBitShared.QueryId.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = queryId_.toBuilder();
+              }
+              queryId_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.QueryId.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(queryId_);
+                queryId_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              rowCount_ = input.readInt32();
+              break;
+            }
+            case 26: {
+              org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = def_.toBuilder();
+              }
+              def_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(def_);
+                def_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              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.UserBitShared.internal_static_exec_shared_QueryData_descriptor;
+    }
 
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.drill.exec.proto.UserBitShared.QueryData.class, org.apache.drill.exec.proto.UserBitShared.QueryData.Builder.class);
+    }
 
-      /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
-       */
-      public java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError> getErrorList() {
-        if (errorBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(error_);
+    public static com.google.protobuf.Parser<QueryData> PARSER =
+        new com.google.protobuf.AbstractParser<QueryData>() {
+      public QueryData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new QueryData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<QueryData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .exec.shared.QueryId query_id = 1;
+    public static final int QUERY_ID_FIELD_NUMBER = 1;
+    private org.apache.drill.exec.proto.UserBitShared.QueryId queryId_;
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    public boolean hasQueryId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.QueryId getQueryId() {
+      return queryId_;
+    }
+    /**
+     * <code>optional .exec.shared.QueryId query_id = 1;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getQueryIdOrBuilder() {
+      return queryId_;
+    }
+
+    // optional int32 row_count = 2;
+    public static final int ROW_COUNT_FIELD_NUMBER = 2;
+    private int rowCount_;
+    /**
+     * <code>optional int32 row_count = 2;</code>
+     */
+    public boolean hasRowCount() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int32 row_count = 2;</code>
+     */
+    public int getRowCount() {
+      return rowCount_;
+    }
+
+    // optional .exec.shared.RecordBatchDef def = 3;
+    public static final int DEF_FIELD_NUMBER = 3;
+    private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_;
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    public boolean hasDef() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
+      return def_;
+    }
+    /**
+     * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
+      return def_;
+    }
+
+    private void initFields() {
+      queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+      rowCount_ = 0;
+      def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+    }
+    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.writeMessage(1, queryId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(2, rowCount_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, def_);
+      }
+      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
+          .computeMessageSize(1, queryId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, rowCount_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, def_);
+      }
+      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.UserBitShared.QueryData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryData 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.UserBitShared.QueryData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryData 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.UserBitShared.QueryData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryData 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.UserBitShared.QueryData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryData 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.UserBitShared.QueryData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryData 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.UserBitShared.QueryData 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.shared.QueryData}
+     *
+     * <pre>
+     *
+     * Used by the server when sending query result data batches to the client
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.drill.exec.proto.UserBitShared.QueryDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.drill.exec.proto.UserBitShared.QueryData.class, org.apache.drill.exec.proto.UserBitShared.QueryData.Builder.class);
+      }
+
+      // Construct using org.apache.drill.exec.proto.UserBitShared.QueryData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getQueryIdFieldBuilder();
+          getDefFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (queryIdBuilder_ == null) {
+          queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
         } else {
-          return errorBuilder_.getMessageList();
+          queryIdBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        rowCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (defBuilder_ == null) {
+          def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+        } else {
+          defBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
       }
-      /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
-       */
-      public int getErrorCount() {
-        if (errorBuilder_ == null) {
-          return error_.size();
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryData_descriptor;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryData getDefaultInstanceForType() {
+        return org.apache.drill.exec.proto.UserBitShared.QueryData.getDefaultInstance();
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryData build() {
+        org.apache.drill.exec.proto.UserBitShared.QueryData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryData buildPartial() {
+        org.apache.drill.exec.proto.UserBitShared.QueryData result = new org.apache.drill.exec.proto.UserBitShared.QueryData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (queryIdBuilder_ == null) {
+          result.queryId_ = queryId_;
         } else {
-          return errorBuilder_.getCount();
+          result.queryId_ = queryIdBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.rowCount_ = rowCount_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
         }
+        if (defBuilder_ == null) {
+          result.def_ = def_;
+        } else {
+          result.def_ = defBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
       }
-      /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError(int index) {
-        if (errorBuilder_ == null) {
-          return error_.get(index);
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.drill.exec.proto.UserBitShared.QueryData) {
+          return mergeFrom((org.apache.drill.exec.proto.UserBitShared.QueryData)other);
         } else {
-          return errorBuilder_.getMessage(index);
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.QueryData other) {
+        if (other == org.apache.drill.exec.proto.UserBitShared.QueryData.getDefaultInstance()) return this;
+        if (other.hasQueryId()) {
+          mergeQueryId(other.getQueryId());
+        }
+        if (other.hasRowCount()) {
+          setRowCount(other.getRowCount());
+        }
+        if (other.hasDef()) {
+          mergeDef(other.getDef());
         }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
       }
-      /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
-       */
-      public Builder setError(
-          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
-        if (errorBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
+
+      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.UserBitShared.QueryData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.drill.exec.proto.UserBitShared.QueryData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
           }
-          ensureErrorIsMutable();
-          error_.set(index, value);
-          onChanged();
-        } else {
-          errorBuilder_.setMessage(index, value);
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional .exec.shared.QueryId query_id = 1;
+      private org.apache.drill.exec.proto.UserBitShared.QueryId queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder> queryIdBuilder_;
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder setError(
-          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
-        if (errorBuilder_ == null) {
-          ensureErrorIsMutable();
-          error_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          errorBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
+      public boolean hasQueryId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder addError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
-        if (errorBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureErrorIsMutable();
-          error_.add(value);
-          onChanged();
+      public org.apache.drill.exec.proto.UserBitShared.QueryId getQueryId() {
+        if (queryIdBuilder_ == null) {
+          return queryId_;
         } else {
-          errorBuilder_.addMessage(value);
+          return queryIdBuilder_.getMessage();
         }
-        return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder addError(
-          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
-        if (errorBuilder_ == null) {
+      public Builder setQueryId(org.apache.drill.exec.proto.UserBitShared.QueryId value) {
+        if (queryIdBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureErrorIsMutable();
-          error_.add(index, value);
+          queryId_ = value;
           onChanged();
         } else {
-          errorBuilder_.addMessage(index, value);
+          queryIdBuilder_.setMessage(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder addError(
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
-        if (errorBuilder_ == null) {
-          ensureErrorIsMutable();
-          error_.add(builderForValue.build());
+      public Builder setQueryId(
+          org.apache.drill.exec.proto.UserBitShared.QueryId.Builder builderForValue) {
+        if (queryIdBuilder_ == null) {
+          queryId_ = builderForValue.build();
           onChanged();
         } else {
-          errorBuilder_.addMessage(builderForValue.build());
+          queryIdBuilder_.setMessage(builderForValue.build());
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder addError(
-          int index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
-        if (errorBuilder_ == null) {
-          ensureErrorIsMutable();
-          error_.add(index, builderForValue.build());
+      public Builder mergeQueryId(org.apache.drill.exec.proto.UserBitShared.QueryId value) {
+        if (queryIdBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              queryId_ != org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance()) {
+            queryId_ =
+              org.apache.drill.exec.proto.UserBitShared.QueryId.newBuilder(queryId_).mergeFrom(value).buildPartial();
+          } else {
+            queryId_ = value;
+          }
           onChanged();
         } else {
-          errorBuilder_.addMessage(index, builderForValue.build());
+          queryIdBuilder_.mergeFrom(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder addAllError(
-          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBError> values) {
-        if (errorBuilder_ == null) {
-          ensureErrorIsMutable();
-          super.addAll(values, error_);
+      public Builder clearQueryId() {
+        if (queryIdBuilder_ == null) {
+          queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
           onChanged();
         } else {
-          errorBuilder_.addAllMessages(values);
+          queryIdBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder clearError() {
-        if (errorBuilder_ == null) {
-          error_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000100);
-          onChanged();
-        } else {
-          errorBuilder_.clear();
-        }
-        return this;
+      public org.apache.drill.exec.proto.UserBitShared.QueryId.Builder getQueryIdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getQueryIdFieldBuilder().getBuilder();
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public Builder removeError(int index) {
-        if (errorBuilder_ == null) {
-          ensureErrorIsMutable();
-          error_.remove(index);
-          onChanged();
+      public org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getQueryIdOrBuilder() {
+        if (queryIdBuilder_ != null) {
+          return queryIdBuilder_.getMessageOrBuilder();
         } else {
-          errorBuilder_.remove(index);
+          return queryId_;
         }
-        return this;
-      }
-      /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder(
-          int index) {
-        return getErrorFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional .exec.shared.QueryId query_id = 1;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder(
-          int index) {
-        if (errorBuilder_ == null) {
-          return error_.get(index);  } else {
-          return errorBuilder_.getMessageOrBuilder(index);
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder> 
+          getQueryIdFieldBuilder() {
+        if (queryIdBuilder_ == null) {
+          queryIdBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder>(
+                  queryId_,
+                  getParentForChildren(),
+                  isClean());
+          queryId_ = null;
         }
+        return queryIdBuilder_;
       }
+
+      // optional int32 row_count = 2;
+      private int rowCount_ ;
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional int32 row_count = 2;</code>
        */
-      public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
-           getErrorOrBuilderList() {
-        if (errorBuilder_ != null) {
-          return errorBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(error_);
-        }
+      public boolean hasRowCount() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional int32 row_count = 2;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder addErrorBuilder() {
-        return getErrorFieldBuilder().addBuilder(
-            org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance());
+      public int getRowCount() {
+        return rowCount_;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional int32 row_count = 2;</code>
        */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder addErrorBuilder(
-          int index) {
-        return getErrorFieldBuilder().addBuilder(
-            index, org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance());
+      public Builder setRowCount(int value) {
+        bitField0_ |= 0x00000002;
+        rowCount_ = value;
+        onChanged();
+        return this;
       }
       /**
-       * <code>repeated .exec.shared.DrillPBError error = 9;</code>
+       * <code>optional int32 row_count = 2;</code>
        */
-      public java.util.List<org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder> 
-           getErrorBuilderList() {
-        return getErrorFieldBuilder().getBuilderList();
-      }
-      private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
-          getErrorFieldBuilder() {
-        if (errorBuilder_ == null) {
-          errorBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
-              org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
-                  error_,
-                  ((bitField0_ & 0x00000100) == 0x00000100),
-                  getParentForChildren(),
-                  isClean());
-          error_ = null;
-        }
-        return errorBuilder_;
+      public Builder clearRowCount() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        rowCount_ = 0;
+        onChanged();
+        return this;
       }
 
-      // optional .exec.shared.RecordBatchDef def = 10;
+      // optional .exec.shared.RecordBatchDef def = 3;
       private org.apache.drill.exec.proto.UserBitShared.RecordBatchDef def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder> defBuilder_;
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public boolean hasDef() {
-        return ((bitField0_ & 0x00000200) == 0x00000200);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef getDef() {
         if (defBuilder_ == null) {
@@ -11772,7 +11565,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public Builder setDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
         if (defBuilder_ == null) {
@@ -11784,11 +11577,11 @@ public final class UserBitShared {
         } else {
           defBuilder_.setMessage(value);
         }
-        bitField0_ |= 0x00000200;
+        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public Builder setDef(
           org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder builderForValue) {
@@ -11798,15 +11591,15 @@ public final class UserBitShared {
         } else {
           defBuilder_.setMessage(builderForValue.build());
         }
-        bitField0_ |= 0x00000200;
+        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public Builder mergeDef(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef value) {
         if (defBuilder_ == null) {
-          if (((bitField0_ & 0x00000200) == 0x00000200) &&
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
               def_ != org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance()) {
             def_ =
               org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(def_).mergeFrom(value).buildPartial();
@@ -11817,11 +11610,11 @@ public final class UserBitShared {
         } else {
           defBuilder_.mergeFrom(value);
         }
-        bitField0_ |= 0x00000200;
+        bitField0_ |= 0x00000004;
         return this;
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public Builder clearDef() {
         if (defBuilder_ == null) {
@@ -11830,19 +11623,19 @@ public final class UserBitShared {
         } else {
           defBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000200);
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder getDefBuilder() {
-        bitField0_ |= 0x00000200;
+        bitField0_ |= 0x00000004;
         onChanged();
         return getDefFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder() {
         if (defBuilder_ != null) {
@@ -11852,7 +11645,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>optional .exec.shared.RecordBatchDef def = 10;</code>
+       * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.drill.exec.proto.UserBitShared.RecordBatchDef, org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.Builder, org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder> 
@@ -11868,48 +11661,15 @@ public final class UserBitShared {
         return defBuilder_;
       }
 
-      // optional bool schema_changed = 11;
-      private boolean schemaChanged_ ;
-      /**
-       * <code>optional bool schema_changed = 11;</code>
-       */
-      public boolean hasSchemaChanged() {
-        return ((bitField0_ & 0x00000400) == 0x00000400);
-      }
-      /**
-       * <code>optional bool schema_changed = 11;</code>
-       */
-      public boolean getSchemaChanged() {
-        return schemaChanged_;
-      }
-      /**
-       * <code>optional bool schema_changed = 11;</code>
-       */
-      public Builder setSchemaChanged(boolean value) {
-        bitField0_ |= 0x00000400;
-        schemaChanged_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool schema_changed = 11;</code>
-       */
-      public Builder clearSchemaChanged() {
-        bitField0_ = (bitField0_ & ~0x00000400);
-        schemaChanged_ = false;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:exec.shared.QueryResult)
+      // @@protoc_insertion_point(builder_scope:exec.shared.QueryData)
     }
 
     static {
-      defaultInstance = new QueryResult(true);
+      defaultInstance = new QueryData(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:exec.shared.QueryResult)
+    // @@protoc_insertion_point(class_scope:exec.shared.QueryData)
   }
 
   public interface QueryInfoOrBuilder
@@ -19790,6 +19550,11 @@ public final class UserBitShared {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_exec_shared_QueryResult_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_exec_shared_QueryData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_exec_shared_QueryData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_exec_shared_QueryInfo_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -19866,77 +19631,74 @@ public final class UserBitShared {
       "t\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\023\n\013grou" +
       "p_count\030\006 \001(\005\022\025\n\rbuffer_length\030\007 \001(\005\"7\n\n" +
       "NodeStatus\022\017\n\007node_id\030\001 \001(\005\022\030\n\020memory_fo" +
-      "otprint\030\002 \001(\003\"\346\003\n\013QueryResult\0228\n\013query_s" +
+      "otprint\030\002 \001(\003\"\206\002\n\013QueryResult\0228\n\013query_s" +
       "tate\030\001 \001(\0162#.exec.shared.QueryResult.Que" +
       "ryState\022&\n\010query_id\030\002 \001(\0132\024.exec.shared." +
-      "QueryId\022\025\n\ris_last_chunk\030\003 \001(\010\022\021\n\trow_co" +
-      "unt\030\004 \001(\005\022\024\n\014records_scan\030\005 \001(\003\022\025\n\rrecor" +
-      "ds_error\030\006 \001(\003\022\027\n\017submission_time\030\007 \001(\003\022" +
-      ",\n\013node_status\030\010 \003(\0132\027.exec.shared.NodeS",
-      "tatus\022(\n\005error\030\t \003(\0132\031.exec.shared.Drill" +
-      "PBError\022(\n\003def\030\n \001(\0132\033.exec.shared.Recor" +
-      "dBatchDef\022\026\n\016schema_changed\030\013 \001(\010\"k\n\nQue" +
-      "ryState\022\013\n\007PENDING\020\000\022\013\n\007RUN

<TRUNCATED>