You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2015/05/10 18:30:46 UTC

[01/12] drill git commit: DRILL-2841: Fix slow web UI on some cluster nodes due to unneeded transitive dependencies

Repository: drill
Updated Branches:
  refs/heads/merge_2015_05_09 [created] 6a7de92e3


DRILL-2841: Fix slow web UI on some cluster nodes due to unneeded transitive dependencies

Fix enforcer rule for jetty servlet-api global exclusion, add explicit exclusions where needed.


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

Branch: refs/heads/merge_2015_05_09
Commit: 684389da5a757d3298479703ed1b03e7e0e3bcd2
Parents: 0f42409
Author: Jason Altekruse <al...@gmail.com>
Authored: Sat May 9 19:07:16 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:24:21 2015 -0700

----------------------------------------------------------------------
 contrib/storage-hive/hive-exec-shade/pom.xml | 4 ++++
 exec/java-exec/pom.xml                       | 6 ++++++
 pom.xml                                      | 2 +-
 3 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/684389da/contrib/storage-hive/hive-exec-shade/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 7864955..270e7df 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -42,6 +42,10 @@
       <scope>compile</scope>
       <exclusions>
         <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
           <groupId>commons-logging</groupId>
           <artifactId>commons-logging</artifactId>
         </exclusion>

http://git-wip-us.apache.org/repos/asf/drill/blob/684389da/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 57cd572..235df0d 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -375,6 +375,12 @@
       <groupId>org.apache.avro</groupId>
       <artifactId>avro-mapred</artifactId>
       <version>1.7.7</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/drill/blob/684389da/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c5004fa..6344306 100644
--- a/pom.xml
+++ b/pom.xml
@@ -288,7 +288,7 @@
                   <excludes>
                     <exclude>commons-logging</exclude>
                     <exclude>javax.servlet:servlet-api</exclude>
-                    <exclude>org.mortbay.jetty:servlet-api-2.5</exclude>
+                    <exclude>org.mortbay.jetty:servlet-api:2.5</exclude>
                   </excludes>
                 </bannedDependencies>
               </rules>


[07/12] drill git commit: DRILL-3012: Fix issue where remote values rel wasn't losing operatorId.

Posted by ja...@apache.org.
DRILL-3012: Fix issue where remote values rel wasn't losing operatorId.

Also enhance Union rule to avoid more than 2 ways inputs


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

Branch: refs/heads/merge_2015_05_09
Commit: 87051d42176c346bfda0d4a9400a0134cdcf566c
Parents: 684389d
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun May 10 07:46:54 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:26:01 2015 -0700

----------------------------------------------------------------------
 .../drill/exec/physical/config/Values.java      |  2 +-
 .../drill/exec/planner/physical/Prel.java       |  4 +++-
 .../exec/planner/physical/UnionAllPrule.java    | 22 ++++++++++-------
 .../drill/exec/planner/physical/ValuesPrel.java | 25 +++++++++++++++-----
 4 files changed, 37 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/87051d42/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
index fbb069b..ab3952a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Values.java
@@ -55,7 +55,7 @@ public class Values extends AbstractBase implements Leaf {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
     assert children.isEmpty();
-    return this;
+    return new Values(content);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/87051d42/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
index 697d5a5..db37040 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
@@ -19,11 +19,11 @@ package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
 
+import org.apache.calcite.plan.Convention;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillRelNode;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.calcite.plan.Convention;
 
 public interface Prel extends DrillRelNode, Iterable<Prel>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Prel.class);
@@ -45,4 +45,6 @@ public interface Prel extends DrillRelNode, Iterable<Prel>{
   public SelectionVectorMode getEncoding();
   boolean needsFinalColumnReordering();
 
+  // DRILL-3011
+  // public abstract Prel copy(RelTraitSet paramRelTraitSet, List<RelNode> paramList);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/87051d42/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
index 0503b44..51cf866 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
@@ -20,15 +20,17 @@ package org.apache.drill.exec.planner.physical;
 import java.util.List;
 import java.util.logging.Logger;
 
-import org.apache.drill.exec.planner.logical.DrillUnionRel;
-import org.apache.drill.exec.planner.logical.RelOptHelper;
-import org.apache.calcite.rel.InvalidRelException;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.trace.CalciteTrace;
+import org.apache.drill.exec.planner.logical.DrillUnionRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 public class UnionAllPrule extends Prule {
@@ -60,11 +62,15 @@ public class UnionAllPrule extends Prule {
       }
 
       traits = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
-      UnionAllPrel unionAll =
-          new UnionAllPrel(union.getCluster(), traits, convertedInputList,
-              false /* compatibility already checked during logical phase */);
 
-      call.transformTo(unionAll);
+      Preconditions.checkArgument(convertedInputList.size() >= 2, "Union list must be at least two items.");
+      RelNode left = convertedInputList.get(0);
+      for (int i = 1; i < convertedInputList.size(); i++) {
+        left = new UnionAllPrel(union.getCluster(), traits, ImmutableList.of(left, convertedInputList.get(i)),
+            false /* compatibility already checked during logical phase */);
+
+      }
+      call.transformTo(left);
 
     } catch (InvalidRelException e) {
       tracer.warning(e.toString());

http://git-wip-us.apache.org/repos/asf/drill/blob/87051d42/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
index be5bb19..cdb4787 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrel.java
@@ -19,15 +19,18 @@ package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.List;
 
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.drill.common.JSONOptions;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.calcite.rel.AbstractRelNode;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.type.RelDataType;
 
 import com.google.common.collect.Iterators;
 
@@ -50,8 +53,18 @@ public class ValuesPrel extends AbstractRelNode implements Prel {
   }
 
   @Override
-  public Values getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
-    return new Values(content);
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    return creator.addMetadata(this, new Values(content));
+  }
+
+  @Override
+  public Prel copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new ValuesPrel(getCluster(), traitSet, rowType, content);
+  }
+
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return super.clone();
   }
 
   @Override


[10/12] drill git commit: DRILL-2755: Use and handle InterruptedException during query processing.

Posted by ja...@apache.org.
DRILL-2755: Use and handle InterruptedException during query processing.

- Interrupt FragmentExecutor thread as part of FragmentExecutor.cancel()
- Handle InterruptedException in ExternalSortBatch.newSV2(). If the fragment status says
  should not continue, then throw the InterruptedException to caller which returns IterOutcome.STOP
- Add comments reg not handling of InterruptedException in SendingAccountor.waitForSendComplete()
- Handle InterruptedException in OrderedPartitionRecordBatch.getPartitionVectors()
  If interrupted in Thread.sleep calls and fragment status says should not run, then
  return IterOutcome.STOP downstream.
- Interrupt partitioner threads if PartitionerRecordBatch is interrupted while waiting for
  partitioner threads to complete.
- Preserve interrupt status if not handled
- Handle null RecordBatches returned by RawBatchBuffer.getNext() in MergingRecordBatch.buildSchema()
- Change timeout in Foreman to be proportional to the number of intermediate fragments sent instead
  of hard coded limit of 90s.
- Change TimedRunnable to enforce a timeout of 15s per runnable.
  Total timeout is (5s * numOfRunnableTasks) / parallelism.
- Add unit tests
  * Testing cancelling a query interrupts the query fragments which are currently blocked
  * Testing interrupting the partitioner sender which in turn interrupts its helper threads
  * Testing TimedRunanble enforeces timeout for the whole task list.


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

Branch: refs/heads/merge_2015_05_09
Commit: d5476995d77192cf60dec9925db5597bafe9d751
Parents: 52dcd7e
Author: vkorukanti <ve...@gmail.com>
Authored: Wed May 6 09:34:25 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:29:58 2015 -0700

----------------------------------------------------------------------
 .../drill/common/concurrent/ExtendedLatch.java  |  18 +--
 .../apache/drill/exec/ops/SendingAccountor.java |   8 ++
 .../impl/mergereceiver/MergingRecordBatch.java  |  19 +++
 .../OrderedPartitionRecordBatch.java            |  31 ++++-
 .../partitionsender/PartitionerDecorator.java   |  70 +++++++++--
 .../partitionsender/PartitionerTemplate.java    |   3 +-
 .../UnorderedReceiverBatch.java                 |  22 +++-
 .../physical/impl/xsort/ExternalSortBatch.java  |  10 +-
 .../exec/record/RawFragmentBatchProvider.java   |   2 +-
 .../org/apache/drill/exec/rpc/BasicClient.java  |   9 +-
 .../org/apache/drill/exec/rpc/BasicServer.java  |   7 +-
 .../drill/exec/rpc/ReconnectingConnection.java  |   7 +-
 .../apache/drill/exec/rpc/RemoteConnection.java |  18 ++-
 .../apache/drill/exec/rpc/data/DataTunnel.java  |  14 ++-
 .../org/apache/drill/exec/server/Drillbit.java  |   4 +
 .../apache/drill/exec/store/TimedRunnable.java  |  52 +++++++--
 .../exec/testing/CountDownLatchInjection.java   |   5 +
 .../testing/CountDownLatchInjectionImpl.java    |   5 +
 .../exec/testing/ExecutionControlsInjector.java |  28 +++++
 .../exec/testing/NoOpControlsInjector.java      |   4 +
 .../drill/exec/testing/PauseInjection.java      |   7 ++
 .../org/apache/drill/exec/work/WorkManager.java |  24 ++--
 .../exec/work/batch/SpoolingRawBatchBuffer.java |   9 +-
 .../work/batch/UnlimitedRawBatchBuffer.java     |   8 +-
 .../apache/drill/exec/work/foreman/Foreman.java |  14 ++-
 .../exec/work/fragment/FragmentExecutor.java    |  14 +++
 .../exec/server/TestDrillbitResilience.java     | 117 ++++++++++++++++++-
 .../drill/exec/store/TestTimedRunnable.java     | 103 ++++++++++++++++
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   4 +
 29 files changed, 540 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/common/src/main/java/org/apache/drill/common/concurrent/ExtendedLatch.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/concurrent/ExtendedLatch.java b/common/src/main/java/org/apache/drill/common/concurrent/ExtendedLatch.java
index a75ac32..6bbc476 100644
--- a/common/src/main/java/org/apache/drill/common/concurrent/ExtendedLatch.java
+++ b/common/src/main/java/org/apache/drill/common/concurrent/ExtendedLatch.java
@@ -35,16 +35,6 @@ public class ExtendedLatch extends CountDownLatch {
   }
 
   /**
-   * Returns whether or not interruptions should continue to be ignored. This can be overridden in subclasses to check a
-   * state variable or similar.
-   *
-   * @return Whether awaitUninterruptibly() should continue ignoring interruptions.
-   */
-  protected boolean ignoreInterruptions() {
-    return true;
-  }
-
-  /**
    * Await without interruption for a given time.
    * @param waitMillis
    *          Time in milliseconds to wait
@@ -77,12 +67,8 @@ public class ExtendedLatch extends CountDownLatch {
         await();
         return;
       } catch (final InterruptedException e) {
-        if (ignoreInterruptions()) {
-          // if we're still not ready, the while loop will cause us to wait again
-          logger.warn("Interrupted while waiting for event latch.", e);
-        } else {
-          return;
-        }
+        // if we're still not ready, the while loop will cause us to wait again
+        logger.warn("Interrupted while waiting for event latch.", e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
index 0cb5fbf..60ab06c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
@@ -47,7 +47,15 @@ class SendingAccountor {
           wait.acquire(waitForBatches);
           waitForBatches = batchesSent.addAndGet(-1 * waitForBatches);
         } catch (InterruptedException e) {
+          // We should always wait for send complete. If we don't, we'll leak memory or have a memory miss when we try
+          // to send. This should be safe because: network connections should get disconnected and fail a send if a
+          // node goes down, otherwise, the receiving side connection should always consume from the rpc layer
+          // (blocking is cooperative and will get triggered before this)
           logger.warn("Interrupted while waiting for send complete. Continuing to wait.", e);
+
+          // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+          // interruption and respond to it if it wants to.
+          Thread.currentThread().interrupt();
         }
       }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index f19f371..5d990f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -70,6 +70,7 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.testing.ExecutionControlsInjector;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.CopyUtil;
 import org.apache.drill.exec.vector.FixedWidthVector;
@@ -88,6 +89,7 @@ import com.sun.codemodel.JExpr;
  */
 public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP> implements RecordBatch {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergingRecordBatch.class);
+  private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(MergingRecordBatch.class);
 
   private static final int OUTGOING_BATCH_SIZE = 32 * 1024;
 
@@ -141,6 +143,7 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
     stats.startWait();
     final RawFragmentBatchProvider provider = fragProviders[providerIndex];
     try {
+      injector.injectInterruptiblePause(context.getExecutionControls(), "waiting-for-data", logger);
       final RawFragmentBatch b = provider.getNext();
       if (b != null) {
         stats.addLongStat(Metric.BYTES_RECEIVED, b.getByteCount());
@@ -148,6 +151,12 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
         inputCounts[providerIndex] += b.getHeader().getDef().getRecordCount();
       }
       return b;
+    } catch(final InterruptedException e) {
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
+
+      return null;
     } finally {
       stats.stopWait();
     }
@@ -359,6 +368,7 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
           while (nextBatch != null && nextBatch.getHeader().getDef().getRecordCount() == 0) {
             nextBatch = getNext(node.batchId);
           }
+
           assert nextBatch != null || inputCounts[node.batchId] == outputCounts[node.batchId]
               : String.format("Stream %d input count: %d output count %d", node.batchId, inputCounts[node.batchId], outputCounts[node.batchId]);
           if (nextBatch == null && !context.shouldContinue()) {
@@ -461,6 +471,15 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
           return;
         }
         final RawFragmentBatch batch = getNext(i);
+        if (batch == null) {
+          if (!context.shouldContinue()) {
+            state = BatchState.STOP;
+          } else {
+            state = BatchState.DONE;
+          }
+
+          break;
+        }
         if (batch.getHeader().getDef().getFieldCount() == 0) {
           i++;
           continue;

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index 63b7eba..ca6d83c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -246,6 +246,24 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   }
 
   /**
+   * Wait until the at least the given timeout is expired or interrupted and the fragment status is not runnable.
+   * @param timeout Timeout in milliseconds.
+   * @return True if the given timeout is expired. False when interrupted and the fragment status is not runnable.
+   */
+  private boolean waitUntilTimeOut(final long timeout) {
+    while(true) {
+      try {
+        Thread.sleep(timeout);
+        return true;
+      } catch (final InterruptedException e) {
+        if (!context.shouldContinue()) {
+          return false;
+        }
+      }
+    }
+  }
+
+  /**
    * This method is called when the first batch comes in. Incoming batches are collected until a threshold is met. At
    * that point, the records in the batches are sorted and sampled, and the sampled records are stored in the
    * distributed cache. Once a sufficient fraction of the fragments have shared their samples, each fragment grabs all
@@ -255,10 +273,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
    * @return True is successful. False if failed.
    */
   private boolean getPartitionVectors() {
-
-
     try {
-
       if (!saveSamples()) {
         return false;
       }
@@ -279,14 +294,18 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
         // TODO: this should be polling.
 
         if (val < fragmentsBeforeProceed) {
-          Thread.sleep(10);
+          if (!waitUntilTimeOut(10)) {
+            return false;
+          }
         }
         for (int i = 0; i < 100 && finalTable == null; i++) {
           finalTable = tableMap.get(finalTableKey);
           if (finalTable != null) {
             break;
           }
-          Thread.sleep(10);
+          if (!waitUntilTimeOut(10)) {
+            return false;
+          }
         }
         if (finalTable == null) {
           buildTable();
@@ -302,7 +321,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
         partitionVectors.add(w.getValueVector());
       }
 
-    } catch (ClassTransformationException | IOException | SchemaChangeException | InterruptedException ex) {
+    } catch (final ClassTransformationException | IOException | SchemaChangeException ex) {
       kill(false);
       context.fail(ex);
       return false;

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
index c3261dc..c355070 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerDecorator.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorStats;
@@ -28,6 +29,8 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
+import org.apache.drill.exec.testing.CountDownLatchInjection;
+import org.apache.drill.exec.testing.ExecutionControlsInjector;
 
 /**
  * Decorator class to hide multiple Partitioner existence from the caller
@@ -38,19 +41,22 @@ import com.google.common.collect.Lists;
  * totalWaitTime = totalAllPartitionersProcessingTime - max(sum(processingTime) by partitioner)
  */
 public class PartitionerDecorator {
-
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionerDecorator.class);
+  private static final ExecutionControlsInjector injector =
+      ExecutionControlsInjector.getInjector(PartitionerDecorator.class);
 
   private List<Partitioner> partitioners;
   private final OperatorStats stats;
   private final String tName;
   private final String childThreadPrefix;
   private final ExecutorService executor;
+  private final FragmentContext context;
 
 
   public PartitionerDecorator(List<Partitioner> partitioners, OperatorStats stats, FragmentContext context) {
     this.partitioners = partitioners;
     this.stats = stats;
+    this.context = context;
     this.executor = context.getDrillbitContext().getExecutor();
     this.tName = Thread.currentThread().getName();
     this.childThreadPrefix = "Partitioner-" + tName + "-";
@@ -145,17 +151,42 @@ public class PartitionerDecorator {
     stats.startWait();
     final CountDownLatch latch = new CountDownLatch(partitioners.size());
     final List<CustomRunnable> runnables = Lists.newArrayList();
+    final List<Future> taskFutures = Lists.newArrayList();
+    CountDownLatchInjection testCountDownLatch = null;
     try {
-      int i = 0;
-      for (final Partitioner part : partitioners ) {
-        runnables.add(new CustomRunnable(childThreadPrefix, latch, iface, part));
-        executor.submit(runnables.get(i++));
+      // To simulate interruption of main fragment thread and interrupting the partition threads, create a
+      // CountDownInject patch. Partitioner threads await on the latch and main fragment thread counts down or
+      // interrupts waiting threads. This makes sures that we are actually interrupting the blocked partitioner threads.
+      testCountDownLatch = injector.getLatch(context.getExecutionControls(), "partitioner-sender-latch");
+      testCountDownLatch.initialize(1);
+      for (final Partitioner part : partitioners) {
+        final CustomRunnable runnable = new CustomRunnable(childThreadPrefix, latch, iface, part, testCountDownLatch);
+        runnables.add(runnable);
+        taskFutures.add(executor.submit(runnable));
       }
-      try {
-        latch.await();
-      } catch (InterruptedException e) {
-        throw new RuntimeException(e);
+
+      while (true) {
+        try {
+          // Wait for main fragment interruption.
+          injector.injectInterruptiblePause(context.getExecutionControls(), "wait-for-fragment-interrupt", logger);
+
+          // If there is no pause inserted at site "wait-for-fragment-interrupt", release the latch.
+          injector.getLatch(context.getExecutionControls(), "partitioner-sender-latch").countDown();
+
+          latch.await();
+          break;
+        } catch (final InterruptedException e) {
+          // If the fragment state says we shouldn't continue, cancel or interrupt partitioner threads
+          if (!context.shouldContinue()) {
+            for(Future f : taskFutures) {
+              f.cancel(true);
+            }
+
+            break;
+          }
+        }
       }
+
       IOException excep = null;
       for (final CustomRunnable runnable : runnables ) {
         IOException myException = runnable.getException();
@@ -180,8 +211,12 @@ public class PartitionerDecorator {
       // scale down main stats wait time based on calculated processing time
       // since we did not wait for whole duration of above execution
       stats.adjustWaitNanos(-maxProcessTime);
-    }
 
+      // Done with the latch, close it.
+      if (testCountDownLatch != null) {
+        testCountDownLatch.close();
+      }
+    }
   }
 
   /**
@@ -190,7 +225,7 @@ public class PartitionerDecorator {
    * protected is for testing purposes
    */
   protected interface GeneralExecuteIface {
-    public void execute(Partitioner partitioner) throws IOException;
+    void execute(Partitioner partitioner) throws IOException;
   }
 
   /**
@@ -242,17 +277,28 @@ public class PartitionerDecorator {
     private final CountDownLatch latch;
     private final GeneralExecuteIface iface;
     private final Partitioner part;
+    private CountDownLatchInjection testCountDownLatch;
+
     private volatile IOException exp;
 
-    public CustomRunnable(String parentThreadName, CountDownLatch latch, GeneralExecuteIface iface, Partitioner part) {
+    public CustomRunnable(final String parentThreadName, final CountDownLatch latch, final GeneralExecuteIface iface,
+        final Partitioner part, CountDownLatchInjection testCountDownLatch) {
       this.parentThreadName = parentThreadName;
       this.latch = latch;
       this.iface = iface;
       this.part = part;
+      this.testCountDownLatch = testCountDownLatch;
     }
 
     @Override
     public void run() {
+      // Test only - Pause until interrupted by fragment thread
+      try {
+        testCountDownLatch.await();
+      } catch (final InterruptedException e) {
+        logger.debug("Test only: partitioner thread is interrupted in test countdown latch await()", e);
+      }
+
       final Thread currThread = Thread.currentThread();
       final String currThreadName = currThread.getName();
       final OperatorStats localStats = part.getStats();

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
index cbea267..aeac01d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerTemplate.java
@@ -285,7 +285,8 @@ public abstract class PartitionerTemplate implements Partitioner {
       //      to terminate we need to send at least one batch with "isLastBatch" set to true, so that receiver knows
       //      sender has acknowledged the terminate request. After sending the last batch, all further batches are
       //      dropped.
-      final boolean isLastBatch = isLast || terminated;
+      //   3. Partitioner thread is interrupted due to cancellation of fragment.
+      final boolean isLastBatch = isLast || terminated || Thread.currentThread().isInterrupted();
 
       // if the batch is not the last batch and the current recordCount is zero, then no need to send any RecordBatches
       if (!isLastBatch && recordCount == 0) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
index 66a2092..e40fe54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
@@ -49,9 +49,12 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.testing.ExecutionControlsInjector;
 
 public class UnorderedReceiverBatch implements CloseableRecordBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnorderedReceiverBatch.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnorderedReceiverBatch.class);
+  private final static ExecutionControlsInjector injector =
+      ExecutionControlsInjector.getInjector(UnorderedReceiverBatch.class);
 
   private final RecordBatchLoader batchLoader;
   private final RawFragmentBatchProvider fragProvider;
@@ -133,6 +136,19 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
     return batchLoader.getValueAccessorById(clazz, ids);
   }
 
+  private RawFragmentBatch getNextBatch() throws IOException {
+    try {
+      injector.injectInterruptiblePause(context.getExecutionControls(), "waiting-for-data", logger);
+      return fragProvider.getNext();
+    } catch(final InterruptedException e) {
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
+
+      return null;
+    }
+  }
+
   @Override
   public IterOutcome next() {
     stats.startProcessing();
@@ -140,11 +156,11 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
       RawFragmentBatch batch;
       try {
         stats.startWait();
-        batch = fragProvider.getNext();
+        batch = getNextBatch();
 
         // skip over empty batches. we do this since these are basically control messages.
         while (batch != null && !batch.getHeader().getIsOutOfMemory() && batch.getHeader().getDef().getRecordCount() == 0 && (!first || batch.getHeader().getDef().getFieldCount() == 0)) {
-          batch = fragProvider.getNext();
+          batch = getNextBatch();
         }
       } finally {
         stats.stopWait();

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index aab3391..3159811 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -277,6 +277,8 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
           } else {
             try {
               sv2 = newSV2();
+            } catch(InterruptedException e) {
+              return IterOutcome.STOP;
             } catch (OutOfMemoryException e) {
               throw new OutOfMemoryRuntimeException(e);
             }
@@ -496,7 +498,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     return size;
   }
 
-  private SelectionVector2 newSV2() throws OutOfMemoryException {
+  private SelectionVector2 newSV2() throws OutOfMemoryException, InterruptedException {
     SelectionVector2 sv2 = new SelectionVector2(oContext.getAllocator());
     if (!sv2.allocateNew(incoming.getRecordCount())) {
       try {
@@ -509,8 +511,10 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       while (true) {
         try {
           Thread.sleep(waitTime * 1000);
-        } catch (InterruptedException e) {
-          throw new OutOfMemoryException(e);
+        } catch(final InterruptedException e) {
+          if (!context.shouldContinue()) {
+            throw e;
+          }
         }
         waitTime *= 2;
         if (sv2.allocateNew(incoming.getRecordCount())) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
index d4dfe96..030785c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
@@ -23,7 +23,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 
 public interface RawFragmentBatchProvider {
 
-  public RawFragmentBatch getNext() throws IOException;
+  public RawFragmentBatch getNext() throws IOException, InterruptedException;
   public void kill(FragmentContext context);
   public void cleanup();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index 1661f81..d551173 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -263,9 +263,12 @@ public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection
     logger.debug("Closing client");
     try {
       connection.getChannel().close().get();
-    } catch (InterruptedException | ExecutionException e) {
-      logger.warn("Failure whiel shutting {}", this.getClass().getName(), e);
-      // TODO InterruptedException
+    } catch (final InterruptedException | ExecutionException e) {
+      logger.warn("Failure while shutting {}", this.getClass().getName(), e);
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/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 a148436..6a7bc65 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
@@ -204,9 +204,12 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
   public void close() throws IOException {
     try {
       eventLoopGroup.shutdownGracefully().get();
-    } catch (InterruptedException | ExecutionException e) {
+    } catch (final InterruptedException | ExecutionException e) {
       logger.warn("Failure while shutting down {}. ", this.getClass().getName(), e);
-      // TODO InterruptedException
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
index 9948d3e..f0787a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ReconnectingConnection.java
@@ -112,9 +112,12 @@ public abstract class ReconnectingConnection<CONNECTION_TYPE extends RemoteConne
           cmd.connectionSucceeded(connection);
 //          logger.debug("Finished connection succeeded activity.");
         }
-      } catch (InterruptedException e) {
+      } catch (final InterruptedException e) {
         cmd.connectionFailed(FailureType.CONNECTION, e);
-        // TODO InterruptedException
+
+        // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+        // interruption and respond to it if it wants to.
+        Thread.currentThread().interrupt();
       } catch (ExecutionException e) {
         throw new IllegalStateException();
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
index 0f095c0..2ee9263 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
@@ -70,9 +70,13 @@ public abstract class RemoteConnection implements ConnectionThrottle, AutoClosea
     try{
       writeManager.waitForWritable();
       return true;
-    }catch(InterruptedException e){
+    }catch(final InterruptedException e){
       listener.failed(new RpcException(e));
-      // TODO InterruptedException
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
+
       return false;
     }
   }
@@ -131,10 +135,14 @@ public abstract class RemoteConnection implements ConnectionThrottle, AutoClosea
       if (channel.isActive()) {
         channel.close().get();
       }
-    } catch (InterruptedException | ExecutionException e) {
+      channel.close().get();
+    } catch (final InterruptedException | ExecutionException e) {
       logger.warn("Caught exception while closing channel.", e);
-      // TODO InterruptedException
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
index 11f5496..ed31bed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataTunnel.java
@@ -48,9 +48,12 @@ public class DataTunnel {
     try{
       sendingSemaphore.acquire();
       manager.runCommand(b);
-    }catch(InterruptedException e){
+    }catch(final InterruptedException e){
       outcomeListener.failed(new RpcException("Interrupted while trying to get sending semaphore.", e));
-      // TODO InterruptedException
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
   }
 
@@ -59,9 +62,12 @@ public class DataTunnel {
     try{
       sendingSemaphore.acquire();
       manager.runCommand(b);
-    }catch(InterruptedException e){
+    }catch(final InterruptedException e){
       b.connectionFailed(FailureType.CONNECTION, new RpcException("Interrupted while trying to get sending semaphore.", e));
-      // TODO InterruptedException
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
     return b.getFuture();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index e7a9a3c..531253e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -271,6 +271,10 @@ public class Drillbit implements AutoCloseable {
       Thread.sleep(context.getConfig().getInt(ExecConstants.ZK_REFRESH) * 2);
     } catch (final InterruptedException e) {
       logger.warn("Interrupted while sleeping during coordination deregistration.");
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
 
     if (embeddedJetty != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java
index 0fb778b..e52820b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/TimedRunnable.java
@@ -20,10 +20,15 @@ package org.apache.drill.exec.store;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
+import org.apache.drill.common.concurrent.ExtendedLatch;
+import org.apache.drill.common.exceptions.UserException;
 import org.slf4j.Logger;
 
 import com.google.common.base.Stopwatch;
@@ -36,6 +41,8 @@ import com.google.common.collect.Lists;
  */
 public abstract class TimedRunnable<V> implements Runnable {
 
+  private static int TIMEOUT_PER_RUNNABLE_IN_MSECS = 15000;
+
   private volatile Exception e;
   private volatile long timeNanos;
   private volatile V value;
@@ -91,10 +98,13 @@ public abstract class TimedRunnable<V> implements Runnable {
   }
 
   /**
-   * Execute the list of runnables with the given parallelization.  At end, return values and report completion time stats to provided logger.
+   * Execute the list of runnables with the given parallelization.  At end, return values and report completion time
+   * stats to provided logger. Each runnable is allowed a certain timeout. If the timeout exceeds, existing/pending
+   * tasks will be cancelled and a {@link UserException} is thrown.
    * @param activity Name of activity for reporting in logger.
    * @param logger The logger to use to report results.
-   * @param runnables List of runnables that should be executed and timed.  If this list has one item, task will be completed in-thread.
+   * @param runnables List of runnables that should be executed and timed.  If this list has one item, task will be
+   *                  completed in-thread. Runnable must handle {@link InterruptedException}s.
    * @param parallelism  The number of threads that should be run to complete this task.
    * @return The list of outcome objects.
    * @throws IOException All exceptions are coerced to IOException since this was build for storage system tasks initially.
@@ -107,25 +117,43 @@ public abstract class TimedRunnable<V> implements Runnable {
       runnables.get(0).run();
     }else{
       parallelism = Math.min(parallelism,  runnables.size());
-      final CountDownLatch latch = new CountDownLatch(runnables.size());
+      final ExtendedLatch latch = new ExtendedLatch(runnables.size());
       final ExecutorService threadPool = Executors.newFixedThreadPool(parallelism);
       try{
         for(TimedRunnable<V> runnable : runnables){
           threadPool.submit(new LatchedRunnable(latch, runnable));
         }
-      }finally{
-        threadPool.shutdown();
-      }
 
-      try{
-        latch.await();
-      }catch(InterruptedException e){
-        // TODO interrupted exception.
-        throw new RuntimeException(e);
+        final long timeout = (long)Math.ceil((TIMEOUT_PER_RUNNABLE_IN_MSECS * runnables.size())/parallelism);
+        if (!latch.awaitUninterruptibly(timeout)) {
+          // Issue a shutdown request. This will cause existing threads to interrupt and pending threads to cancel.
+          // It is highly important that the task Runnables are handling interrupts correctly.
+          threadPool.shutdownNow();
+
+          try {
+            // Wait for 5s for currently running threads to terminate. Above call (threadPool.shutdownNow()) interrupts
+            // any running threads. If the runnables are handling the interrupts properly they should be able to
+            // wrap up and terminate. If not waiting for 5s here gives a chance to identify and log any potential
+            // thread leaks.
+            threadPool.awaitTermination(5, TimeUnit.SECONDS);
+          } catch (final InterruptedException e) {
+            logger.warn("Interrupted while waiting for pending threads in activity '{}' to terminate.", activity);
+          }
+
+          final String errMsg = String.format("Waited for %dms, but tasks for '%s' are not complete. " +
+              "Total runnable size %d, parallelism %d.", timeout, activity, runnables.size(), parallelism);
+          logger.error(errMsg);
+          throw UserException.resourceError()
+              .message(errMsg)
+              .build();
+        }
+      } finally {
+        if (!threadPool.isShutdown()) {
+          threadPool.shutdown();
+        }
       }
     }
 
-
     List<V> values = Lists.newArrayList();
     long sum = 0;
     long max = 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
index de4a181..d26e2bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
@@ -48,4 +48,9 @@ public interface CountDownLatchInjection {
    * Decrements the count of the latch, releasing all waiting threads if the count reaches zero.
    */
   void countDown();
+
+  /**
+   * Close the latch.
+   */
+  void close();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
index f4012c1..561d816 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
@@ -82,4 +82,9 @@ public class CountDownLatchInjectionImpl extends Injection implements CountDownL
     Preconditions.checkArgument(latch.getCount() > 0, "Counting down on latch more than intended.");
     latch.countDown();
   }
+
+  @Override
+  public void close() {
+    latch = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
index 05f8433..387d300 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
@@ -124,6 +124,34 @@ public class ExecutionControlsInjector {
     }
   }
 
+  /**
+   * Insert a pause that can be interrupted using {@link Thread#interrupt()} at the given site point, if such an
+   * injection is specified (i.e. matches the site description).
+   * <p/>
+   * <p>Implementors use this in their code at a site where they want to simulate a interruptible pause
+   * during testing.
+   *
+   * @param executionControls the controls in the current context
+   * @param desc              the site description
+   * @param logger            logger of the class containing the injection site
+   * @throws InterruptedException if interrupted using {@link Thread#interrupt()}
+   */
+  public void injectInterruptiblePause(final ExecutionControls executionControls, final String desc,
+      final Logger logger) throws InterruptedException {
+    final PauseInjection pauseInjection = executionControls.lookupPauseInjection(this, desc);
+
+    if (pauseInjection != null) {
+      logger.debug("Interruptible pausing at {}", desc);
+      try {
+        pauseInjection.interruptiblePause();
+      } catch (final InterruptedException e) {
+        logger.debug("Pause interrupted at {}", desc);
+        throw e;
+      }
+      logger.debug("Interruptible pause resuming at {}", desc);
+    }
+  }
+
   public CountDownLatchInjection getLatch(final ExecutionControls executionControls, final String desc) {
     return executionControls.lookupCountDownLatchInjection(this, desc);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
index 33ab783..bb13d1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
@@ -61,6 +61,10 @@ public final class NoOpControlsInjector extends ExecutionControlsInjector {
     @Override
     public void countDown() {
     }
+
+    @Override
+    public void close() {
+    }
   };
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
index ff0340b..fc4d8ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
@@ -54,6 +54,13 @@ public class PauseInjection extends Injection {
     latch.awaitUninterruptibly();
   }
 
+  public void interruptiblePause() throws InterruptedException {
+    if (!injectNow()) {
+      return;
+    }
+    latch.await();
+  }
+
   public void unpause() {
     latch.countDown();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index f2352e6..1d3a0b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -21,13 +21,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.SelfCleaningRunnable;
+import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
@@ -173,6 +173,10 @@ public class WorkManager implements AutoCloseable {
       }
     } catch (final InterruptedException e) {
       logger.warn("Executor interrupted while awaiting termination");
+
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
     }
   }
 
@@ -180,7 +184,7 @@ public class WorkManager implements AutoCloseable {
     return dContext;
   }
 
-  private CountDownLatch exitLatch = null; // used to wait to exit when things are still running
+  private ExtendedLatch exitLatch = null; // used to wait to exit when things are still running
 
   /**
    * Waits until it is safe to exit. Blocks until all currently running fragments have completed.
@@ -193,17 +197,11 @@ public class WorkManager implements AutoCloseable {
         return;
       }
 
-      exitLatch = new CountDownLatch(1);
+      exitLatch = new ExtendedLatch();
     }
 
-    while(true) {
-      try {
-        exitLatch.await(5, TimeUnit.SECONDS);
-      } catch(final InterruptedException e) {
-        // keep waiting
-      }
-      break;
-    }
+    // Wait for at most 5 seconds or until the latch is released.
+    exitLatch.awaitUninterruptibly(5000);
   }
 
   /**
@@ -328,6 +326,10 @@ public class WorkManager implements AutoCloseable {
         try {
           Thread.sleep(STATUS_PERIOD_SECONDS * 1000);
         } catch(final InterruptedException e) {
+          // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+          // interruption and respond to it if it wants to.
+          Thread.currentThread().interrupt();
+
           // exit status thread on interrupt.
           break;
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
index 2a79e42..07a3505 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
@@ -141,7 +141,7 @@ public class SpoolingRawBatchBuffer implements RawBatchBuffer {
   }
 
   @Override
-  public RawFragmentBatch getNext() throws IOException {
+  public RawFragmentBatch getNext() throws IOException, InterruptedException {
     if (outOfMemory && buffer.size() < 10) {
       outOfMemory = false;
       fragmentManager.setAutoRead(true);
@@ -160,9 +160,12 @@ public class SpoolingRawBatchBuffer implements RawBatchBuffer {
         }
         queueSize -= w.getBodySize();
         return batch;
-      } catch (InterruptedException e) {
+      } catch (final InterruptedException e) {
+        // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+        // interruption and respond to it if it wants to.
+        Thread.currentThread().interrupt();
+
         return null;
-        // TODO InterruptedException
       }
     }
     if (w == null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
index d23655c..4750666 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
@@ -30,7 +30,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Queues;
 
 public class UnlimitedRawBatchBuffer implements RawBatchBuffer{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlimitedRawBatchBuffer.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlimitedRawBatchBuffer.class);
 
   private static enum BufferState {
     INIT,
@@ -149,7 +149,7 @@ public class UnlimitedRawBatchBuffer implements RawBatchBuffer{
   }
 
   @Override
-  public RawFragmentBatch getNext() {
+  public RawFragmentBatch getNext() throws IOException, InterruptedException {
 
     if (outOfMemory.get() && buffer.size() < 10) {
       logger.trace("Setting autoread true");
@@ -166,8 +166,8 @@ public class UnlimitedRawBatchBuffer implements RawBatchBuffer{
       try {
         b = buffer.take();
       } catch (final InterruptedException e) {
-        return null;
-        // TODO InterruptedException
+        logger.debug("Interrupted while waiting for incoming data.", e);
+        throw e;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/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 0122ef8..bf62ccb 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
@@ -109,7 +109,7 @@ public class Foreman implements Runnable {
   private static final org.slf4j.Logger queryLogger = org.slf4j.LoggerFactory.getLogger("query.logger");
   private static final ObjectMapper MAPPER = new ObjectMapper();
   private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(Foreman.class);
-  private static final int RPC_WAIT_IN_SECONDS = 90;
+  private static final int RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
 
   private final QueryId queryId;
   private final RunQuery queryRequest;
@@ -967,7 +967,8 @@ public class Foreman implements Runnable {
      * count down (see FragmentSubmitFailures), but we count the number of failures so that we'll
      * know if any submissions did fail.
      */
-    final ExtendedLatch endpointLatch = new ExtendedLatch(intFragmentMap.keySet().size());
+    final int numIntFragments = intFragmentMap.keySet().size();
+    final ExtendedLatch endpointLatch = new ExtendedLatch(numIntFragments);
     final FragmentSubmitFailures fragmentSubmitFailures = new FragmentSubmitFailures();
 
     // send remote intermediate fragments
@@ -975,16 +976,17 @@ public class Foreman implements Runnable {
       sendRemoteFragments(ep, intFragmentMap.get(ep), endpointLatch, fragmentSubmitFailures);
     }
 
-    if(!endpointLatch.awaitUninterruptibly(RPC_WAIT_IN_SECONDS * 1000)){
+    final long timeout = RPC_WAIT_IN_MSECS_PER_FRAGMENT * numIntFragments;
+    if(numIntFragments > 0 && !endpointLatch.awaitUninterruptibly(timeout)){
       long numberRemaining = endpointLatch.getCount();
       throw UserException.connectionError()
           .message(
-              "Exceeded timeout while waiting send intermediate work fragments to remote nodes.  Sent %d and only heard response back from %d nodes.",
-              intFragmentMap.keySet().size(), intFragmentMap.keySet().size() - numberRemaining)
+              "Exceeded timeout (%d) while waiting send intermediate work fragments to remote nodes. " +
+                  "Sent %d and only heard response back from %d nodes.",
+              timeout, numIntFragments, numIntFragments - numberRemaining)
           .build();
     }
 
-
     // if any of the intermediate fragment submissions failed, fail the query
     final List<FragmentSubmitFailures.SubmissionException> submissionExceptions = fragmentSubmitFailures.submissionExceptions;
     if (submissionExceptions.size() > 0) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index 24e2556..d96e6d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -62,6 +62,9 @@ public class FragmentExecutor implements Runnable {
   private final AtomicReference<FragmentState> fragmentState = new AtomicReference<>(FragmentState.AWAITING_ALLOCATION);
   private final ExtendedLatch acceptExternalEvents = new ExtendedLatch();
 
+  // Thread that is currently executing the Fragment. Value is null if the fragment hasn't started running or finished
+  private final AtomicReference<Thread> myThreadRef = new AtomicReference<>(null);
+
   public FragmentExecutor(final FragmentContext context, final FragmentRoot rootOperator,
                           final StatusReporter listener) {
     this.fragmentContext = context;
@@ -136,6 +139,14 @@ public class FragmentExecutor implements Runnable {
        * We set the cancel requested flag but the actual cancellation is managed by the run() loop, if called.
        */
       updateState(FragmentState.CANCELLATION_REQUESTED);
+
+      /*
+       * Interrupt the thread so that it exits from any blocking operation it could be executing currently.
+       */
+      final Thread myThread = myThreadRef.get();
+      if (myThread != null) {
+        myThread.interrupt();
+      }
     }
   }
 
@@ -168,6 +179,7 @@ public class FragmentExecutor implements Runnable {
   @Override
   public void run() {
     final Thread myThread = Thread.currentThread();
+    myThreadRef.set(myThread);
     final String originalThreadName = myThread.getName();
     final FragmentHandle fragmentHandle = fragmentContext.getHandle();
     final DrillbitContext drillbitContext = fragmentContext.getDrillbitContext();
@@ -244,6 +256,8 @@ public class FragmentExecutor implements Runnable {
       clusterCoordinator.removeDrillbitStatusListener(drillbitStatusListener);
 
       myThread.setName(originalThreadName);
+
+      myThreadRef.set(null);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/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 3e4dcb2..d72d498 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
@@ -17,6 +17,10 @@
  */
 package org.apache.drill.exec.server;
 
+import static org.apache.drill.exec.ExecConstants.SLICE_TARGET;
+import static org.apache.drill.exec.ExecConstants.SLICE_TARGET_DEFAULT;
+import static org.apache.drill.exec.planner.physical.PlannerSettings.HASHAGG;
+import static org.apache.drill.exec.planner.physical.PlannerSettings.PARTITION_SENDER_SET_THREADS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -43,6 +47,10 @@ 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.physical.impl.ScreenCreator;
+import org.apache.drill.exec.physical.impl.mergereceiver.MergingRecordBatch;
+import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec;
+import org.apache.drill.exec.physical.impl.partitionsender.PartitionerDecorator;
+import org.apache.drill.exec.physical.impl.unorderedreceiver.UnorderedReceiverBatch;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
@@ -437,6 +445,20 @@ public class TestDrillbitResilience {
     }
   }
 
+  private static class ListenerThatCancelsQueryAfterFirstBatchOfData extends WaitUntilCompleteListener {
+    private boolean cancelRequested = false;
+
+    @Override
+    public void dataArrived(final QueryDataBatch result, final ConnectionThrottle throttle) {
+      if (!cancelRequested) {
+        check(queryId != null, "Query id should not be null, since we have waited long enough.");
+        (new CancellingThread(queryId, ex, null)).start();
+        cancelRequested = true;
+      }
+      result.release();
+    }
+  };
+
   /**
    * Thread that cancels the given query id. After the cancel is acknowledged, the latch is counted down.
    */
@@ -459,7 +481,9 @@ public class TestDrillbitResilience {
       } catch (final RpcException ex) {
         this.ex.value = ex;
       }
-      latch.countDown();
+      if (latch != null) {
+        latch.countDown();
+      }
     }
   }
 
@@ -507,13 +531,33 @@ public class TestDrillbitResilience {
    * Given a set of controls, this method ensures that the TEST_QUERY completes with a CANCELED state.
    */
   private static void assertCancelledWithoutException(final String controls, final WaitUntilCompleteListener listener) {
+    assertCancelled(controls, TEST_QUERY, listener);
+  }
+
+  /**
+   * Given a set of controls, this method ensures that the given query completes with a CANCELED state.
+   */
+  private static void assertCancelled(final String controls, final String testQuery,
+      final WaitUntilCompleteListener listener) {
     setControls(controls);
 
-    QueryTestUtil.testWithListener(drillClient, QueryType.SQL, TEST_QUERY, listener);
+    QueryTestUtil.testWithListener(drillClient, QueryType.SQL, testQuery, listener);
     final Pair<QueryState, Exception> result = listener.waitForCompletion();
     assertCompleteState(result, QueryState.CANCELED);
   }
 
+  private static void setSessionOption(final String option, final String value) {
+    try {
+      final List<QueryDataBatch> results = drillClient.runQuery(QueryType.SQL,
+          String.format("alter session set `%s` = %s", option, value));
+      for (final QueryDataBatch data : results) {
+        data.release();
+      }
+    } catch(RpcException e) {
+      fail(String.format("Failed to set session option `%s` = %s, Error: %s", option, value, e.toString()));
+    }
+  }
+
   private static String createPauseInjection(final Class siteClass, final String siteDesc, final int nSkip) {
     return "{\"injections\" : [{"
       + "\"type\" : \"pause\"," +
@@ -667,4 +711,73 @@ public class TestDrillbitResilience {
     final String controls = createSingleException(FragmentExecutor.class, exceptionDesc, exceptionClass);
     assertFailsWithException(controls, exceptionClass, exceptionDesc);
   }
+
+  /**
+   * Test cancelling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
+   * Specifically tests cancelling fragment which has {@link MergingRecordBatch} blocked waiting for data.
+   */
+  @Test
+  public void testInterruptingBlockedMergingRecordBatch() {
+    final String control = createPauseInjection(MergingRecordBatch.class, "waiting-for-data", 1);
+    testInterruptingBlockedFragmentsWaitingForData(control);
+  }
+
+  /**
+   * Test cancelling query interrupts currently blocked FragmentExecutor threads waiting for some event to happen.
+   * Specifically tests cancelling fragment which has {@link UnorderedReceiverBatch} blocked waiting for data.
+   */
+  @Test
+  public void testInterruptingBlockedUnorderedReceiverBatch() {
+    final String control = createPauseInjection(UnorderedReceiverBatch.class, "waiting-for-data", 1);
+    testInterruptingBlockedFragmentsWaitingForData(control);
+  }
+
+  private static void testInterruptingBlockedFragmentsWaitingForData(final String control) {
+    try {
+      setSessionOption(SLICE_TARGET, "1");
+      setSessionOption(HASHAGG.getOptionName(), "false");
+
+      final String query = "SELECT sales_city, COUNT(*) cnt FROM cp.`region.json` GROUP BY sales_city";
+      assertCancelled(control, query, new ListenerThatCancelsQueryAfterFirstBatchOfData());
+    } finally {
+      setSessionOption(SLICE_TARGET, Long.toString(SLICE_TARGET_DEFAULT));
+      setSessionOption(HASHAGG.getOptionName(), HASHAGG.getDefault().bool_val.toString());
+    }
+  }
+
+  /**
+   * Tests interrupting the fragment thread that is running {@link PartitionSenderRootExec}.
+   * {@link PartitionSenderRootExec} spawns threads for partitioner. Interrupting fragment thread should also interrupt
+   * the partitioner threads.
+   */
+  @Test
+  public void testInterruptingPartitionerThreadFragment() {
+    try {
+      setSessionOption(SLICE_TARGET, "1");
+      setSessionOption(HASHAGG.getOptionName(), "true");
+      setSessionOption(PARTITION_SENDER_SET_THREADS.getOptionName(), "6");
+
+      final String controls = "{\"injections\" : ["
+          + "{"
+          + "\"type\" : \"latch\","
+          + "\"siteClass\" : \"" + PartitionerDecorator.class.getName() + "\","
+          + "\"desc\" : \"partitioner-sender-latch\""
+          + "},"
+          + "{"
+          + "\"type\" : \"pause\","
+          + "\"siteClass\" : \"" + PartitionerDecorator.class.getName() + "\","
+          + "\"desc\" : \"wait-for-fragment-interrupt\","
+          + "\"nSkip\" : 1"
+          + "}" +
+          "]}";
+
+      final String query = "SELECT sales_city, COUNT(*) cnt FROM cp.`region.json` GROUP BY sales_city";
+      assertCancelled(controls, query, new ListenerThatCancelsQueryAfterFirstBatchOfData());
+    } finally {
+      setSessionOption(SLICE_TARGET, Long.toString(SLICE_TARGET_DEFAULT));
+      setSessionOption(HASHAGG.getOptionName(), HASHAGG.getDefault().bool_val.toString());
+      setSessionOption(PARTITION_SENDER_SET_THREADS.getOptionName(),
+          Long.toString(PARTITION_SENDER_SET_THREADS.getDefault().num_val));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestTimedRunnable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestTimedRunnable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestTimedRunnable.java
new file mode 100644
index 0000000..2807c35
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestTimedRunnable.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.test.DrillTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Unit testing for {@link TimedRunnable}.
+ */
+public class TestTimedRunnable extends DrillTest {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTimedRunnable.class);
+
+  @Rule
+  public final TestRule TIMEOUT = TestTools.getTimeoutRule(180000); // 3mins
+
+  private static class TestTask extends TimedRunnable {
+    final long sleepTime; // sleep time in ms
+
+    public TestTask(final long sleepTime) {
+      this.sleepTime = sleepTime;
+    }
+
+    @Override
+    protected Void runInner() throws Exception {
+      try {
+        Thread.sleep(sleepTime);
+      } catch (InterruptedException e) {
+        throw e;
+      }
+      return null;
+    }
+
+    @Override
+    protected IOException convertToIOException(Exception e) {
+      return new IOException("Failure while trying to sleep for sometime", e);
+    }
+  }
+
+  @Test
+  public void withoutAnyTasksTriggeringTimeout() throws Exception {
+    List<TimedRunnable<TestTask>> tasks = Lists.newArrayList();
+
+    for(int i=0; i<100; i++){
+      tasks.add(new TestTask(2000));
+    }
+
+    TimedRunnable.run("Execution without triggering timeout", logger, tasks, 16);
+  }
+
+  @Test
+  public void withTasksExceedingTimeout() throws Exception {
+    UserException ex = null;
+
+    try {
+      List<TimedRunnable<TestTask>> tasks = Lists.newArrayList();
+
+      for (int i = 0; i < 100; i++) {
+        if ((i & (i + 1)) == 0) {
+          tasks.add(new TestTask(2000));
+        } else {
+          tasks.add(new TestTask(20000));
+        }
+      }
+
+      TimedRunnable.run("Execution with some tasks triggering timeout", logger, tasks, 16);
+    } catch (UserException e) {
+      ex = e;
+    }
+
+    assertNotNull("Expected a UserException", ex);
+    assertThat(ex.getMessage(),
+        containsString("Waited for 93750ms, but tasks for 'Execution with some tasks triggering timeout' are not " +
+            "complete. Total runnable size 100, parallelism 16."));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/d5476995/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 8ef2af3..2fe6c28 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -156,6 +156,10 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
       // calling some wait method?
       resultsListener.latch.await();
     } catch ( InterruptedException e ) {
+      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
+      // interruption and respond to it if it wants to.
+      Thread.currentThread().interrupt();
+
       // Not normally expected--Drill doesn't interrupt in this area (right?)--
       // but JDBC client certainly could.
       throw new SQLException( "Interrupted", e );


[11/12] drill git commit: DRILL-2870: Part1 - Separate count aggregate functions from others Count is the only aggregate function with required type as output, this will reduce some of the conditional logic in the templates

Posted by ja...@apache.org.
DRILL-2870: Part1 - Separate count aggregate functions from others
Count is the only aggregate function with required type as output, this will reduce some of the conditional logic in the templates


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

Branch: refs/heads/merge_2015_05_09
Commit: ed5fa922ad22652388962b9fb5424dd8390b5036
Parents: d547699
Author: Mehant Baid <me...@gmail.com>
Authored: Sat May 9 12:31:36 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:30:12 2015 -0700

----------------------------------------------------------------------
 exec/java-exec/src/main/codegen/config.fmpp     |  3 +-
 .../src/main/codegen/data/AggrTypes1.tdd        | 29 -------
 .../src/main/codegen/data/CountAggrTypes.tdd    | 39 ++++++++++
 .../src/main/codegen/data/DecimalAggrTypes1.tdd | 13 +---
 .../templates/CountAggregateFunctions.java      | 80 ++++++++++++++++++++
 5 files changed, 122 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/ed5fa922/exec/java-exec/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index 8db120d..c70f6da 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -43,7 +43,8 @@ data: {
     extract:                  tdd(../data/ExtractTypes.tdd),
     sumzero:                  tdd(../data/SumZero.tdd),
     numericTypes:             tdd(../data/NumericTypes.tdd),
-    casthigh:                 tdd(../data/CastHigh.tdd)
+    casthigh:                 tdd(../data/CastHigh.tdd),
+    countAggrTypes:           tdd(../data/CountAggrTypes.tdd)
 }
 freemarkerLinks: {
     includes: includes/

http://git-wip-us.apache.org/repos/asf/drill/blob/ed5fa922/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index 8952417..1a98a07 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -88,35 +88,6 @@
       {inputType: "Interval", outputType: "Interval", runningType: "Interval", major: "Date", initialValue: "0"},
       {inputType: "NullableInterval", outputType: "Interval", runningType: "Interval", major: "Date", initialValue: "0"}
      ]
-   },
-   {className: "Count", funcName: "count", types: [
-      {inputType: "Bit", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Int", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "NullableBit", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "NullableInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Float4", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Float8", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "NullableFloat4", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "NullableFloat8", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Date", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableDate", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "TimeStamp", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableTimeStamp", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "Time", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableTime", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "IntervalDay", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableIntervalDay", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "IntervalYear", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableIntervalYear", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "Interval", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "NullableInterval", outputType: "BigInt", runningType: "BigInt", major: "Date", initialValue: "0"},
-      {inputType: "VarChar", outputType: "BigInt", runningType: "BigInt", major: "VarBytes"},
-      {inputType: "NullableVarChar", outputType: "BigInt", runningType: "BigInt", major: "VarBytes"},
-      {inputType: "VarBinary", outputType: "BigInt", runningType: "BigInt", major: "VarBytes"},
-      {inputType: "NullableVarBinary", outputType: "BigInt", runningType: "BigInt", major: "VarBytes"}
-     ]
    }
   ]
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/ed5fa922/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
new file mode 100644
index 0000000..53e25f7
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
@@ -0,0 +1,39 @@
+{
+  countFunctionsInput: [
+    "Bit",
+    "NullableBit",
+    "Int",
+    "NullableInt",
+    "BigInt",
+    "NullableBigInt",
+    "Float4",
+    "NullableFloat4",
+    "Float8",
+    "NullableFloat8",
+    "Date",
+    "NullableDate",
+    "TimeStamp",
+    "NullableTimeStamp",
+    "Time",
+    "NullableTime",
+    "IntervalDay",
+    "NullableIntervalDay",
+    "IntervalYear",
+    "NullableIntervalYear",
+    "Interval",
+    "NullableInterval",
+    "VarChar",
+    "NullableVarChar",
+    "VarBinary"
+    "NullableVarBinary",
+    "Decimal9",
+    "NullableDecimal9",
+    "Decimal18",
+    "NullableDecimal18",
+    "Decimal28Sparse",
+    "NullableDecimal28Sparse",
+    "Decimal38Sparse",
+    "NullableDecimal38Sparse"
+  ]
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/ed5fa922/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
index 5ac299c..92e93bb 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
@@ -37,18 +37,7 @@
       {inputType: "Decimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"},
       {inputType: "NullableDecimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"}
      ]
-   },
-      {className: "Count", funcName: "count", types: [
-         {inputType: "Decimal9", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "NullableDecimal9", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "Decimal18", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "NullableDecimal18", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "Decimal28Sparse", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "NullableDecimal28Sparse", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "Decimal38Sparse", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-         {inputType: "NullableDecimal38Sparse", outputType: "BigInt", runningType: "BigInt", major: "Numeric"}
-        ]
-      },
+   }
       {className: "Sum", funcName: "sum", types: [
         {inputType: "Decimal9", outputType: "Decimal38Sparse", major: "Numeric"},
         {inputType: "NullableDecimal9", outputType: "Decimal38Sparse", major: "Numeric"},

http://git-wip-us.apache.org/repos/asf/drill/blob/ed5fa922/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java b/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
new file mode 100644
index 0000000..71ac6a7
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.lang.Override;
+
+<@pp.dropOutputFile />
+
+<#-- A utility class that is used to generate java code for count aggregate functions -->
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/CountFunctions.java" />
+
+<#include "/@includes/license.ftl" />
+
+// Source code generated using FreeMarker template ${.template_name}
+
+package org.apache.drill.exec.expr.fn.impl.gaggr;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.*;
+
+@SuppressWarnings("unused")
+
+public class CountFunctions {
+  <#list countAggrTypes.countFunctionsInput as inputType>
+  @FunctionTemplate(name = "count", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class ${inputType}CountFunction implements DrillAggFunc {
+
+    @Param ${inputType}Holder in;
+    @Workspace BigIntHolder value;
+    @Output BigIntHolder out;
+
+    @Override
+    public void setup() {
+	    value = new BigIntHolder();
+      value.value = 0;
+    }
+
+    @Override
+    public void add() {
+      <#if inputType?starts_with("Nullable")>
+        if (in.isSet == 1) {
+          value.value++;
+        }
+      <#else>
+        value.value++;
+      </#if>
+    }
+
+    @Override
+    public void output() {
+      out.value = value.value;
+    }
+
+    @Override
+    public void reset() {
+      value.value = 0;
+    }
+  }
+  </#list>
+}
\ No newline at end of file


[05/12] drill git commit: DRILL-3005: Ensure that only one outcome can occur for a QueryResult.

Posted by ja...@apache.org.
DRILL-3005: Ensure that only one outcome can occur for a QueryResult.

Includes removing QueryResultHandler close listener which should fix most times.  For exceptional race cases, also added isTerminal condition which ensures only a single transition.


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

Branch: refs/heads/merge_2015_05_09
Commit: fb5e455c77b720bb2f350acf5cd535ffd612c671
Parents: a392e53
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat May 9 10:44:45 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:24:21 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/rpc/user/QueryResultHandler.java | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/fb5e455c/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 302be72..143d104 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
@@ -25,6 +25,7 @@ import io.netty.util.concurrent.GenericFutureListener;
 
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.UserRemoteException;
@@ -279,6 +280,7 @@ public class QueryResultHandler {
     private final RemoteConnection connection;
     private final ChannelFuture closeFuture;
     private final ChannelClosedListener closeListener;
+    private final AtomicBoolean isTerminal = new AtomicBoolean(false);
 
     public SubmissionListener(RemoteConnection connection, UserResultsListener resultsListener) {
       super();
@@ -302,11 +304,22 @@ public class QueryResultHandler {
 
     @Override
     public void failed(RpcException ex) {
+      if (!isTerminal.compareAndSet(false, true)) {
+        return;
+      }
+
+      closeFuture.removeListener(closeListener);
       resultsListener.submissionFailed(UserException.systemError(ex).build());
+
     }
 
     @Override
     public void success(QueryId queryId, ByteBuf buf) {
+      if (!isTerminal.compareAndSet(false, true)) {
+        return;
+      }
+
+      closeFuture.removeListener(closeListener);
       resultsListener.queryIdArrived(queryId);
       if (logger.isDebugEnabled()) {
         logger.debug("Received QueryId {} successfully. Adding results listener {}.",


[09/12] drill git commit: DRILL-2697: Pauses sites wait indefinitely for a resume signal DrillClient sends a resume signal to UserServer. UserServer triggers a resume call in the correct Foreman. Foreman resumes all pauses related to the query through th

Posted by ja...@apache.org.
DRILL-2697: Pauses sites wait indefinitely for a resume signal DrillClient sends a resume signal to UserServer. UserServer triggers a resume call in the correct Foreman. Foreman resumes all pauses related to the query through the Control layer.

+ Better error messages and more tests in TestDrillbitResilience and TestPauseInjection
+ Added execution controls to operator context
+ Removed ControlMessageHandler interface, renamed ControlHandlerImpl to ControlMessageHandler
+ Added CountDownLatchInjection, useful in cases like ParititionedSender that spawns multiple threads


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

Branch: refs/heads/merge_2015_05_09
Commit: 52dcd7e460e37031a08f6194cd597cc035d25a65
Parents: 87051d4
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Thu Apr 30 13:27:08 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:27:11 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/client/DrillClient.java   |  10 +-
 .../apache/drill/exec/ops/OperatorContext.java  |   3 +
 .../drill/exec/ops/OperatorContextImpl.java     |   8 +
 .../drill/exec/physical/impl/ScreenCreator.java |   6 +-
 .../exec/rpc/control/ControlRpcConfig.java      |   4 +-
 .../drill/exec/rpc/control/ControlTunnel.java   |  20 +-
 .../drill/exec/rpc/user/UserRpcConfig.java      |   3 +-
 .../apache/drill/exec/rpc/user/UserServer.java  |   9 +
 .../drill/exec/store/pojo/PojoRecordReader.java |  16 +-
 .../exec/testing/CountDownLatchInjection.java   |  51 ++++
 .../testing/CountDownLatchInjectionImpl.java    |  85 ++++++
 .../drill/exec/testing/ExecutionControls.java   |  37 ++-
 .../exec/testing/ExecutionControlsInjector.java |  13 +-
 .../apache/drill/exec/testing/Injection.java    |   6 +-
 .../exec/testing/NoOpControlsInjector.java      |  36 ++-
 .../drill/exec/testing/PauseInjection.java      |  33 +-
 .../org/apache/drill/exec/work/WorkManager.java |   3 +-
 .../exec/work/batch/ControlHandlerImpl.java     | 197 ------------
 .../exec/work/batch/ControlMessageHandler.java  | 195 +++++++++++-
 .../apache/drill/exec/work/foreman/Foreman.java |  25 +-
 .../drill/exec/work/foreman/QueryManager.java   |  44 ++-
 .../exec/work/fragment/FragmentExecutor.java    |  15 +-
 .../exec/work/fragment/FragmentManager.java     |   6 +
 .../work/fragment/NonRootFragmentManager.java   |   5 +
 .../exec/work/fragment/RootFragmentManager.java |   9 +-
 .../apache/drill/exec/work/user/UserWorker.java |   8 +
 .../exec/server/TestDrillbitResilience.java     | 303 +++++++++++--------
 .../testing/TestCountDownLatchInjection.java    | 155 ++++++++++
 .../drill/exec/testing/TestPauseInjection.java  | 146 ++++++++-
 .../org/apache/drill/exec/proto/BitControl.java |  48 ++-
 .../org/apache/drill/exec/proto/UserProtos.java |  46 ++-
 .../apache/drill/exec/proto/beans/RpcType.java  |   2 +
 protocol/src/main/protobuf/BitControl.proto     |  12 +-
 protocol/src/main/protobuf/User.proto           |   1 +
 34 files changed, 1106 insertions(+), 454 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 136d8c7..c642c4a 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
@@ -312,10 +312,18 @@ public class DrillClient implements Closeable, ConnectionThrottle {
   }
 
   public DrillRpcFuture<Ack> cancelQuery(QueryId id) {
-    logger.debug("Cancelling query {}", QueryIdHelper.getQueryId(id));
+    if(logger.isDebugEnabled()) {
+      logger.debug("Cancelling query {}", QueryIdHelper.getQueryId(id));
+    }
     return client.send(RpcType.CANCEL_QUERY, id, Ack.class);
   }
 
+  public DrillRpcFuture<Ack> resumeQuery(final QueryId queryId) {
+    if(logger.isDebugEnabled()) {
+      logger.debug("Resuming query {}", QueryIdHelper.getQueryId(queryId));
+    }
+    return client.send(RpcType.RESUME_PAUSED_QUERY, queryId, Ack.class);
+  }
 
   /**
    * Submits a Logical plan for direct execution (bypasses parsing)

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 7cc52ba..35139d5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.testing.ExecutionControls;
 
 public abstract class OperatorContext {
 
@@ -36,6 +37,8 @@ public abstract class OperatorContext {
 
   public abstract OperatorStats getStats();
 
+  public abstract ExecutionControls getExecutionControls();
+
   public static int getChildCount(PhysicalOperator popConfig) {
     Iterator<PhysicalOperator> iter = popConfig.iterator();
     int i = 0;

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index 6dbd880..9fa8867 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -24,11 +24,13 @@ import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 
 import com.carrotsearch.hppc.LongObjectOpenHashMap;
+import org.apache.drill.exec.testing.ExecutionControls;
 
 class OperatorContextImpl extends OperatorContext implements AutoCloseable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorContextImpl.class);
 
   private final BufferAllocator allocator;
+  private final ExecutionControls executionControls;
   private boolean closed = false;
   private PhysicalOperator popConfig;
   private OperatorStats stats;
@@ -42,6 +44,7 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
 
     OpProfileDef def = new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(), getChildCount(popConfig));
     this.stats = context.getStats().getOperatorStats(def, allocator);
+    executionControls = context.getExecutionControls();
   }
 
   public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats, boolean applyFragmentLimit) throws OutOfMemoryException {
@@ -49,6 +52,7 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
     this.popConfig = popConfig;
     this.stats     = stats;
+    executionControls = context.getExecutionControls();
   }
 
   public DrillBuf replace(DrillBuf old, int newSize) {
@@ -70,6 +74,10 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
     return newBuf;
   }
 
+  public ExecutionControls getExecutionControls() {
+    return executionControls;
+  }
+
   public BufferAllocator getAllocator() {
     if (allocator == null) {
       throw new UnsupportedOperationException("Operator context does not have an allocator");

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 c31de66..76dc91c 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
@@ -52,7 +52,6 @@ public class ScreenCreator implements RootCreator<Screen>{
 
 
   static class ScreenRoot extends BaseRootExec {
-//    private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
     private final RecordBatch incoming;
     private final FragmentContext context;
     private final AccountingUserConnection userConnection;
@@ -136,6 +135,11 @@ public class ScreenCreator implements RootCreator<Screen>{
     }
 
 
+    @Override
+    public void close() throws Exception {
+      injector.injectPause(context.getExecutionControls(), "send-complete", logger);
+      super.close();
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
index f92bb49..0cfa56e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlRpcConfig.java
@@ -40,16 +40,18 @@ public class ControlRpcConfig {
         .name("CONTROL")
         .timeout(config.getInt(ExecConstants.BIT_RPC_TIMEOUT))
         .add(RpcType.HANDSHAKE, BitControlHandshake.class, RpcType.HANDSHAKE, BitControlHandshake.class)
-        .add(RpcType.REQ_INIATILIZE_FRAGMENTS, InitializeFragments.class, RpcType.ACK, Ack.class)
+        .add(RpcType.REQ_INITIALIZE_FRAGMENTS, InitializeFragments.class, RpcType.ACK, Ack.class)
         .add(RpcType.REQ_CANCEL_FRAGMENT, FragmentHandle.class, RpcType.ACK, Ack.class)
         .add(RpcType.REQ_QUERY_CANCEL, QueryId.class, RpcType.ACK, Ack.class)
         .add(RpcType.REQ_RECEIVER_FINISHED, FinishedReceiver.class, RpcType.ACK, Ack.class)
         .add(RpcType.REQ_FRAGMENT_STATUS, FragmentStatus.class, RpcType.ACK, Ack.class)
         .add(RpcType.REQ_QUERY_STATUS, QueryId.class, RpcType.RESP_QUERY_STATUS, QueryProfile.class)
+        .add(RpcType.REQ_UNPAUSE_FRAGMENT, FragmentHandle.class, RpcType.ACK, Ack.class)
         .build();
   }
 
   public static int RPC_VERSION = 3;
 
   public static final Response OK = new Response(RpcType.ACK, Acks.OK);
+  public static final Response FAIL = new Response(RpcType.ACK, Acks.FAIL);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
index a4f9fdf..16b9b63 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
@@ -17,12 +17,9 @@
  */
 package org.apache.drill.exec.rpc.control;
 
-import java.util.Collection;
-
 import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.BitControl.InitializeFragments;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.BitControl.RpcType;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
@@ -56,7 +53,12 @@ public class ControlTunnel {
   }
 
   public void cancelFragment(RpcOutcomeListener<Ack> outcomeListener, FragmentHandle handle){
-    CancelFragment b = new CancelFragment(outcomeListener, handle);
+    final SignalFragment b = new SignalFragment(outcomeListener, handle, RpcType.REQ_CANCEL_FRAGMENT);
+    manager.runCommand(b);
+  }
+
+  public void resumeFragment(final RpcOutcomeListener<Ack> outcomeListener, final FragmentHandle handle) {
+    final SignalFragment b = new SignalFragment(outcomeListener, handle, RpcType.REQ_UNPAUSE_FRAGMENT);
     manager.runCommand(b);
   }
 
@@ -114,17 +116,19 @@ public class ControlTunnel {
     }
   }
 
-  public static class CancelFragment extends ListeningCommand<Ack, ControlConnection> {
+  public static class SignalFragment extends ListeningCommand<Ack, ControlConnection> {
     final FragmentHandle handle;
+    final RpcType type;
 
-    public CancelFragment(RpcOutcomeListener<Ack> listener, FragmentHandle handle) {
+    public SignalFragment(RpcOutcomeListener<Ack> listener, FragmentHandle handle, RpcType type) {
       super(listener);
       this.handle = handle;
+      this.type = type;
     }
 
     @Override
     public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, ControlConnection connection) {
-      connection.sendUnsafe(outcomeListener, RpcType.REQ_CANCEL_FRAGMENT, handle, Ack.class);
+      connection.sendUnsafe(outcomeListener, type, handle, Ack.class);
     }
 
   }
@@ -139,7 +143,7 @@ public class ControlTunnel {
 
     @Override
     public void doRpcCall(RpcOutcomeListener<Ack> outcomeListener, ControlConnection connection) {
-      connection.send(outcomeListener, RpcType.REQ_INIATILIZE_FRAGMENTS, fragments, Ack.class);
+      connection.send(outcomeListener, RpcType.REQ_INITIALIZE_FRAGMENTS, fragments, Ack.class);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 ae728d8..3f8122d 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
@@ -36,11 +36,12 @@ public class UserRpcConfig {
     return RpcConfig.newBuilder()
         .name("USER")
         .timeout(config.getInt(ExecConstants.USER_RPC_TIMEOUT))
-        .add(RpcType.HANDSHAKE, UserToBitHandshake.class, RpcType.HANDSHAKE, BitToUserHandshake.class) // user to bit.
+        .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
+        .add(RpcType.RESUME_PAUSED_QUERY, QueryId.class, RpcType.ACK, Ack.class) // user to bit
         .build();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 b3b7ae9..72b07ba 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
@@ -113,6 +113,15 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
         throw new RpcException("Failure while decoding QueryId body.", e);
       }
 
+    case RpcType.RESUME_PAUSED_QUERY_VALUE:
+      try {
+        final QueryId queryId = QueryId.PARSER.parseFrom(new ByteBufInputStream(pBody));
+        final Ack ack = worker.resumeQuery(queryId);
+        return new Response(RpcType.ACK, ack);
+      } catch (final InvalidProtocolBufferException e) {
+        throw new RpcException("Failure while decoding QueryId body.", e);
+      }
+
     default:
       throw new UnsupportedOperationException(String.format("UserServer received rpc of unknown type.  Type was %d.", rpcType));
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
index cf98b83..a893da1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
@@ -42,13 +42,16 @@ import org.apache.drill.exec.store.pojo.Writers.NDoubleWriter;
 import org.apache.drill.exec.store.pojo.Writers.NIntWriter;
 import org.apache.drill.exec.store.pojo.Writers.NTimeStampWriter;
 import org.apache.drill.exec.store.pojo.Writers.StringWriter;
+import org.apache.drill.exec.testing.ExecutionControlsInjector;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.google.common.collect.Lists;
 
 public class PojoRecordReader<T> extends AbstractRecordReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoRecordReader.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoRecordReader.class);
+  private static final ExecutionControlsInjector injector =
+    ExecutionControlsInjector.getInjector(PojoRecordReader.class);
 
   public final int forJsonIgnore = 1;
 
@@ -64,16 +67,9 @@ public class PojoRecordReader<T> extends AbstractRecordReader {
     this.iterator = iterator;
   }
 
-  public OperatorContext getOperatorContext() {
-    return operatorContext;
-  }
-
-  public void setOperatorContext(OperatorContext operatorContext) {
-    this.operatorContext = operatorContext;
-  }
-
   @Override
   public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+    operatorContext = context;
     try {
       Field[] fields = pojoClass.getDeclaredFields();
       List<PojoWriter> writers = Lists.newArrayList();
@@ -147,7 +143,7 @@ public class PojoRecordReader<T> extends AbstractRecordReader {
   @Override
   public int next() {
     boolean allocated = false;
-
+    injector.injectPause(operatorContext.getExecutionControls(), "read-next", logger);
     try {
       int i =0;
       outside:

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
new file mode 100644
index 0000000..de4a181
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjection.java
@@ -0,0 +1,51 @@
+/**
+ * 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.testing;
+
+/**
+ * This class is used internally for tracking injected countdown latches. These latches are specified via
+ * {@link org.apache.drill.exec.ExecConstants#DRILLBIT_CONTROL_INJECTIONS} session option.
+ *
+ * This injection is useful in the case where a thread spawns multiple threads. The parent thread initializes the latch
+ * with the expected number of countdown and awaits. The child threads count down on the same latch (same site class
+ * and same descriptor), and once there are enough, the parent thread continues.
+ */
+public interface CountDownLatchInjection {
+
+  /**
+   * Initializes the underlying latch
+   * @param count the number of times {@link #countDown} must be invoke before threads can pass through {@link #await}
+   */
+  void initialize(final int count);
+
+  /**
+   * Causes the current thread to wait until the latch has counted down to zero, unless the thread is
+   * {@link Thread#interrupt interrupted}.
+   */
+  void await() throws InterruptedException;
+
+  /**
+   * Await without interruption. In the case of interruption, log a warning and continue to wait.
+   */
+  void awaitUninterruptibly();
+
+  /**
+   * Decrements the count of the latch, releasing all waiting threads if the count reaches zero.
+   */
+  void countDown();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
new file mode 100644
index 0000000..f4012c1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/CountDownLatchInjectionImpl.java
@@ -0,0 +1,85 @@
+/**
+ * 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.testing;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.drill.common.concurrent.ExtendedLatch;
+
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * See {@link org.apache.drill.exec.testing.CountDownLatchInjection} Degenerates to
+ * {@link org.apache.drill.exec.testing.PauseInjection#pause}, if initialized to zero count. In any case, this injection
+ * provides more control than PauseInjection.
+ */
+@JsonAutoDetect(fieldVisibility = Visibility.ANY)
+public class CountDownLatchInjectionImpl extends Injection implements CountDownLatchInjection {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CountDownLatchInjectionImpl.class);
+
+  private ExtendedLatch latch = null;
+
+  @JsonCreator // ensures instances are created only through JSON
+  private CountDownLatchInjectionImpl(@JsonProperty("address") final String address,
+                                      @JsonProperty("port") final int port,
+                                      @JsonProperty("siteClass") final String siteClass,
+                                      @JsonProperty("desc") final String desc) throws InjectionConfigurationException {
+    super(address, port, siteClass, desc, 0, 1);
+  }
+
+  @Override
+  protected boolean injectNow() {
+    return true;
+  }
+
+  @Override
+  public void initialize(final int count) {
+    Preconditions.checkArgument(latch == null, "Latch can be initialized only once at %s in %s.", desc,
+      siteClass.getSimpleName());
+    Preconditions.checkArgument(count > 0, "Count has to be a positive integer at %s in %s.", desc,
+      siteClass.getSimpleName());
+    latch = new ExtendedLatch(count);
+  }
+
+  @Override
+  public void await() throws InterruptedException {
+    Preconditions.checkNotNull(latch, "Latch not initialized in %s at %s.", siteClass.getSimpleName(), desc);
+    try {
+      latch.await();
+    } catch (final InterruptedException e) {
+      logger.warn("Interrupted while awaiting in %s at %s.", siteClass.getSimpleName(), desc);
+      throw e;
+    }
+  }
+
+  @Override
+  public void awaitUninterruptibly() {
+    Preconditions.checkNotNull(latch, "Latch not initialized in %s at %s.", siteClass.getSimpleName(), desc);
+    latch.awaitUninterruptibly();
+  }
+
+  @Override
+  public void countDown() {
+    Preconditions.checkNotNull(latch, "Latch not initialized in %s at %s.", siteClass.getSimpleName(), desc);
+    Preconditions.checkArgument(latch.getCount() > 0, "Counting down on latch more than intended.");
+    latch.countDown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
index 1171bf8..639802f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
@@ -50,13 +50,15 @@ public final class ExecutionControls {
     controlsOptionMapper.addMixInAnnotations(Injection.class, InjectionMixIn.class);
   }
 
-  // Jackson MixIn for all types of injections
+  // Jackson MixIn: an annotated class that is used only by Jackson's ObjectMapper to allow a list of injections to
+  // hold various types of injections
   @JsonTypeInfo(
     use = JsonTypeInfo.Id.NAME,
     include = JsonTypeInfo.As.PROPERTY,
     property = "type")
   @JsonSubTypes({
     @Type(value = ExceptionInjection.class, name = "exception"),
+    @Type(value = CountDownLatchInjectionImpl.class, name = "latch"),
     @Type(value = PauseInjection.class, name = "pause")})
   public static abstract class InjectionMixIn {
   }
@@ -99,7 +101,7 @@ public final class ExecutionControls {
       final String jsonString = v.string_val;
       try {
         controlsOptionMapper.readValue(jsonString, Controls.class);
-      } catch (IOException e) {
+      } catch (final IOException e) {
         throw new ExpressionParsingException("Invalid control options string (" + jsonString + ").", e);
       }
     }
@@ -137,7 +139,7 @@ public final class ExecutionControls {
     final Controls controls;
     try {
       controls = controlsOptionMapper.readValue(opString, Controls.class);
-    } catch (IOException e) {
+    } catch (final IOException e) {
       // This never happens. opString must have been validated.
       logger.warn("Could not parse injections. Injections must have been validated before this point.");
       throw new DrillRuntimeException("Could not parse injections.", e);
@@ -153,7 +155,7 @@ public final class ExecutionControls {
   }
 
   /**
-   * Look for an exception injection matching the given injector, site description and endpoint.
+   * Look for an exception injection matching the given injector, site descriptor, and endpoint.
    *
    * @param injector the injector, which indicates a class
    * @param desc     the injection site description
@@ -165,7 +167,7 @@ public final class ExecutionControls {
   }
 
   /**
-   * Look for an pause injection matching the given injector, site description and endpoint.
+   * Look for an pause injection matching the given injector, site descriptor, and endpoint.
    *
    * @param injector the injector, which indicates a class
    * @param desc     the injection site description
@@ -176,6 +178,20 @@ public final class ExecutionControls {
     return injection != null ? (PauseInjection) injection : null;
   }
 
+  /**
+   * Look for a count down latch injection matching the given injector, site descriptor, and endpoint.
+   *
+   * @param injector the injector, which indicates a class
+   * @param desc     the injection site description
+   * @return the count down latch injection, if there is one for the injector, site and endpoint;
+   * otherwise, a latch that does nothing
+   */
+  public CountDownLatchInjection lookupCountDownLatchInjection(final ExecutionControlsInjector injector,
+                                                               final String desc) {
+    final Injection injection = lookupInjection(injector, desc);
+    return injection != null ? (CountDownLatchInjection) injection : NoOpControlsInjector.LATCH;
+  }
+
   private Injection lookupInjection(final ExecutionControlsInjector injector, final String desc) {
     if (controls.isEmpty()) {
       return null;
@@ -190,4 +206,15 @@ public final class ExecutionControls {
     // return only if injection was meant for this drillbit
     return injection.isValidForBit(endpoint) ? injection : null;
   }
+
+  /**
+   * This method resumes all pauses within the current context (QueryContext or FragmentContext).
+   */
+  public void unpauseAll() {
+    for (final Injection injection : controls.values()) {
+      if (injection instanceof PauseInjection) {
+        ((PauseInjection) injection).unpause();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
index 4b1cd0c..05f8433 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControlsInjector.java
@@ -75,12 +75,11 @@ public class ExecutionControlsInjector {
    * @param desc              the site description
    *                          throws the exception specified by the injection, if it is time
    */
-  public ExecutionControlsInjector injectUnchecked(final ExecutionControls executionControls, final String desc) {
+  public void injectUnchecked(final ExecutionControls executionControls, final String desc) {
     final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc);
     if (exceptionInjection != null) {
       exceptionInjection.throwUnchecked();
     }
-    return this;
   }
 
   /**
@@ -95,13 +94,12 @@ public class ExecutionControlsInjector {
    * @param exceptionClass    the expected class of the exception (or a super class of it)
    * @throws T the exception specified by the injection, if it is time
    */
-  public <T extends Throwable> ExecutionControlsInjector injectChecked(
+  public <T extends Throwable> void injectChecked(
     final ExecutionControls executionControls, final String desc, final Class<T> exceptionClass) throws T {
     final ExceptionInjection exceptionInjection = executionControls.lookupExceptionInjection(this, desc);
     if (exceptionInjection != null) {
       exceptionInjection.throwChecked(exceptionClass);
     }
-    return this;
   }
 
   /**
@@ -114,7 +112,7 @@ public class ExecutionControlsInjector {
    * @param desc              the site description
    * @param logger            logger of the class containing the injection site
    */
-  public ExecutionControlsInjector injectPause(final ExecutionControls executionControls, final String desc,
+  public void injectPause(final ExecutionControls executionControls, final String desc,
                                                final Logger logger) {
     final PauseInjection pauseInjection =
       executionControls.lookupPauseInjection(this, desc);
@@ -124,6 +122,9 @@ public class ExecutionControlsInjector {
       pauseInjection.pause();
       logger.debug("Resuming at {}", desc);
     }
-    return this;
+  }
+
+  public CountDownLatchInjection getLatch(final ExecutionControls executionControls, final String desc) {
+    return executionControls.lookupCountDownLatchInjection(this, desc);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/Injection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/Injection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/Injection.java
index 96fed3a..08ade51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/Injection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/Injection.java
@@ -28,8 +28,8 @@ public abstract class Injection {
 
   protected final String address;  // the address of the drillbit on which to inject
   protected final int port; // user port of the drillbit; useful when there are multiple drillbits on same machine
-  private final Class<?> siteClass; // the class where the injection should happen
-  private final String desc; // description of the injection site; useful for multiple exception injections in a single class
+  protected final Class<?> siteClass; // the class where the injection should happen
+  protected final String desc; // description of the injection site; useful for multiple exception injections in a single class
   private final AtomicInteger nSkip; // the number of times to skip the injection; starts >= 0
   private final AtomicInteger nFire;  // the number of times to do the injection, after any skips; starts > 0
 
@@ -64,7 +64,7 @@ public abstract class Injection {
    *
    * @return if the injection should be injected now
    */
-  protected final boolean injectNow() {
+  protected boolean injectNow() {
     return nSkip.decrementAndGet() < 0 && nFire.decrementAndGet() >= 0;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
index 80d9790..33ab783 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/NoOpControlsInjector.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.testing;
 import org.slf4j.Logger;
 
 /**
- * An injector that does not inject any controls.
+ * An injector that does not inject any controls, useful when not testing (i.e. assertions are not enabled).
  */
 public final class NoOpControlsInjector extends ExecutionControlsInjector {
 
@@ -29,20 +29,42 @@ public final class NoOpControlsInjector extends ExecutionControlsInjector {
   }
 
   @Override
-  public ExecutionControlsInjector injectUnchecked(final ExecutionControls executionControls, final String desc) {
-    return this;
+  public void injectUnchecked(final ExecutionControls executionControls, final String desc) {
   }
 
   @Override
-  public <T extends Throwable> ExecutionControlsInjector injectChecked(
+  public <T extends Throwable> void injectChecked(
     final ExecutionControls executionControls, final String desc, final Class<T> exceptionClass) throws T {
-    return this;
   }
 
   @Override
-  public ExecutionControlsInjector injectPause(final ExecutionControls executionControls, final String desc,
+  public void injectPause(final ExecutionControls executionControls, final String desc,
                                                final Logger logger) {
-    return this;
   }
 
+  /**
+   * When assertions are not enabled, this count down latch that does nothing is injected.
+   */
+  public static final CountDownLatchInjection LATCH = new CountDownLatchInjection() {
+    @Override
+    public void initialize(int count) {
+    }
+
+    @Override
+    public void await() {
+    }
+
+    @Override
+    public void awaitUninterruptibly() {
+    }
+
+    @Override
+    public void countDown() {
+    }
+  };
+
+  @Override
+  public CountDownLatchInjection getLatch(final ExecutionControls executionControls, final String desc) {
+    return LATCH;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
index e5f9c9c..ff0340b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/PauseInjection.java
@@ -21,43 +21,40 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 
 /**
- * Injection for a single pause. Specifies how long to pause. This class is used internally for tracking
- * injected pauses; these pauses are specified via
+ * Injection for a single pause. Pause indefinitely until signalled. This class is used internally for tracking
+ * injected pauses. Note that pauses can be fired only once; nFire field is ignored. These pauses are specified via
  * {@link org.apache.drill.exec.ExecConstants#DRILLBIT_CONTROL_INJECTIONS} session option.
  *
- * TODO(DRILL-2697): Pause indefinitely until signalled, rather than for a specified time.
+ * After the pauses are set, the user sends another signal to unpause all the pauses. This triggers the Foreman to
+ * 1) unpause all pauses in QueryContext, and
+ * 2) send an unpause signal to all fragments, each of which unpauses all pauses in FragmentContext.
  */
 @JsonAutoDetect(fieldVisibility = Visibility.ANY)
 public class PauseInjection extends Injection {
 
-  private final long millis;
+  private final ExtendedLatch latch = new ExtendedLatch(1);
 
   @JsonCreator // ensures instances are created only through JSON
   private PauseInjection(@JsonProperty("address") final String address,
                          @JsonProperty("port") final int port,
                          @JsonProperty("siteClass") final String siteClass,
                          @JsonProperty("desc") final String desc,
-                         @JsonProperty("nSkip") final int nSkip,
-                         @JsonProperty("nFire") final int nFire,
-                         @JsonProperty("millis") final long millis) throws InjectionConfigurationException {
-    super(address, port, siteClass, desc, nSkip, nFire);
-    if (millis <= 0) {
-      throw new InjectionConfigurationException("Pause millis is non-positive.");
-    }
-    this.millis = millis;
+                         @JsonProperty("nSkip") final int nSkip) throws InjectionConfigurationException {
+    super(address, port, siteClass, desc, nSkip, 1);
   }
 
   public void pause() {
-    if (! injectNow()) {
+    if (!injectNow()) {
       return;
     }
-    try {
-      Thread.sleep(millis);
-    } catch (InterruptedException e) {
-      throw new DrillRuntimeException("Well, I should be sleeping.");
-    }
+    latch.awaitUninterruptibly();
+  }
+
+  public void unpause() {
+    latch.countDown();
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 3e4f3d1..f2352e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -45,7 +45,6 @@ import org.apache.drill.exec.rpc.data.DataResponseHandlerImpl;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.sys.PStoreProvider;
-import org.apache.drill.exec.work.batch.ControlHandlerImpl;
 import org.apache.drill.exec.work.batch.ControlMessageHandler;
 import org.apache.drill.exec.work.foreman.Foreman;
 import org.apache.drill.exec.work.foreman.QueryManager;
@@ -116,7 +115,7 @@ public class WorkManager implements AutoCloseable {
 
 
     // TODO references to this escape here (via WorkerBee) before construction is done
-    controlMessageWorker = new ControlHandlerImpl(bee); // TODO getFragmentRunner(), getForemanForQueryId()
+    controlMessageWorker = new ControlMessageHandler(bee); // TODO getFragmentRunner(), getForemanForQueryId()
     userWorker = new UserWorker(bee); // TODO should just be an interface? addNewForeman(), getForemanForQueryId()
     statusThread = new StatusThread();
     dataHandler = new DataResponseHandlerImpl(bee); // TODO only uses startFragmentPendingRemote()

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java
deleted file mode 100644
index b6c6852..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlHandlerImpl.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.work.batch;
-
-import static org.apache.drill.exec.rpc.RpcBus.get;
-import io.netty.buffer.ByteBuf;
-
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
-import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.BitControl.InitializeFragments;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
-import org.apache.drill.exec.proto.BitControl.RpcType;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
-import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.rpc.Acks;
-import org.apache.drill.exec.rpc.Response;
-import org.apache.drill.exec.rpc.RpcConstants;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.UserRpcException;
-import org.apache.drill.exec.rpc.control.ControlConnection;
-import org.apache.drill.exec.rpc.control.ControlTunnel;
-import org.apache.drill.exec.rpc.data.DataRpcConfig;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.work.WorkManager.WorkerBee;
-import org.apache.drill.exec.work.foreman.Foreman;
-import org.apache.drill.exec.work.fragment.FragmentExecutor;
-import org.apache.drill.exec.work.fragment.FragmentManager;
-import org.apache.drill.exec.work.fragment.NonRootFragmentManager;
-import org.apache.drill.exec.work.fragment.NonRootStatusReporter;
-
-public class ControlHandlerImpl implements ControlMessageHandler {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlHandlerImpl.class);
-  private final WorkerBee bee;
-
-  public ControlHandlerImpl(final WorkerBee bee) {
-    this.bee = bee;
-  }
-
-  @Override
-  public Response handle(final ControlConnection connection, final int rpcType,
-      final ByteBuf pBody, final ByteBuf dBody) throws RpcException {
-    if (RpcConstants.EXTRA_DEBUGGING) {
-      logger.debug("Received bit com message of type {}", rpcType);
-    }
-
-    switch (rpcType) {
-
-    case RpcType.REQ_CANCEL_FRAGMENT_VALUE: {
-      final FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
-      cancelFragment(handle);
-      return DataRpcConfig.OK;
-    }
-
-    case RpcType.REQ_RECEIVER_FINISHED_VALUE: {
-      final FinishedReceiver finishedReceiver = get(pBody, FinishedReceiver.PARSER);
-      receivingFragmentFinished(finishedReceiver);
-      return DataRpcConfig.OK;
-    }
-
-    case RpcType.REQ_FRAGMENT_STATUS_VALUE:
-      bee.getContext().getWorkBus().statusUpdate( get(pBody, FragmentStatus.PARSER));
-      // TODO: Support a type of message that has no response.
-      return DataRpcConfig.OK;
-
-    case RpcType.REQ_QUERY_CANCEL_VALUE: {
-      final QueryId queryId = get(pBody, QueryId.PARSER);
-      final Foreman foreman = bee.getForemanForQueryId(queryId);
-      if (foreman != null) {
-        foreman.cancel();
-        return DataRpcConfig.OK;
-      } else {
-        return DataRpcConfig.FAIL;
-      }
-    }
-
-    case RpcType.REQ_INIATILIZE_FRAGMENTS_VALUE: {
-      final InitializeFragments fragments = get(pBody, InitializeFragments.PARSER);
-      for(int i = 0; i < fragments.getFragmentCount(); i++) {
-        startNewRemoteFragment(fragments.getFragment(i));
-      }
-      return DataRpcConfig.OK;
-    }
-
-    case RpcType.REQ_QUERY_STATUS_VALUE: {
-      final QueryId queryId = get(pBody, QueryId.PARSER);
-      final Foreman foreman = bee.getForemanForQueryId(queryId);
-      if (foreman == null) {
-        throw new RpcException("Query not running on node.");
-      }
-      final QueryProfile profile = foreman.getQueryManager().getQueryProfile();
-      return new Response(RpcType.RESP_QUERY_STATUS, profile);
-    }
-
-    default:
-      throw new RpcException("Not yet supported.");
-    }
-  }
-
-  @Override
-  public void startNewRemoteFragment(final PlanFragment fragment) throws UserRpcException {
-    logger.debug("Received remote fragment start instruction", fragment);
-
-    final DrillbitContext drillbitContext = bee.getContext();
-    try {
-      // we either need to start the fragment if it is a leaf fragment, or set up a fragment manager if it is non leaf.
-      if (fragment.getLeafFragment()) {
-        final FragmentContext context = new FragmentContext(drillbitContext, fragment,
-            drillbitContext.getFunctionImplementationRegistry());
-        final ControlTunnel tunnel = drillbitContext.getController().getTunnel(fragment.getForeman());
-        final NonRootStatusReporter listener = new NonRootStatusReporter(context, tunnel);
-        final FragmentRoot rootOperator = drillbitContext.getPlanReader().readFragmentOperator(
-            fragment.getFragmentJson());
-        final FragmentExecutor fr = new FragmentExecutor(context, rootOperator, listener);
-        bee.addFragmentRunner(fr);
-      } else {
-        // isIntermediate, store for incoming data.
-        final NonRootFragmentManager manager = new NonRootFragmentManager(fragment, drillbitContext);
-        drillbitContext.getWorkBus().addFragmentManager(manager);
-      }
-
-    } catch (final Exception e) {
-        throw new UserRpcException(drillbitContext.getEndpoint(),
-            "Failure while trying to start remote fragment", e);
-    } catch (final OutOfMemoryError t) {
-      if (t.getMessage().startsWith("Direct buffer")) {
-        throw new UserRpcException(drillbitContext.getEndpoint(),
-            "Out of direct memory while trying to start remote fragment", t);
-      } else {
-        throw t;
-      }
-    }
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
-   */
-  @Override
-  public Ack cancelFragment(final FragmentHandle handle) {
-    final FragmentManager manager = bee.getContext().getWorkBus().getFragmentManagerIfExists(handle);
-    if (manager != null) {
-      // try remote fragment cancel.
-      manager.cancel();
-    } else {
-      // then try local cancel.
-      final FragmentExecutor runner = bee.getFragmentRunner(handle);
-      if (runner != null) {
-        runner.cancel();
-      }
-    }
-
-    return Acks.OK;
-  }
-
-  private Ack receivingFragmentFinished(final FinishedReceiver finishedReceiver) {
-
-    final FragmentManager manager =
-        bee.getContext().getWorkBus().getFragmentManagerIfExists(finishedReceiver.getSender());
-
-    FragmentExecutor executor;
-    if (manager != null) {
-      manager.receivingFragmentFinished(finishedReceiver.getReceiver());
-    } else {
-      // then try local cancel.
-      executor = bee.getFragmentRunner(finishedReceiver.getSender());
-      if (executor != null) {
-        executor.receivingFragmentFinished(finishedReceiver.getReceiver());
-      } else {
-        logger.warn(
-            "Dropping request for early fragment termination for path {} -> {} as path to executor unavailable.",
-            QueryIdHelper.getFragmentId(finishedReceiver.getSender()),
-            QueryIdHelper.getFragmentId(finishedReceiver.getReceiver()));
-      }
-    }
-
-    return Acks.OK;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
index c5d78cc..d12e6d5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/ControlMessageHandler.java
@@ -17,24 +17,207 @@
  */
 package org.apache.drill.exec.work.batch;
 
+import static org.apache.drill.exec.rpc.RpcBus.get;
 import io.netty.buffer.ByteBuf;
 
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.InitializeFragments;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.RpcType;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
+import org.apache.drill.exec.rpc.Acks;
 import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConstants;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.UserRpcException;
 import org.apache.drill.exec.rpc.control.ControlConnection;
+import org.apache.drill.exec.rpc.control.ControlRpcConfig;
+import org.apache.drill.exec.rpc.control.ControlTunnel;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.foreman.Foreman;
+import org.apache.drill.exec.work.fragment.FragmentExecutor;
+import org.apache.drill.exec.work.fragment.FragmentManager;
+import org.apache.drill.exec.work.fragment.NonRootFragmentManager;
+import org.apache.drill.exec.work.fragment.NonRootStatusReporter;
 
-public interface ControlMessageHandler {
+public class ControlMessageHandler {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlMessageHandler.class);
+  private final WorkerBee bee;
 
-  public abstract Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
-      throws RpcException;
+  public ControlMessageHandler(final WorkerBee bee) {
+    this.bee = bee;
+  }
 
-  public abstract void startNewRemoteFragment(PlanFragment fragment) throws UserRpcException;
+  public Response handle(final ControlConnection connection, final int rpcType,
+      final ByteBuf pBody, final ByteBuf dBody) throws RpcException {
+    if (RpcConstants.EXTRA_DEBUGGING) {
+      logger.debug("Received bit com message of type {}", rpcType);
+    }
 
-  public abstract Ack cancelFragment(FragmentHandle handle);
+    switch (rpcType) {
 
+    case RpcType.REQ_CANCEL_FRAGMENT_VALUE: {
+      final FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
+      cancelFragment(handle);
+      return ControlRpcConfig.OK;
+    }
 
-}
\ No newline at end of file
+    case RpcType.REQ_RECEIVER_FINISHED_VALUE: {
+      final FinishedReceiver finishedReceiver = get(pBody, FinishedReceiver.PARSER);
+      receivingFragmentFinished(finishedReceiver);
+      return ControlRpcConfig.OK;
+    }
+
+    case RpcType.REQ_FRAGMENT_STATUS_VALUE:
+      bee.getContext().getWorkBus().statusUpdate( get(pBody, FragmentStatus.PARSER));
+      // TODO: Support a type of message that has no response.
+      return ControlRpcConfig.OK;
+
+    case RpcType.REQ_QUERY_CANCEL_VALUE: {
+      final QueryId queryId = get(pBody, QueryId.PARSER);
+      final Foreman foreman = bee.getForemanForQueryId(queryId);
+      if (foreman != null) {
+        foreman.cancel();
+        return ControlRpcConfig.OK;
+      } else {
+        return ControlRpcConfig.FAIL;
+      }
+    }
+
+    case RpcType.REQ_INITIALIZE_FRAGMENTS_VALUE: {
+      final InitializeFragments fragments = get(pBody, InitializeFragments.PARSER);
+      for(int i = 0; i < fragments.getFragmentCount(); i++) {
+        startNewRemoteFragment(fragments.getFragment(i));
+      }
+      return ControlRpcConfig.OK;
+    }
+
+    case RpcType.REQ_QUERY_STATUS_VALUE: {
+      final QueryId queryId = get(pBody, QueryId.PARSER);
+      final Foreman foreman = bee.getForemanForQueryId(queryId);
+      if (foreman == null) {
+        throw new RpcException("Query not running on node.");
+      }
+      final QueryProfile profile = foreman.getQueryManager().getQueryProfile();
+      return new Response(RpcType.RESP_QUERY_STATUS, profile);
+    }
+
+    case RpcType.REQ_UNPAUSE_FRAGMENT_VALUE: {
+      final FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
+      resumeFragment(handle);
+      return ControlRpcConfig.OK;
+    }
+
+    default:
+      throw new RpcException("Not yet supported.");
+    }
+  }
+
+  private void startNewRemoteFragment(final PlanFragment fragment) throws UserRpcException {
+    logger.debug("Received remote fragment start instruction", fragment);
+
+    final DrillbitContext drillbitContext = bee.getContext();
+    try {
+      // we either need to start the fragment if it is a leaf fragment, or set up a fragment manager if it is non leaf.
+      if (fragment.getLeafFragment()) {
+        final FragmentContext context = new FragmentContext(drillbitContext, fragment,
+            drillbitContext.getFunctionImplementationRegistry());
+        final ControlTunnel tunnel = drillbitContext.getController().getTunnel(fragment.getForeman());
+        final NonRootStatusReporter listener = new NonRootStatusReporter(context, tunnel);
+        final FragmentRoot rootOperator = drillbitContext.getPlanReader().readFragmentOperator(
+            fragment.getFragmentJson());
+        final FragmentExecutor fr = new FragmentExecutor(context, rootOperator, listener);
+        bee.addFragmentRunner(fr);
+      } else {
+        // isIntermediate, store for incoming data.
+        final NonRootFragmentManager manager = new NonRootFragmentManager(fragment, drillbitContext);
+        drillbitContext.getWorkBus().addFragmentManager(manager);
+      }
+
+    } catch (final Exception e) {
+        throw new UserRpcException(drillbitContext.getEndpoint(),
+            "Failure while trying to start remote fragment", e);
+    } catch (final OutOfMemoryError t) {
+      if (t.getMessage().startsWith("Direct buffer")) {
+        throw new UserRpcException(drillbitContext.getEndpoint(),
+            "Out of direct memory while trying to start remote fragment", t);
+      } else {
+        throw t;
+      }
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
+   */
+  private Ack cancelFragment(final FragmentHandle handle) {
+    // cancel a pending fragment
+    final FragmentManager manager = bee.getContext().getWorkBus().getFragmentManagerIfExists(handle);
+    if (manager != null) {
+      manager.cancel();
+      return Acks.OK;
+    }
+
+    // cancel a running fragment
+    final FragmentExecutor runner = bee.getFragmentRunner(handle);
+    if (runner != null) {
+      runner.cancel();
+      return Acks.OK;
+    }
+
+    // fragment completed or does not exist
+    logger.warn("Dropping request to cancel fragment. {} does not exist.", QueryIdHelper.getFragmentId(handle));
+    return Acks.OK;
+  }
+
+  private Ack resumeFragment(final FragmentHandle handle) {
+    // resume a pending fragment
+    final FragmentManager manager = bee.getContext().getWorkBus().getFragmentManagerIfExists(handle);
+    if (manager != null) {
+      manager.unpause();
+      return Acks.OK;
+    }
+
+    // resume a paused fragment
+    final FragmentExecutor runner = bee.getFragmentRunner(handle);
+    if (runner != null) {
+      runner.unpause();
+      return Acks.OK;
+    }
+
+    // fragment completed or does not exist
+    logger.warn("Dropping request to resume fragment. {} does not exist.", QueryIdHelper.getFragmentId(handle));
+    return Acks.OK;
+  }
+
+  private Ack receivingFragmentFinished(final FinishedReceiver finishedReceiver) {
+
+    final FragmentManager manager =
+        bee.getContext().getWorkBus().getFragmentManagerIfExists(finishedReceiver.getSender());
+
+    FragmentExecutor executor;
+    if (manager != null) {
+      manager.receivingFragmentFinished(finishedReceiver.getReceiver());
+    } else {
+      executor = bee.getFragmentRunner(finishedReceiver.getSender());
+      if (executor != null) {
+        executor.receivingFragmentFinished(finishedReceiver.getReceiver());
+      } else {
+        logger.warn(
+            "Dropping request for early fragment termination for path {} -> {} as path to executor unavailable.",
+            QueryIdHelper.getFragmentId(finishedReceiver.getSender()),
+            QueryIdHelper.getFragmentId(finishedReceiver.getReceiver()));
+      }
+    }
+
+    return Acks.OK;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 4d403b8..0122ef8 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
@@ -119,6 +119,7 @@ public class Foreman implements Runnable {
   private final DrillbitContext drillbitContext;
   private final UserClientConnection initiatingClient; // used to send responses
   private volatile QueryState state;
+  private boolean resume = false;
 
   private volatile DistributedLease lease; // used to limit the number of concurrent queries
 
@@ -196,6 +197,18 @@ public class Foreman implements Runnable {
   }
 
   /**
+   * Resume the query. Regardless of the current state, this method sends a resume signal to all fragments.
+   * This method can be called multiple times.
+   */
+  public void resume() {
+    resume = true;
+    // resume all pauses through query context
+    queryContext.getExecutionControls().unpauseAll();
+    // resume all pauses through all fragment contexts
+    queryManager.unpauseExecutingFragments(drillbitContext, rootRunner);
+  }
+
+  /**
    * Called by execution pool to do query setup, and kick off remote execution.
    *
    * <p>Note that completion of this function is not the end of the Foreman's role
@@ -268,9 +281,20 @@ public class Foreman implements Runnable {
        * If we do throw an exception during setup, and have already moved to QueryState.FAILED, we just need to
        * make sure that we can't make things any worse as those events are delivered, but allow
        * any necessary remaining cleanup to proceed.
+       *
+       * Note that cancellations cannot be simulated before this point, i.e. pauses can be injected, because Foreman
+       * would wait on the cancelling thread to signal a resume and the cancelling thread would wait on the Foreman
+       * to accept events.
        */
       acceptExternalEvents.countDown();
 
+      // If we received the resume signal before fragments are setup, the first call does not actually resume the
+      // fragments. Since setup is done, all fragments must have been delivered to remote nodes. Now we can resume.
+      if(resume) {
+        resume();
+      }
+      injector.injectPause(queryContext.getExecutionControls(), "foreman-ready", logger);
+
       // restore the thread's original name
       currentThread.setName(originalName);
     }
@@ -375,7 +399,6 @@ public class Foreman implements Runnable {
     drillbitContext.getClusterCoordinator().addDrillbitStatusListener(queryManager.getDrillbitStatusListener());
 
     logger.debug("Submitting fragments to run.");
-    injector.injectPause(queryContext.getExecutionControls(), "pause-run-plan", logger);
 
     // set up the root fragment first so we'll have incoming buffers available.
     setupRootFragment(rootPlanFragment, work.getRootOperator());

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index c4646bd..090a377 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -189,7 +189,8 @@ public class QueryManager {
           final DrillbitEndpoint endpoint = data.getEndpoint();
           final FragmentHandle handle = data.getHandle();
           // TODO is the CancelListener redundant? Does the FragmentStatusListener get notified of the same?
-          controller.getTunnel(endpoint).cancelFragment(new CancelListener(endpoint, handle), handle);
+          controller.getTunnel(endpoint).cancelFragment(new SignalListener(endpoint, handle,
+            SignalListener.Signal.CANCEL), handle);
         }
         break;
 
@@ -203,25 +204,52 @@ public class QueryManager {
     }
   }
 
+  /**
+   * Sends a resume signal to all fragments, regardless of their state, since the fragment might have paused before
+   * sending any message. Resume the root fragment directly and all other (local and remote) fragments through the
+   * control tunnel.
+   */
+  void unpauseExecutingFragments(final DrillbitContext drillbitContext, final FragmentExecutor rootRunner) {
+    if (rootRunner != null) {
+      rootRunner.unpause();
+    }
+    final Controller controller = drillbitContext.getController();
+    for(final FragmentData data : fragmentDataSet) {
+      final DrillbitEndpoint endpoint = data.getEndpoint();
+      final FragmentHandle handle = data.getHandle();
+      controller.getTunnel(endpoint).resumeFragment(new SignalListener(endpoint, handle,
+        SignalListener.Signal.UNPAUSE), handle);
+    }
+  }
+
   /*
    * This assumes that the FragmentStatusListener implementation takes action when it hears
-   * that the target fragment has been cancelled. As a result, this listener doesn't do anything
+   * that the target fragment has acknowledged the signal. As a result, this listener doesn't do anything
    * but log messages.
    */
-  private class CancelListener extends EndpointListener<Ack, FragmentHandle> {
-    public CancelListener(final DrillbitEndpoint endpoint, final FragmentHandle handle) {
+  private static class SignalListener extends EndpointListener<Ack, FragmentHandle> {
+    /**
+     * An enum of possible signals that {@link SignalListener} listens to.
+     */
+    public static enum Signal { CANCEL, UNPAUSE }
+
+    private final Signal signal;
+
+    public SignalListener(final DrillbitEndpoint endpoint, final FragmentHandle handle, final Signal signal) {
       super(endpoint, handle);
+      this.signal = signal;
     }
 
     @Override
     public void failed(final RpcException ex) {
-      logger.error("Failure while attempting to cancel fragment {} on endpoint {}.", value, endpoint, ex);
+      logger.error("Failure while attempting to {} fragment {} on endpoint {} with {}.", signal, value, endpoint, ex);
     }
 
     @Override
-    public void success(final Ack value, final ByteBuf buf) {
-      if (!value.getOk()) {
-        logger.warn("Remote node {} responded negative on cancellation request for fragment {}.", endpoint, value);
+    public void success(final Ack ack, final ByteBuf buf) {
+      if (!ack.getOk()) {
+        logger.warn("Remote node {} responded negative on {} request for fragment {} with {}.", endpoint, signal, value,
+          ack);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index 7baafc4..24e2556 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -108,7 +108,7 @@ public class FragmentExecutor implements Runnable {
 
   /**
    * Cancel the execution of this fragment is in an appropriate state. Messages come from external.
-   * NOTE that this can be called from threads *other* than the one running this runnable(),
+   * NOTE that this will be called from threads *other* than the one running this runnable(),
    * so we need to be careful about the state transitions that can result.
    */
   public void cancel() {
@@ -140,11 +140,18 @@ public class FragmentExecutor implements Runnable {
   }
 
   /**
+   * Resume all the pauses within the current context. Note that this method will be called from threads *other* than
+   * the one running this runnable(). Also, this method can be called multiple times.
+   */
+  public synchronized void unpause() {
+    fragmentContext.getExecutionControls().unpauseAll();
+  }
+
+  /**
    * Inform this fragment that one of its downstream partners no longer needs additional records. This is most commonly
    * called in the case that a limit query is executed.
    *
-   * @param handle
-   *          The downstream FragmentHandle of the Fragment that needs no more records from this Fragment.
+   * @param handle The downstream FragmentHandle of the Fragment that needs no more records from this Fragment.
    */
   public void receivingFragmentFinished(final FragmentHandle handle) {
     acceptExternalEvents.awaitUninterruptibly();
@@ -277,6 +284,8 @@ public class FragmentExecutor implements Runnable {
 
     // first close the operators and release all memory.
     try {
+      // Say executor was cancelled before setup. Now when executor actually runs, root is not initialized, but this
+      // method is called in finally. So root can be null.
       if (root != null) {
         root.close();
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
index 0ba91b4..ad880da 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
@@ -49,6 +49,12 @@ public interface FragmentManager {
 
   public abstract void cancel();
 
+  /**
+   * If the executor is paused (for testing), this method should unpause the executor. This method should handle
+   * multiple calls.
+   */
+  public abstract void unpause();
+
   public boolean isWaiting();
 
   public abstract FragmentHandle getHandle();

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
index f526fbe..ca5d5b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
@@ -112,6 +112,11 @@ public class NonRootFragmentManager implements FragmentManager {
   }
 
   @Override
+  public void unpause() {
+    runner.unpause();
+  }
+
+  @Override
   public FragmentHandle getHandle() {
     return fragment.getHandle();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
index b1c3fe0..67ef9b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
@@ -29,8 +29,8 @@ import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
 
 // TODO a lot of this is the same as NonRootFragmentManager
-public class RootFragmentManager implements FragmentManager{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootFragmentManager.class);
+public class RootFragmentManager implements FragmentManager {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootFragmentManager.class);
 
   private final IncomingBuffers buffers;
   private final FragmentExecutor runner;
@@ -70,6 +70,11 @@ public class RootFragmentManager implements FragmentManager{
   }
 
   @Override
+  public void unpause() {
+    runner.unpause();
+  }
+
+  @Override
   public boolean isWaiting() {
     return !buffers.isDone() && !cancel;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
index 8854ef3..e8deb4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
@@ -68,6 +68,14 @@ public class UserWorker{
     return Acks.OK;
   }
 
+  public Ack resumeQuery(final QueryId queryId) {
+    final Foreman foreman = bee.getForemanForQueryId(queryId);
+    if (foreman != null) {
+      foreman.resume();
+    }
+    return Acks.OK;
+  }
+
   public OptionManager getSystemOptions() {
     return bee.getContext().getOptionManager();
   }


[06/12] drill git commit: DRILL-2723: Add option to change text format row size estimate.

Posted by ja...@apache.org.
DRILL-2723: Add option to change text format row size estimate.


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

Branch: refs/heads/merge_2015_05_09
Commit: 5de7d6e9059f5877b4bd368ab12022e47a74baa6
Parents: fb5e455
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat May 9 11:20:47 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:24:21 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |  3 +++
 .../exec/physical/base/AbstractGroupScan.java   | 13 +++++++++-
 .../drill/exec/physical/base/GroupScan.java     |  6 ++---
 .../exec/planner/logical/DrillScanRel.java      | 25 +++++++++++--------
 .../physical/ConvertCountToDirectScan.java      | 25 ++++++++++---------
 .../drill/exec/planner/physical/PrelUtil.java   |  6 ++++-
 .../drill/exec/planner/physical/ScanPrel.java   | 10 +++++---
 .../server/options/SystemOptionManager.java     |  1 +
 .../exec/store/dfs/easy/EasyFormatPlugin.java   | 14 +++++++++++
 .../exec/store/dfs/easy/EasyGroupScan.java      | 26 ++++++++++++--------
 .../exec/store/easy/text/TextFormatPlugin.java  | 19 ++++++++++++--
 11 files changed, 105 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/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 97d5770..1a10aa2 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
@@ -26,6 +26,7 @@ import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.PositiveLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.PowerOfTwoLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
+import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.testing.ExecutionControls;
 
@@ -116,6 +117,8 @@ public interface ExecConstants {
   public static String JSON_ALL_TEXT_MODE = "store.json.all_text_mode";
   public static BooleanValidator JSON_READER_ALL_TEXT_MODE_VALIDATOR = new BooleanValidator(JSON_ALL_TEXT_MODE, false);
   public static final BooleanValidator JSON_EXTENDED_TYPES = new BooleanValidator("store.json.extended_types", true);
+  public static final DoubleValidator TEXT_ESTIMATED_ROW_SIZE = new RangeDoubleValidator(
+      "store.text.estimated_row_size_bytes", 1, Long.MAX_VALUE, 100.0);
 
   /**
    * The column label (for directory levels) in results when querying files in a directory

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index 242bd5e..5c4ee4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -22,10 +22,11 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.google.common.collect.Iterators;
-import org.apache.drill.exec.physical.EndpointAffinity;
 
 public abstract class AbstractGroupScan extends AbstractBase implements GroupScan {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractGroupScan.class);
@@ -70,6 +71,16 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
   }
 
   @Override
+  public ScanStats getScanStats(PlannerSettings settings) {
+    return getScanStats();
+  }
+
+  @JsonIgnore
+  public ScanStats getScanStats() {
+    throw new UnsupportedOperationException("This should be implemented.");
+  }
+
+  @Override
   @JsonIgnore
   public boolean enforceWidth() {
     return false;

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index 60b8330..2d16cd0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -19,14 +19,14 @@ package org.apache.drill.exec.physical.base;
 
 import java.util.List;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
 
 /**
  * A GroupScan operator represents all data which will be scanned by a given physical
@@ -68,7 +68,7 @@ public interface GroupScan extends Scan, HasAffinity{
   public abstract String getDigest();
 
   @JsonIgnore
-  public ScanStats getScanStats();
+  public ScanStats getScanStats(PlannerSettings settings);
 
   /**
    * Returns a clone of GroupScan instance, except that the new GroupScan will use the provided list of columns .

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
index a724e23..9a009ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
@@ -29,6 +29,7 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.apache.calcite.rel.RelWriter;
@@ -52,19 +53,22 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
   final private RelDataType rowType;
   private GroupScan groupScan;
   private List<SchemaPath> columns;
+  private PlannerSettings settings;
 
   /** Creates a DrillScan. */
-  public DrillScanRel(RelOptCluster cluster, RelTraitSet traits,
-      RelOptTable table) {
+  public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
+      final RelOptTable table) {
     // By default, scan does not support project pushdown.
     // Decision whether push projects into scan will be made solely in DrillPushProjIntoScanRule.
     this(cluster, traits, table, table.getRowType(), GroupScan.ALL_COLUMNS);
+    this.settings = PrelUtil.getPlannerSettings(cluster.getPlanner());
   }
 
   /** Creates a DrillScan. */
-  public DrillScanRel(RelOptCluster cluster, RelTraitSet traits,
-      RelOptTable table, RelDataType rowType, List<SchemaPath> columns) {
+  public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
+      final RelOptTable table, final RelDataType rowType, final List<SchemaPath> columns) {
     super(DRILL_LOGICAL, cluster, traits, table);
+    this.settings = PrelUtil.getPlannerSettings(cluster.getPlanner());
     this.rowType = rowType;
     if (columns == null) { // planner asks to scan all of the columns
       this.columns =  ColumnList.all();
@@ -75,18 +79,19 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
     }
     try {
       this.groupScan = drillTable.getGroupScan().clone(this.columns);
-    } catch (IOException e) {
+    } catch (final IOException e) {
       throw new DrillRuntimeException("Failure creating scan.", e);
     }
   }
 
   /** Creates a DrillScanRel for a particular GroupScan */
-  public DrillScanRel(RelOptCluster cluster, RelTraitSet traits,
-      RelOptTable table, GroupScan groupScan, RelDataType rowType, List<SchemaPath> columns) {
+  public DrillScanRel(final RelOptCluster cluster, final RelTraitSet traits,
+      final RelOptTable table, final GroupScan groupScan, final RelDataType rowType, final List<SchemaPath> columns) {
     super(DRILL_LOGICAL, cluster, traits, table);
     this.rowType = rowType;
     this.columns = columns;
     this.groupScan = groupScan;
+    this.settings = PrelUtil.getPlannerSettings(cluster.getPlanner());
   }
 
 //
@@ -128,15 +133,15 @@ public class DrillScanRel extends DrillScanRelBase implements DrillRel {
 
   @Override
   public double getRows() {
-    return this.groupScan.getScanStats().getRecordCount();
+    return this.groupScan.getScanStats(settings).getRecordCount();
   }
 
   /// TODO: this method is same as the one for ScanPrel...eventually we should consolidate
   /// this and few other methods in a common base class which would be extended
   /// by both logical and physical rels.
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    ScanStats stats = groupScan.getScanStats();
+  public RelOptCost computeSelfCost(final RelOptPlanner planner) {
+    final ScanStats stats = groupScan.getScanStats(settings);
     int columnCount = getRowType().getFieldCount();
     double ioCost = 0;
     boolean isStarQuery = Iterables.tryFind(getRowType().getFieldNames(), new Predicate<String>() {

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
index 8b71b70..3b820b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ConvertCountToDirectScan.java
@@ -21,18 +21,10 @@ package org.apache.drill.exec.planner.physical;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.planner.logical.DrillAggregateRel;
-import org.apache.drill.exec.planner.logical.DrillProjectRel;
-import org.apache.drill.exec.planner.logical.DrillScanRel;
-import org.apache.drill.exec.planner.logical.RelOptHelper;
-import org.apache.drill.exec.store.direct.DirectGroupScan;
-import org.apache.drill.exec.store.pojo.PojoRecordReader;
-import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -41,6 +33,14 @@ import org.apache.calcite.rel.type.RelRecordType;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.store.direct.DirectGroupScan;
+import org.apache.drill.exec.store.pojo.PojoRecordReader;
 
 import com.google.common.collect.Lists;
 
@@ -88,14 +88,15 @@ public class ConvertCountToDirectScan extends Prule {
     final DrillScanRel scan = (DrillScanRel) call.rel(call.rels.length -1);
     final DrillProjectRel proj = call.rels.length == 3 ? (DrillProjectRel) call.rel(1) : null;
 
-    GroupScan oldGrpScan = scan.getGroupScan();
+    final GroupScan oldGrpScan = scan.getGroupScan();
+    final PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
 
     // Only apply the rule when :
     //    1) scan knows the exact row count in getSize() call,
     //    2) No GroupBY key,
     //    3) only one agg function (Check if it's count(*) below).
     //    4) No distinct agg call.
-    if (! (oldGrpScan.getScanStats().getGroupScanProperty().hasExactRowCount()
+    if (!(oldGrpScan.getScanStats(settings).getGroupScanProperty().hasExactRowCount()
         && agg.getGroupCount() == 0
         && agg.getAggCallList().size() == 1
         && !agg.containsDistinctCall())) {
@@ -112,7 +113,7 @@ public class ConvertCountToDirectScan extends Prule {
       if (aggCall.getArgList().isEmpty() ||
           (aggCall.getArgList().size() == 1 &&
            ! agg.getInput().getRowType().getFieldList().get(aggCall.getArgList().get(0).intValue()).getType().isNullable())) {
-        cnt = (long) oldGrpScan.getScanStats().getRecordCount();
+        cnt = (long) oldGrpScan.getScanStats(settings).getRecordCount();
       } else if (aggCall.getArgList().size() == 1) {
       // count(columnName) ==> Agg ( Scan )) ==> columnValueCount
         int index = aggCall.getArgList().get(0);

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index 44d828b..d63bf54 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -82,7 +82,11 @@ public class PrelUtil {
     return (Iterator<Prel>) (Object) nodes.iterator();
   }
 
-  public static PlannerSettings getSettings(RelOptCluster cluster) {
+  public static PlannerSettings getSettings(final RelOptCluster cluster) {
+    return getPlannerSettings(cluster);
+  }
+
+  public static PlannerSettings getPlannerSettings(final RelOptCluster cluster) {
     return cluster.getPlanner().getContext().unwrap(PlannerSettings.class);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index da558b2..d48e7cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -101,13 +101,15 @@ public class ScanPrel extends AbstractRelNode implements DrillScanPrel {
 
   @Override
   public double getRows() {
-    return this.groupScan.getScanStats().getRecordCount();
+    final PlannerSettings settings = PrelUtil.getPlannerSettings(getCluster());
+    return this.groupScan.getScanStats(settings).getRecordCount();
   }
 
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    ScanStats stats = this.groupScan.getScanStats();
-    int columnCount = this.getRowType().getFieldCount();
+  public RelOptCost computeSelfCost(final RelOptPlanner planner) {
+    final PlannerSettings settings = PrelUtil.getPlannerSettings(planner);
+    final ScanStats stats = this.groupScan.getScanStats(settings);
+    final int columnCount = this.getRowType().getFieldCount();
 
     if(PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
       return planner.getCostFactory().makeCost(stats.getRecordCount() * columnCount, stats.getCpuCost(), stats.getDiskCost());

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index e9ee68b..e268e64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -81,6 +81,7 @@ public class SystemOptionManager extends BaseOptionManager {
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,
+      ExecConstants.TEXT_ESTIMATED_ROW_SIZE,
       ExecConstants.JSON_EXTENDED_TYPES,
       ExecConstants.JSON_READ_NUMBERS_AS_DOUBLE_VALIDATOR,
       ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL_VALIDATOR,

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 762760a..d1b46e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -34,9 +34,12 @@ import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.physical.impl.WriterRecordBatch;
 import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractRecordReader;
@@ -48,6 +51,7 @@ import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
+import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.hadoop.conf.Configuration;
 
 import com.google.common.collect.ImmutableSet;
@@ -200,6 +204,16 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     }
   }
 
+  protected ScanStats getScanStats(final PlannerSettings settings, final EasyGroupScan scan) {
+    long data = 0;
+    for (final CompleteFileWork work : scan.getWorkIterable()) {
+      data += work.getTotalBytes();
+    }
+
+    final long estRowCount = data / 1024;
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, estRowCount, 1, data);
+  }
+
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location) throws IOException {
     return new EasyWriter(child, location, this);

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index 5f9e02b..a559beb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.dfs.easy;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -30,7 +31,7 @@ import org.apache.drill.exec.physical.base.FileGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -48,6 +49,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 
@@ -102,7 +104,17 @@ public class EasyGroupScan extends AbstractFileGroupScan{
     initFromSelection(selection, formatPlugin);
   }
 
-  private EasyGroupScan(EasyGroupScan that) {
+  @JsonIgnore
+  public Iterable<CompleteFileWork> getWorkIterable() {
+    return new Iterable<CompleteFileWork>() {
+      @Override
+      public Iterator<CompleteFileWork> iterator() {
+        return Iterators.unmodifiableIterator(chunks.iterator());
+      }
+    };
+  }
+
+  private EasyGroupScan(final EasyGroupScan that) {
     super(that.getUserName());
     selection = that.selection;
     formatPlugin = that.formatPlugin;
@@ -134,14 +146,8 @@ public class EasyGroupScan extends AbstractFileGroupScan{
 
 
   @Override
-  public ScanStats getScanStats() {
-    long data =0;
-    for (CompleteFileWork work : chunks) {
-      data += work.getTotalBytes();
-    }
-
-    long estRowCount = data/1024;
-    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, estRowCount, 1, data);
+  public ScanStats getScanStats(final PlannerSettings settings) {
+    return formatPlugin.getScanStats(settings, this);
   }
 
   @JsonProperty("files")

http://git-wip-us.apache.org/repos/asf/drill/blob/5de7d6e9/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index 5756a6a..967d920 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -29,6 +29,9 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -43,6 +46,7 @@ import org.apache.drill.exec.store.dfs.easy.EasyWriter;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.exec.store.easy.text.compliant.CompliantTextRecordReader;
 import org.apache.drill.exec.store.easy.text.compliant.TextParsingSettings;
+import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.drill.exec.store.text.DrillTextRecordReader;
 import org.apache.drill.exec.store.text.DrillTextRecordWriter;
 import org.apache.hadoop.conf.Configuration;
@@ -95,8 +99,19 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
   }
 
   @Override
-  public RecordWriter getRecordWriter(FragmentContext context, EasyWriter writer) throws IOException {
-    Map<String, String> options = Maps.newHashMap();
+  protected ScanStats getScanStats(final PlannerSettings settings, final EasyGroupScan scan) {
+    long data = 0;
+    for (final CompleteFileWork work : scan.getWorkIterable()) {
+      data += work.getTotalBytes();
+    }
+    final double estimatedRowSize = settings.getOptions().getOption(ExecConstants.TEXT_ESTIMATED_ROW_SIZE);
+    final double estRowCount = data / estimatedRowSize;
+    return new ScanStats(GroupScanProperty.NO_EXACT_ROW_COUNT, (long) estRowCount, 1, data);
+  }
+
+  @Override
+  public RecordWriter getRecordWriter(final FragmentContext context, final EasyWriter writer) throws IOException {
+    final Map<String, String> options = Maps.newHashMap();
 
     options.put("location", writer.getLocation());
 


[04/12] drill git commit: DRILL-2981: Add queries log. Update profile to store normal and verbose exception as well as node and errorid.

Posted by ja...@apache.org.
DRILL-2981: Add queries log.  Update profile to store normal and verbose exception as well as node and errorid.


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

Branch: refs/heads/merge_2015_05_09
Commit: 42d5f818a5501dbd05808c53959db86e66202792
Parents: 5de7d6e
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri May 8 08:16:27 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:24:21 2015 -0700

----------------------------------------------------------------------
 .../drill/common/exceptions/UserException.java  |  29 +-
 .../common/exceptions/UserExceptionContext.java |  23 +-
 distribution/src/resources/drillbit.sh          |   3 +
 distribution/src/resources/logback.xml          |  24 +
 distribution/src/resources/runbit               |   2 +-
 distribution/src/resources/sqlline              |   3 +-
 distribution/src/resources/sqlline.bat          |   2 +-
 distribution/src/resources/submit_plan          |   2 +-
 .../server/rest/profile/ProfileResources.java   |  14 +-
 .../server/rest/profile/ProfileWrapper.java     |   4 +
 .../apache/drill/exec/work/foreman/Foreman.java |  68 +-
 .../drill/exec/work/foreman/LoggedQuery.java    |  80 ++
 .../drill/exec/work/foreman/QueryManager.java   |  39 +-
 .../src/main/resources/rest/profile/list.ftl    |  13 +
 .../src/main/resources/rest/profile/profile.ftl |  25 +
 exec/java-exec/src/test/resources/logback.xml   |   5 +
 .../drill/exec/proto/SchemaUserBitShared.java   |  35 +
 .../apache/drill/exec/proto/UserBitShared.java  | 962 +++++++++++++++++--
 .../drill/exec/proto/beans/QueryInfo.java       |   6 +-
 .../drill/exec/proto/beans/QueryProfile.java    | 112 +++
 protocol/src/main/protobuf/UserBitShared.proto  |   7 +-
 21 files changed, 1324 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
index d90ace1..21859ed 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserException.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.exceptions;
 
 import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 
 /**
@@ -531,7 +532,11 @@ public class UserException extends DrillRuntimeException {
    */
   @Override
   public String getMessage() {
-    return generateMessage();
+    return generateMessage(true);
+  }
+
+  public String getMessage(boolean includeErrorIdAndIdentity) {
+    return generateMessage(includeErrorIdAndIdentity);
   }
 
   /**
@@ -548,7 +553,11 @@ public class UserException extends DrillRuntimeException {
    * @return verbose error message
    */
   public String getVerboseMessage() {
-    return generateMessage() + "\n\n" + ErrorHelper.buildCausesMessage(getCause());
+    return getVerboseMessage(true);
+  }
+
+  public String getVerboseMessage(boolean includeErrorIdAndIdentity) {
+    return generateMessage(includeErrorIdAndIdentity) + "\n\n" + ErrorHelper.buildCausesMessage(getCause());
   }
 
   /**
@@ -575,6 +584,18 @@ public class UserException extends DrillRuntimeException {
     return builder.build();
   }
 
+  public String getErrorId() {
+    return context.getErrorId();
+  }
+
+  public String getErrorLocation() {
+    DrillbitEndpoint ep = context.getEndpoint();
+    if (ep != null) {
+      return ep.getAddress() + ":" + ep.getUserPort();
+    } else {
+      return null;
+    }
+  }
   /**
    * Generates a user error message that has the following structure:
    * ERROR TYPE: ERROR_MESSAGE
@@ -583,9 +604,9 @@ public class UserException extends DrillRuntimeException {
    *
    * @return generated user error message
    */
-  private String generateMessage() {
+  private String generateMessage(boolean includeErrorIdAndIdentity) {
     return errorType + " ERROR: " + super.getMessage() + "\n\n" +
-      context.generateContextMessage();
+        context.generateContextMessage(includeErrorIdAndIdentity);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java b/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
index 7acf81d..962973b 100644
--- a/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
+++ b/common/src/main/java/org/apache/drill/common/exceptions/UserExceptionContext.java
@@ -133,24 +133,25 @@ class UserExceptionContext {
    * generate a context message
    * @return string containing all context information concatenated
    */
-  String generateContextMessage() {
+  String generateContextMessage(boolean includeErrorIdAndIdentity) {
     StringBuilder sb = new StringBuilder();
 
     for (String context : contextList) {
       sb.append(context).append("\n");
     }
 
-    // add identification infos
-    sb.append("\n[Error Id: ");
-    sb.append(errorId).append(" ");
-    if(endpoint != null) {
-      sb.append("on ")
-        .append(endpoint.getAddress())
-        .append(":")
-        .append(endpoint.getUserPort());
+    if (includeErrorIdAndIdentity) {
+      // add identification infos
+      sb.append("\n[Error Id: ");
+      sb.append(errorId).append(" ");
+      if (endpoint != null) {
+        sb.append("on ")
+            .append(endpoint.getAddress())
+            .append(":")
+            .append(endpoint.getUserPort());
+      }
+      sb.append("]");
     }
-    sb.append("]");
-
     return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/drillbit.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drillbit.sh b/distribution/src/resources/drillbit.sh
index 212d5d8..10a501b 100755
--- a/distribution/src/resources/drillbit.sh
+++ b/distribution/src/resources/drillbit.sh
@@ -147,12 +147,15 @@ JAVA=$JAVA_HOME/bin/java
 export DRILL_LOG_PREFIX=drillbit
 export DRILL_LOGFILE=$DRILL_LOG_PREFIX.log
 export DRILL_OUTFILE=$DRILL_LOG_PREFIX.out
+export DRILL_QUERYFILE=$DRILL_LOG_PREFIX_queries.json
 loggc=$DRILL_LOG_DIR/$DRILL_LOG_PREFIX.gc
 loglog="${DRILL_LOG_DIR}/${DRILL_LOGFILE}"
 logout="${DRILL_LOG_DIR}/${DRILL_OUTFILE}"
+logqueries="${DRILL_LOG_DIR}/${DRILL_QUERYFILE}"
 pid=$DRILL_PID_DIR/drillbit.pid
 
 export DRILLBIT_LOG_PATH=$loglog
+export DRILLBIT_QUERY_LOG_PATH=$logqueries
 
 if [ -n "$SERVER_GC_OPTS" ]; then
   export SERVER_GC_OPTS=${SERVER_GC_OPTS/"-Xloggc:<FILE-PATH>"/"-Xloggc:${loggc}"}

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/logback.xml
----------------------------------------------------------------------
diff --git a/distribution/src/resources/logback.xml b/distribution/src/resources/logback.xml
index 29b0e37..350383a 100644
--- a/distribution/src/resources/logback.xml
+++ b/distribution/src/resources/logback.xml
@@ -32,6 +32,23 @@
     </encoder>
   </appender>
 
+    <appender name="QUERY" class="ch.qos.logback.core.rolling.RollingFileAppender">
+      <file>${log.query.path}</file>
+      <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
+        <fileNamePattern>${log.query.path}.%i</fileNamePattern>
+        <minIndex>1</minIndex>
+        <maxIndex>10</maxIndex>
+      </rollingPolicy>
+
+      <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
+        <maxFileSize>100MB</maxFileSize>
+      </triggeringPolicy>
+      <encoder>
+        <pattern>%msg</pattern>
+      </encoder>
+    </appender>
+    
+    
     <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
       <file>${log.path}</file>
       <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
@@ -48,11 +65,18 @@
       </encoder>
     </appender>
 
+
   <logger name="org.apache.drill" additivity="false">
     <level value="info" />
     <appender-ref ref="FILE" />
   </logger>
 
+  <logger name="query.logger" additivity="false">
+    <level value="info" />
+    <appender-ref ref="QUERY" />
+    <!--     <appender-ref ref="SOCKET" /> -->
+  </logger>
+
   <!-- 
   <logger name="org.apache.drill" additivity="false">
     <level value="debug" />

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/runbit
----------------------------------------------------------------------
diff --git a/distribution/src/resources/runbit b/distribution/src/resources/runbit
index a8d8de8..8c54b1a 100755
--- a/distribution/src/resources/runbit
+++ b/distribution/src/resources/runbit
@@ -22,4 +22,4 @@ bin=`cd "$bin">/dev/null; pwd`
 
 DRILL_ALL_JAVA_OPTS="$DRILLBIT_JAVA_OPTS $DRILL_JAVA_OPTS $SERVER_GC_OPTS"
 
-exec $JAVA -Dlog.path=$DRILLBIT_LOG_PATH $DRILL_ALL_JAVA_OPTS -cp $CP org.apache.drill.exec.server.Drillbit
+exec $JAVA -Dlog.path=$DRILLBIT_LOG_PATH -Dlog.query.path=$DRILLBIT_QUERY_LOG_PATH $DRILL_ALL_JAVA_OPTS -cp $CP org.apache.drill.exec.server.Drillbit

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/sqlline
----------------------------------------------------------------------
diff --git a/distribution/src/resources/sqlline b/distribution/src/resources/sqlline
index ee74057..b1b9014 100644
--- a/distribution/src/resources/sqlline
+++ b/distribution/src/resources/sqlline
@@ -41,8 +41,7 @@ bin=`cd "$bin">/dev/null; pwd`
 # so that it can still be overridden via DRILL_SHELL_JAVA_OPTS.
 DRILL_SHELL_JAVA_OPTS="-Dsqlline.isolation=TRANSACTION_NONE $DRILL_SHELL_JAVA_OPTS"
 
-DRILL_SHELL_JAVA_OPTS="$DRILL_SHELL_JAVA_OPTS -Dlog.path=$DRILL_LOG_DIR/sqlline.log"
-
+DRILL_SHELL_JAVA_OPTS="$DRILL_SHELL_JAVA_OPTS -Dlog.path=$DRILL_LOG_DIR/sqlline.log -Dlog.query.path=$DRILL_LOG_DIR/sqlline_queries.json"
 
 if [ -n "$QUERY" ] ; then
   echo $QUERY | exec "$JAVA" $DRILL_SHELL_JAVA_OPTS $DRILL_JAVA_OPTS -cp $CP sqlline.SqlLine -d org.apache.drill.jdbc.Driver  --maxWidth=10000 "${ARGS[@]}"

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/sqlline.bat
----------------------------------------------------------------------
diff --git a/distribution/src/resources/sqlline.bat b/distribution/src/resources/sqlline.bat
index 2564b8c..3451b0e 100755
--- a/distribution/src/resources/sqlline.bat
+++ b/distribution/src/resources/sqlline.bat
@@ -176,7 +176,7 @@ rem Put our property specification before previous value of DRILL_SHELL_JAVA_OPT
 rem so that it can still be overridden via DRILL_SHELL_JAVA_OPTS.
 set DRILL_SHELL_JAVA_OPTS=-Dsqlline.isolation=TRANSACTION_NONE %DRILL_SHELL_JAVA_OPTS%
 
-set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% -Dlog.path="%DRILL_LOG_DIR%\sqlline.log"
+set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% -Dlog.path="%DRILL_LOG_DIR%\sqlline.log" -Dlog.query.path="%DRILL_LOG_DIR%\sqlline_queries.log"
 
 SET JAVA_CMD=%JAVA_HOME%\bin\%JAVA_EXE%
 if "%JAVA_HOME%" == "" (set JAVA_CMD=%JAVA_EXE%)

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/distribution/src/resources/submit_plan
----------------------------------------------------------------------
diff --git a/distribution/src/resources/submit_plan b/distribution/src/resources/submit_plan
index 4ed0662..0a58d8c 100755
--- a/distribution/src/resources/submit_plan
+++ b/distribution/src/resources/submit_plan
@@ -20,6 +20,6 @@ bin=`cd "$bin">/dev/null; pwd`
 
 . "$bin"/drill-config.sh
 
-DRILL_SHELL_JAVA_OPTS="$DRILL_SHELL_JAVA_OPTS -Dlog.path=$DRILL_LOG_DIR/submitter.log"
+DRILL_SHELL_JAVA_OPTS="$DRILL_SHELL_JAVA_OPTS -Dlog.path=$DRILL_LOG_DIR/submitter.log -Dlog.query.path=$DRILL_LOG_DIR/submitter_queries.json"
 
 exec $JAVA $DRILL_SHELL_JAVA_OPTS $DRILL_JAVA_OPTS -cp $CP org.apache.drill.exec.client.QuerySubmitter "$@"

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
index c83acad..6656bf6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
@@ -63,14 +63,20 @@ public class ProfileResources {
     private String foreman;
     private String query;
     private String state;
+    private String user;
 
-    public ProfileInfo(String queryId, long time, String foreman, String query, String state) {
+    public ProfileInfo(String queryId, long time, String foreman, String query, String state, String user) {
       this.queryId = queryId;
       this.time = new Date(time);
       this.foreman = foreman;
       this.location = "http://localhost:8047/profile/" + queryId + ".json";
       this.query = query = query.substring(0,  Math.min(query.length(), 150));
       this.state = state;
+      this.user = user;
+    }
+
+    public String getUser() {
+      return user;
     }
 
     public String getQuery(){
@@ -146,7 +152,8 @@ public class ProfileResources {
 
     for (Map.Entry<String, QueryInfo> entry : running) {
       QueryInfo profile = entry.getValue();
-      runningQueries.add(new ProfileInfo(entry.getKey(), profile.getStart(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name()));
+      runningQueries.add(new ProfileInfo(entry.getKey(), profile.getStart(), profile.getForeman().getAddress(), profile
+          .getQuery(), profile.getState().name(), profile.getUser()));
     }
 
     Collections.sort(runningQueries, Collections.reverseOrder());
@@ -155,7 +162,8 @@ public class ProfileResources {
     List<ProfileInfo> finishedQueries = Lists.newArrayList();
     for (Map.Entry<String, QueryProfile> entry : completed) {
       QueryProfile profile = entry.getValue();
-      finishedQueries.add(new ProfileInfo(entry.getKey(), profile.getStart(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name()));
+      finishedQueries.add(new ProfileInfo(entry.getKey(), profile.getStart(), profile.getForeman().getAddress(),
+          profile.getQuery(), profile.getState().name(), profile.getUser()));
     }
 
     return new QProfiles(runningQueries, finishedQueries);

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
index b34bb3a..dd26a76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
@@ -43,6 +43,10 @@ public class ProfileWrapper {
     this.id = QueryIdHelper.getQueryId(profile.getId());
   }
 
+  public boolean hasError() {
+    return profile.hasError() && profile.getError() != null;
+  }
+
   public QueryProfile getProfile() {
     return profile;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/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 7b36e21..4d403b8 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
@@ -24,6 +24,7 @@ import io.netty.util.concurrent.GenericFutureListener;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Date;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
@@ -105,6 +106,8 @@ import com.google.common.collect.Sets;
  */
 public class Foreman implements Runnable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
+  private static final org.slf4j.Logger queryLogger = org.slf4j.LoggerFactory.getLogger("query.logger");
+  private static final ObjectMapper MAPPER = new ObjectMapper();
   private final static ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(Foreman.class);
   private static final int RPC_WAIT_IN_SECONDS = 90;
 
@@ -129,6 +132,7 @@ public class Foreman implements Runnable {
   private final ConnectionClosedListener closeListener = new ConnectionClosedListener();
   private final ChannelFuture closeFuture;
 
+  private String queryText;
 
   /**
    * Constructor. Sets up the Foreman, but does not initiate any execution.
@@ -209,6 +213,8 @@ public class Foreman implements Runnable {
 
     try {
       injector.injectChecked(queryContext.getExecutionControls(), "run-try-beginning", ForemanException.class);
+      queryText = queryRequest.getPlan();
+
       // convert a run query request into action
       switch (queryRequest.getType()) {
       case LOGICAL:
@@ -435,19 +441,22 @@ public class Foreman implements Runnable {
       }
 
       final long queueTimeout = optionManager.getOption(ExecConstants.QUEUE_TIMEOUT);
+      final String queueName;
 
       try {
         @SuppressWarnings("resource")
         final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator();
-        DistributedSemaphore distributedSemaphore;
+        final DistributedSemaphore distributedSemaphore;
 
         // get the appropriate semaphore
         if (totalCost > queueThreshold) {
           final int largeQueue = (int) optionManager.getOption(ExecConstants.LARGE_QUEUE_SIZE);
           distributedSemaphore = clusterCoordinator.getSemaphore("query.large", largeQueue);
+          queueName = "large";
         } else {
           final int smallQueue = (int) optionManager.getOption(ExecConstants.SMALL_QUEUE_SIZE);
           distributedSemaphore = clusterCoordinator.getSemaphore("query.small", smallQueue);
+          queueName = "small";
         }
 
 
@@ -459,14 +468,19 @@ public class Foreman implements Runnable {
       if (lease == null) {
         throw UserException
             .resourceError()
-            .message("Unable to acquire queue resources for query within timeout.  Timeout was set at %d seconds.",
-                queueTimeout / 1000)
+            .message(
+                "Unable to acquire queue resources for query within timeout.  Timeout for %s queue was set at %d seconds.",
+                queueTimeout / 1000, queueName)
             .build();
       }
 
     }
   }
 
+  Exception getCurrentException() {
+    return foremanResult.getException();
+  }
+
   private QueryWorkUnit getQueryWorkUnit(final PhysicalPlan plan) throws ExecutionSetupException {
     final PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next();
     final Fragment rootFragment = rootOperator.accept(MakeFragmentsVisitor.INSTANCE, null);
@@ -536,7 +550,7 @@ public class Foreman implements Runnable {
    */
   private class ForemanResult implements AutoCloseable {
     private QueryState resultState = null;
-    private Exception resultException = null;
+    private volatile Exception resultException = null;
     private boolean isClosed = false;
 
     /**
@@ -589,11 +603,20 @@ public class Foreman implements Runnable {
     }
 
     /**
-     * Close the given resource, catching and adding any caught exceptions via
-     * {@link #addException(Exception)}. If an exception is caught, it will change
-     * the result state to FAILED, regardless of what its current value.
+     * Expose the current exception (if it exists). This is useful for secondary reporting to the query profile.
+     *
+     * @return the current Foreman result exception or null.
+     */
+    public Exception getException() {
+      return resultException;
+    }
+
+    /**
+     * Close the given resource, catching and adding any caught exceptions via {@link #addException(Exception)}. If an
+     * exception is caught, it will change the result state to FAILED, regardless of what its current value.
      *
-     * @param autoCloseable the resource to close
+     * @param autoCloseable
+     *          the resource to close
      */
     private void suppressingClose(final AutoCloseable autoCloseable) {
       Preconditions.checkState(!isClosed);
@@ -615,6 +638,22 @@ public class Foreman implements Runnable {
       }
     }
 
+    private void logQuerySummary() {
+      try {
+        LoggedQuery q = new LoggedQuery(
+            QueryIdHelper.getQueryId(queryId),
+            queryContext.getQueryContextInfo().getDefaultSchemaName(),
+            queryText,
+            new Date(queryContext.getQueryContextInfo().getQueryStartTime()),
+            new Date(System.currentTimeMillis()),
+            state,
+            queryContext.getSession().getCredentials().getUserName());
+        queryLogger.info(MAPPER.writeValueAsString(q));
+      } catch (Exception e) {
+        logger.error("Failure while recording query information to query log.", e);
+      }
+    }
+
     @Override
     public void close() {
       Preconditions.checkState(!isClosed);
@@ -626,6 +665,9 @@ public class Foreman implements Runnable {
       // remove the channel disconnected listener (doesn't throw)
       closeFuture.removeListener(closeListener);
 
+      // log the query summary
+      logQuerySummary();
+
       // These are straight forward removals from maps, so they won't throw.
       drillbitContext.getWorkBus().removeFragmentStatusListener(queryId);
       drillbitContext.getClusterCoordinator().removeDrillbitStatusListener(queryManager.getDrillbitStatusListener());
@@ -653,12 +695,18 @@ public class Foreman implements Runnable {
       final QueryResult.Builder resultBuilder = QueryResult.newBuilder()
           .setQueryId(queryId)
           .setQueryState(resultState);
+      final UserException uex;
       if (resultException != null) {
         final boolean verbose = queryContext.getOptions().getOption(ExecConstants.ENABLE_VERBOSE_ERRORS_KEY).bool_val;
-        final UserException uex = UserException.systemError(resultException).addIdentity(queryContext.getCurrentEndpoint()).build();
+        uex = UserException.systemError(resultException).addIdentity(queryContext.getCurrentEndpoint()).build();
         resultBuilder.addError(uex.getOrCreatePBError(verbose));
+      } else {
+        uex = null;
       }
 
+      // we store the final result here so we can capture any error/errorId in the profile for later debugging.
+      queryManager.writeFinalProfile(uex);
+
       /*
        * If sending the result fails, we don't really have any way to modify the result we tried to send;
        * it is possible it got sent but the result came from a later part of the code path. It is also
@@ -802,7 +850,7 @@ public class Foreman implements Runnable {
 
   private void recordNewState(final QueryState newState) {
     state = newState;
-    queryManager.updateQueryStateInStore(newState);
+    queryManager.updateEphemeralState(newState);
   }
 
   private void runSQL(final String sql) throws ExecutionSetupException {

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/LoggedQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/LoggedQuery.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/LoggedQuery.java
new file mode 100644
index 0000000..f856d53
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/LoggedQuery.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.work.foreman;
+
+import java.util.Date;
+
+import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class LoggedQuery {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LoggedQuery.class);
+
+  private final String queryId;
+  private final String schema;
+  private final String queryText;
+  private final Date start;
+  private final Date finish;
+  private final QueryState outcome;
+  private final String username;
+
+  public LoggedQuery(String queryId, String schema, String queryText, Date start, Date finish, QueryState outcome,
+      String username) {
+    super();
+    this.queryId = queryId;
+    this.schema = schema;
+    this.queryText = queryText;
+    this.start = start;
+    this.finish = finish;
+    this.outcome = outcome;
+    this.username = username;
+  }
+
+  @JsonProperty("id")
+  public String getQueryId() {
+    return queryId;
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  @JsonProperty("query")
+  public String getQueryText() {
+    return queryText;
+  }
+
+  public Date getStart() {
+    return start;
+  }
+
+  public Date getFinish() {
+    return finish;
+  }
+
+  public QueryState getOutcome() {
+    return outcome;
+  }
+
+  @JsonProperty("user")
+  public String getUsername() {
+    return username;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 34fa639..c4646bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -225,7 +226,7 @@ public class QueryManager {
     }
   }
 
-  QueryState updateQueryStateInStore(final QueryState queryState) {
+  QueryState updateEphemeralState(final QueryState queryState) {
     switch (queryState) {
       case PENDING:
       case RUNNING:
@@ -242,8 +243,6 @@ public class QueryManager {
           logger.warn("Failure while trying to delete the estore profile for this query.", e);
         }
 
-        // TODO(DRILL-2362) when do these ever get deleted?
-        profilePStore.put(stringQueryId, getQueryProfile());
         break;
 
       default:
@@ -253,18 +252,33 @@ public class QueryManager {
     return queryState;
   }
 
+  void writeFinalProfile(UserException ex) {
+    try {
+      // TODO(DRILL-2362) when do these ever get deleted?
+      profilePStore.put(stringQueryId, getQueryProfile(ex));
+    } catch (Exception e) {
+      logger.error("Failure while storing Query Profile", e);
+    }
+  }
+
   private QueryInfo getQueryInfo() {
     return QueryInfo.newBuilder()
-      .setQuery(runQuery.getPlan())
-      .setState(foreman.getState())
-      .setForeman(foreman.getQueryContext().getCurrentEndpoint())
-      .setStart(startTime)
-      .build();
+        .setQuery(runQuery.getPlan())
+        .setState(foreman.getState())
+        .setUser(foreman.getQueryContext().getQueryUserName())
+        .setForeman(foreman.getQueryContext().getCurrentEndpoint())
+        .setStart(startTime)
+        .build();
   }
 
   public QueryProfile getQueryProfile() {
+    return getQueryProfile(null);
+  }
+
+  private QueryProfile getQueryProfile(UserException ex) {
     final QueryProfile.Builder profileBuilder = QueryProfile.newBuilder()
         .setQuery(runQuery.getPlan())
+        .setUser(foreman.getQueryContext().getQueryUserName())
         .setType(runQuery.getType())
         .setId(queryId)
         .setState(foreman.getState())
@@ -274,6 +288,15 @@ public class QueryManager {
         .setTotalFragments(fragmentDataSet.size())
         .setFinishedFragments(finishedFragments.get());
 
+    if (ex != null) {
+      profileBuilder.setError(ex.getMessage(false));
+      profileBuilder.setVerboseError(ex.getVerboseMessage(false));
+      profileBuilder.setErrorId(ex.getErrorId());
+      if (ex.getErrorLocation() != null) {
+        profileBuilder.setErrorNode(ex.getErrorLocation());
+      }
+    }
+
     if (planText != null) {
       profileBuilder.setPlan(planText);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/resources/rest/profile/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/list.ftl b/exec/java-exec/src/main/resources/rest/profile/list.ftl
index 921e52f..cf92ede 100644
--- a/exec/java-exec/src/main/resources/rest/profile/list.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/list.ftl
@@ -24,6 +24,7 @@
         <thead>
            <td>Time</td>
            <!-- <td>Query Id</td> -->
+           <td>User</td>
            <td>Query</td>
            <td>State</td>
            <td>Foreman</td>
@@ -43,6 +44,11 @@
             -->
             <td>
               <a href="/profiles/${query.getQueryId()}">
+              <div style="height:100%;width:100%;white-space:pre-line">${query.getUser()}</div>
+              </a>
+            </td> 
+            <td>
+              <a href="/profiles/${query.getQueryId()}">
               <div style="height:100%;width:100%;white-space:pre-line">${query.getQuery()}</div>
               </a>
             </td> 
@@ -72,6 +78,7 @@
     <table class="table table-hover">
       <thead>
          <td>Time</td>
+         <td>User</td>
          <!-- <td>Query Id</td> -->
          <td>Query</td>
          <td>State</td>
@@ -92,6 +99,12 @@
           -->
           <td>
             <a href="/profiles/${query.getQueryId()}">
+            <div style="height:100%;width:100%;white-space:pre-line">${query.getUser()}</div>
+            </a>
+          </td> 
+          
+          <td>
+            <a href="/profiles/${query.getQueryId()}">
               <div style="height:100%;width:100%;white-space:pre-line">${query.getQuery()}</div>
             </a>
           </td>      

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/main/resources/rest/profile/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
index 42a7447..46cdc83 100644
--- a/exec/java-exec/src/main/resources/rest/profile/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -32,6 +32,7 @@
     <li><a href="#query-physical" role="tab" data-toggle="tab">Physical Plan</a></li>
     <li><a href="#query-visual" role="tab" data-toggle="tab">Visualized Plan</a></li>
     <li><a href="#query-edit" role="tab" data-toggle="tab">Edit Query</a></li>
+    <#if model.hasError() ><li><a href="#query-error" role="tab" data-toggle="tab">Error</a></li></#if>
   </ul>
   <div id="query-content" class="tab-content">
     <div id="query-query" class="tab-pane">
@@ -74,6 +75,30 @@
         <button type="link" class="btn btn-default">Cancel query</button>
       </form>
     </div>
+    <#if model.hasError() >
+    <div id="query-error" class="tab-pane">
+      <p>
+      <pre>
+      ${model.getProfile().error?trim}
+      </pre>
+      </p>
+      <p>ErrorId: ${model.getProfile().errorNode}</p>
+      <p>Failure node:  ${model.getProfile().errorId}</p> 
+      
+        <h3 class="panel-title">
+          <a data-toggle="collapse" href="#error-verbose">
+            Verbose Error Message...
+          </a>
+        </h3>
+      <div id="error-verbose" class="panel-collapse collapse">
+        <div class="panel-body">
+        <p></p><p></p>
+          <pre>
+            ${model.getProfile().verboseError?trim}
+          </pre>
+        </div>
+    </div>
+    </#if>
   </div>
 
   <div class="page-header"></div>

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/logback.xml b/exec/java-exec/src/test/resources/logback.xml
index 2dcfed9..54ccb42 100644
--- a/exec/java-exec/src/test/resources/logback.xml
+++ b/exec/java-exec/src/test/resources/logback.xml
@@ -36,6 +36,11 @@
     <appender-ref ref="SOCKET" />
   </logger>
 
+  <logger name="query.logger" additivity="false">
+    <level value="info" />
+    <appender-ref ref="SOCKET" />
+  </logger>
+
 <!--   <logger name="io.netty" additivity="false"> -->
 <!--     <level value="debug" /> -->
 <!--     <appender-ref ref="SOCKET" /> -->

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/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 eca19a0..bee2a3d 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
@@ -1778,6 +1778,16 @@ public final class SchemaUserBitShared
                 for(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile fragmentProfile : message.getFragmentProfileList())
                     output.writeObject(11, fragmentProfile, org.apache.drill.exec.proto.SchemaUserBitShared.MajorFragmentProfile.WRITE, true);
 
+                if(message.hasUser())
+                    output.writeString(12, message.getUser(), false);
+                if(message.hasError())
+                    output.writeString(13, message.getError(), false);
+                if(message.hasVerboseError())
+                    output.writeString(14, message.getVerboseError(), false);
+                if(message.hasErrorId())
+                    output.writeString(15, message.getErrorId(), false);
+                if(message.hasErrorNode())
+                    output.writeString(16, message.getErrorNode(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.QueryProfile message)
             {
@@ -1853,6 +1863,21 @@ public final class SchemaUserBitShared
                             builder.addFragmentProfile(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.MajorFragmentProfile.MERGE));
 
                             break;
+                        case 12:
+                            builder.setUser(input.readString());
+                            break;
+                        case 13:
+                            builder.setError(input.readString());
+                            break;
+                        case 14:
+                            builder.setVerboseError(input.readString());
+                            break;
+                        case 15:
+                            builder.setErrorId(input.readString());
+                            break;
+                        case 16:
+                            builder.setErrorNode(input.readString());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -1904,6 +1929,11 @@ public final class SchemaUserBitShared
                 case 9: return "totalFragments";
                 case 10: return "finishedFragments";
                 case 11: return "fragmentProfile";
+                case 12: return "user";
+                case 13: return "error";
+                case 14: return "verboseError";
+                case 15: return "errorId";
+                case 16: return "errorNode";
                 default: return null;
             }
         }
@@ -1926,6 +1956,11 @@ public final class SchemaUserBitShared
             fieldMap.put("totalFragments", 9);
             fieldMap.put("finishedFragments", 10);
             fieldMap.put("fragmentProfile", 11);
+            fieldMap.put("user", 12);
+            fieldMap.put("error", 13);
+            fieldMap.put("verboseError", 14);
+            fieldMap.put("errorId", 15);
+            fieldMap.put("errorNode", 16);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/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 a229450..ec92392 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
@@ -12012,17 +12012,17 @@ public final class UserBitShared {
      */
     org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState getState();
 
-    // optional string user = 4;
+    // optional string user = 4 [default = "-"];
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     boolean hasUser();
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     java.lang.String getUser();
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     com.google.protobuf.ByteString
         getUserBytes();
@@ -12246,17 +12246,17 @@ public final class UserBitShared {
       return state_;
     }
 
-    // optional string user = 4;
+    // optional string user = 4 [default = "-"];
     public static final int USER_FIELD_NUMBER = 4;
     private java.lang.Object user_;
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     public boolean hasUser() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     public java.lang.String getUser() {
       java.lang.Object ref = user_;
@@ -12273,7 +12273,7 @@ public final class UserBitShared {
       }
     }
     /**
-     * <code>optional string user = 4;</code>
+     * <code>optional string user = 4 [default = "-"];</code>
      */
     public com.google.protobuf.ByteString
         getUserBytes() {
@@ -12315,7 +12315,7 @@ public final class UserBitShared {
       query_ = "";
       start_ = 0L;
       state_ = org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState.PENDING;
-      user_ = "";
+      user_ = "-";
       foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
@@ -12497,7 +12497,7 @@ public final class UserBitShared {
         bitField0_ = (bitField0_ & ~0x00000002);
         state_ = org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState.PENDING;
         bitField0_ = (bitField0_ & ~0x00000004);
-        user_ = "";
+        user_ = "-";
         bitField0_ = (bitField0_ & ~0x00000008);
         if (foremanBuilder_ == null) {
           foreman_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
@@ -12762,16 +12762,16 @@ public final class UserBitShared {
         return this;
       }
 
-      // optional string user = 4;
-      private java.lang.Object user_ = "";
+      // optional string user = 4 [default = "-"];
+      private java.lang.Object user_ = "-";
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public boolean hasUser() {
         return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public java.lang.String getUser() {
         java.lang.Object ref = user_;
@@ -12785,7 +12785,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public com.google.protobuf.ByteString
           getUserBytes() {
@@ -12801,7 +12801,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public Builder setUser(
           java.lang.String value) {
@@ -12814,7 +12814,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public Builder clearUser() {
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -12823,7 +12823,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>optional string user = 4;</code>
+       * <code>optional string user = 4 [default = "-"];</code>
        */
       public Builder setUserBytes(
           com.google.protobuf.ByteString value) {
@@ -13109,6 +13109,81 @@ public final class UserBitShared {
      */
     org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
         int index);
+
+    // optional string user = 12 [default = "-"];
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    boolean hasUser();
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    java.lang.String getUser();
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    com.google.protobuf.ByteString
+        getUserBytes();
+
+    // optional string error = 13;
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    boolean hasError();
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    java.lang.String getError();
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorBytes();
+
+    // optional string verboseError = 14;
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    boolean hasVerboseError();
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    java.lang.String getVerboseError();
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    com.google.protobuf.ByteString
+        getVerboseErrorBytes();
+
+    // optional string error_id = 15;
+    /**
+     * <code>optional string error_id = 15;</code>
+     */
+    boolean hasErrorId();
+    /**
+     * <code>optional string error_id = 15;</code>
+     */
+    java.lang.String getErrorId();
+    /**
+     * <code>optional string error_id = 15;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorIdBytes();
+
+    // optional string error_node = 16;
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    boolean hasErrorNode();
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    java.lang.String getErrorNode();
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    com.google.protobuf.ByteString
+        getErrorNodeBytes();
   }
   /**
    * Protobuf type {@code exec.shared.QueryProfile}
@@ -13247,6 +13322,31 @@ public final class UserBitShared {
               fragmentProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.PARSER, extensionRegistry));
               break;
             }
+            case 98: {
+              bitField0_ |= 0x00000400;
+              user_ = input.readBytes();
+              break;
+            }
+            case 106: {
+              bitField0_ |= 0x00000800;
+              error_ = input.readBytes();
+              break;
+            }
+            case 114: {
+              bitField0_ |= 0x00001000;
+              verboseError_ = input.readBytes();
+              break;
+            }
+            case 122: {
+              bitField0_ |= 0x00002000;
+              errorId_ = input.readBytes();
+              break;
+            }
+            case 130: {
+              bitField0_ |= 0x00004000;
+              errorNode_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -13552,6 +13652,221 @@ public final class UserBitShared {
       return fragmentProfile_.get(index);
     }
 
+    // optional string user = 12 [default = "-"];
+    public static final int USER_FIELD_NUMBER = 12;
+    private java.lang.Object user_;
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    public boolean hasUser() {
+      return ((bitField0_ & 0x00000400) == 0x00000400);
+    }
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    public java.lang.String getUser() {
+      java.lang.Object ref = user_;
+      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()) {
+          user_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string user = 12 [default = "-"];</code>
+     */
+    public com.google.protobuf.ByteString
+        getUserBytes() {
+      java.lang.Object ref = user_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        user_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string error = 13;
+    public static final int ERROR_FIELD_NUMBER = 13;
+    private java.lang.Object error_;
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    public boolean hasError() {
+      return ((bitField0_ & 0x00000800) == 0x00000800);
+    }
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    public java.lang.String getError() {
+      java.lang.Object ref = error_;
+      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()) {
+          error_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string error = 13;</code>
+     */
+    public com.google.protobuf.ByteString
+        getErrorBytes() {
+      java.lang.Object ref = error_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        error_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string verboseError = 14;
+    public static final int VERBOSEERROR_FIELD_NUMBER = 14;
+    private java.lang.Object verboseError_;
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    public boolean hasVerboseError() {
+      return ((bitField0_ & 0x00001000) == 0x00001000);
+    }
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    public java.lang.String getVerboseError() {
+      java.lang.Object ref = verboseError_;
+      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()) {
+          verboseError_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string verboseError = 14;</code>
+     */
+    public com.google.protobuf.ByteString
+        getVerboseErrorBytes() {
+      java.lang.Object ref = verboseError_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        verboseError_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional string error_id = 15;
+    public static final int ERROR_ID_FIELD_NUMBER = 15;
+    private java.lang.Object errorId_;
+    /**
+     * <code>optional string error_id = 15;</code>
+     */
+    public boolean hasErrorId() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>optional string error_id = 15;</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 error_id = 15;</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 error_node = 16;
+    public static final int ERROR_NODE_FIELD_NUMBER = 16;
+    private java.lang.Object errorNode_;
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    public boolean hasErrorNode() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    public java.lang.String getErrorNode() {
+      java.lang.Object ref = errorNode_;
+      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()) {
+          errorNode_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string error_node = 16;</code>
+     */
+    public com.google.protobuf.ByteString
+        getErrorNodeBytes() {
+      java.lang.Object ref = errorNode_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        errorNode_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
     private void initFields() {
       id_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
       type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
@@ -13564,6 +13879,11 @@ public final class UserBitShared {
       totalFragments_ = 0;
       finishedFragments_ = 0;
       fragmentProfile_ = java.util.Collections.emptyList();
+      user_ = "-";
+      error_ = "";
+      verboseError_ = "";
+      errorId_ = "";
+      errorNode_ = "";
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -13610,6 +13930,21 @@ public final class UserBitShared {
       for (int i = 0; i < fragmentProfile_.size(); i++) {
         output.writeMessage(11, fragmentProfile_.get(i));
       }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeBytes(12, getUserBytes());
+      }
+      if (((bitField0_ & 0x00000800) == 0x00000800)) {
+        output.writeBytes(13, getErrorBytes());
+      }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        output.writeBytes(14, getVerboseErrorBytes());
+      }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        output.writeBytes(15, getErrorIdBytes());
+      }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        output.writeBytes(16, getErrorNodeBytes());
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -13663,6 +13998,26 @@ public final class UserBitShared {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(11, fragmentProfile_.get(i));
       }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(12, getUserBytes());
+      }
+      if (((bitField0_ & 0x00000800) == 0x00000800)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(13, getErrorBytes());
+      }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(14, getVerboseErrorBytes());
+      }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(15, getErrorIdBytes());
+      }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(16, getErrorNodeBytes());
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -13816,6 +14171,16 @@ public final class UserBitShared {
         } else {
           fragmentProfileBuilder_.clear();
         }
+        user_ = "-";
+        bitField0_ = (bitField0_ & ~0x00000800);
+        error_ = "";
+        bitField0_ = (bitField0_ & ~0x00001000);
+        verboseError_ = "";
+        bitField0_ = (bitField0_ & ~0x00002000);
+        errorId_ = "";
+        bitField0_ = (bitField0_ & ~0x00004000);
+        errorNode_ = "";
+        bitField0_ = (bitField0_ & ~0x00008000);
         return this;
       }
 
@@ -13901,6 +14266,26 @@ public final class UserBitShared {
         } else {
           result.fragmentProfile_ = fragmentProfileBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+          to_bitField0_ |= 0x00000400;
+        }
+        result.user_ = user_;
+        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+          to_bitField0_ |= 0x00000800;
+        }
+        result.error_ = error_;
+        if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
+          to_bitField0_ |= 0x00001000;
+        }
+        result.verboseError_ = verboseError_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00002000;
+        }
+        result.errorId_ = errorId_;
+        if (((from_bitField0_ & 0x00008000) == 0x00008000)) {
+          to_bitField0_ |= 0x00004000;
+        }
+        result.errorNode_ = errorNode_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -13977,17 +14362,42 @@ public final class UserBitShared {
             }
           }
         }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        if (other.hasUser()) {
+          bitField0_ |= 0x00000800;
+          user_ = other.user_;
+          onChanged();
+        }
+        if (other.hasError()) {
+          bitField0_ |= 0x00001000;
+          error_ = other.error_;
+          onChanged();
+        }
+        if (other.hasVerboseError()) {
+          bitField0_ |= 0x00002000;
+          verboseError_ = other.verboseError_;
+          onChanged();
+        }
+        if (other.hasErrorId()) {
+          bitField0_ |= 0x00004000;
+          errorId_ = other.errorId_;
+          onChanged();
+        }
+        if (other.hasErrorNode()) {
+          bitField0_ |= 0x00008000;
+          errorNode_ = other.errorNode_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      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.QueryProfile parsedMessage = null;
         try {
@@ -14830,6 +15240,376 @@ public final class UserBitShared {
         return fragmentProfileBuilder_;
       }
 
+      // optional string user = 12 [default = "-"];
+      private java.lang.Object user_ = "-";
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public boolean hasUser() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public java.lang.String getUser() {
+        java.lang.Object ref = user_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          user_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public com.google.protobuf.ByteString
+          getUserBytes() {
+        java.lang.Object ref = user_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          user_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public Builder setUser(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000800;
+        user_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public Builder clearUser() {
+        bitField0_ = (bitField0_ & ~0x00000800);
+        user_ = getDefaultInstance().getUser();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user = 12 [default = "-"];</code>
+       */
+      public Builder setUserBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000800;
+        user_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string error = 13;
+      private java.lang.Object error_ = "";
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public boolean hasError() {
+        return ((bitField0_ & 0x00001000) == 0x00001000);
+      }
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public java.lang.String getError() {
+        java.lang.Object ref = error_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          error_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public com.google.protobuf.ByteString
+          getErrorBytes() {
+        java.lang.Object ref = error_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          error_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public Builder setError(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        error_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public Builder clearError() {
+        bitField0_ = (bitField0_ & ~0x00001000);
+        error_ = getDefaultInstance().getError();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error = 13;</code>
+       */
+      public Builder setErrorBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00001000;
+        error_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string verboseError = 14;
+      private java.lang.Object verboseError_ = "";
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public boolean hasVerboseError() {
+        return ((bitField0_ & 0x00002000) == 0x00002000);
+      }
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public java.lang.String getVerboseError() {
+        java.lang.Object ref = verboseError_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          verboseError_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public com.google.protobuf.ByteString
+          getVerboseErrorBytes() {
+        java.lang.Object ref = verboseError_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          verboseError_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public Builder setVerboseError(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00002000;
+        verboseError_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public Builder clearVerboseError() {
+        bitField0_ = (bitField0_ & ~0x00002000);
+        verboseError_ = getDefaultInstance().getVerboseError();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string verboseError = 14;</code>
+       */
+      public Builder setVerboseErrorBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00002000;
+        verboseError_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string error_id = 15;
+      private java.lang.Object errorId_ = "";
+      /**
+       * <code>optional string error_id = 15;</code>
+       */
+      public boolean hasErrorId() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional string error_id = 15;</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 error_id = 15;</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 error_id = 15;</code>
+       */
+      public Builder setErrorId(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        errorId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error_id = 15;</code>
+       */
+      public Builder clearErrorId() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        errorId_ = getDefaultInstance().getErrorId();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error_id = 15;</code>
+       */
+      public Builder setErrorIdBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        errorId_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional string error_node = 16;
+      private java.lang.Object errorNode_ = "";
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public boolean hasErrorNode() {
+        return ((bitField0_ & 0x00008000) == 0x00008000);
+      }
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public java.lang.String getErrorNode() {
+        java.lang.Object ref = errorNode_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          errorNode_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public com.google.protobuf.ByteString
+          getErrorNodeBytes() {
+        java.lang.Object ref = errorNode_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          errorNode_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public Builder setErrorNode(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00008000;
+        errorNode_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public Builder clearErrorNode() {
+        bitField0_ = (bitField0_ & ~0x00008000);
+        errorNode_ = getDefaultInstance().getErrorNode();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string error_node = 16;</code>
+       */
+      public Builder setErrorNodeBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00008000;
+        errorNode_ = value;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.shared.QueryProfile)
     }
 
@@ -20109,68 +20889,70 @@ public final class UserBitShared {
       "\022\n\n\006FAILED\020\004\022\032\n\026CANCELLATION_REQUESTED\020\005" +
       "\"p\n\tQueryData\022&\n\010query_id\030\001 \001(\0132\024.exec.s" +
       "hared.QueryId\022\021\n\trow_count\030\002 \001(\005\022(\n\003def\030" +
-      "\003 \001(\0132\033.exec.shared.RecordBatchDef\"\224\001\n\tQ" +
+      "\003 \001(\0132\033.exec.shared.RecordBatchDef\"\227\001\n\tQ" +
       "ueryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222" +
       "\n\005state\030\003 \001(\0162#.exec.shared.QueryResult.",
-      "QueryState\022\014\n\004user\030\004 \001(\t\022\'\n\007foreman\030\005 \001(" +
-      "\0132\026.exec.DrillbitEndpoint\"\336\002\n\014QueryProfi" +
-      "le\022 \n\002id\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004" +
-      "type\030\002 \001(\0162\026.exec.shared.QueryType\022\r\n\005st" +
-      "art\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n" +
-      "\004plan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.Dril" +
-      "lbitEndpoint\0222\n\005state\030\010 \001(\0162#.exec.share" +
-      "d.QueryResult.QueryState\022\027\n\017total_fragme" +
-      "nts\030\t \001(\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n" +
-      "\020fragment_profile\030\013 \003(\0132!.exec.shared.Ma",
-      "jorFragmentProfile\"t\n\024MajorFragmentProfi" +
-      "le\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026minor_f" +
-      "ragment_profile\030\002 \003(\0132!.exec.shared.Mino" +
-      "rFragmentProfile\"\350\002\n\024MinorFragmentProfil" +
-      "e\022)\n\005state\030\001 \001(\0162\032.exec.shared.FragmentS" +
-      "tate\022(\n\005error\030\002 \001(\0132\031.exec.shared.DrillP" +
-      "BError\022\031\n\021minor_fragment_id\030\003 \001(\005\0226\n\020ope" +
-      "rator_profile\030\004 \003(\0132\034.exec.shared.Operat" +
-      "orProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n\010end_tim" +
-      "e\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017max_memo",
-      "ry_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.exec.D" +
-      "rillbitEndpoint\022\023\n\013last_update\030\n \001(\003\022\025\n\r" +
-      "last_progress\030\013 \001(\003\"\377\001\n\017OperatorProfile\022" +
-      "1\n\rinput_profile\030\001 \003(\0132\032.exec.shared.Str" +
-      "eamProfile\022\023\n\013operator_id\030\003 \001(\005\022\025\n\ropera" +
-      "tor_type\030\004 \001(\005\022\023\n\013setup_nanos\030\005 \001(\003\022\025\n\rp" +
-      "rocess_nanos\030\006 \001(\003\022#\n\033peak_local_memory_" +
-      "allocated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132\030.exec.s" +
-      "hared.MetricValue\022\022\n\nwait_nanos\030\t \001(\003\"B\n" +
-      "\rStreamProfile\022\017\n\007records\030\001 \001(\003\022\017\n\007batch",
-      "es\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013MetricValue" +
-      "\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_value\030\002 \001(\003\022" +
-      "\024\n\014double_value\030\003 \001(\001*5\n\nRpcChannel\022\017\n\013B" +
-      "IT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*/\n\t" +
-      "QueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSI" +
-      "CAL\020\003*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023" +
-      "AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FI" +
-      "NISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026C" +
-      "ANCELLATION_REQUESTED\020\006*\335\005\n\020CoreOperator" +
-      "Type\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SEN",
-      "DER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n" +
-      "\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PAR" +
-      "TITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RE" +
-      "CEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013" +
-      "\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\020\n\014R" +
-      "ANGE_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_V" +
-      "ECTOR_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017" +
-      "\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005T" +
-      "RACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQU" +
-      "ET_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_JSON\020\037\022\025\n\021PRODUCER_CONS" +
-      "UMER\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024" +
-      "\n\020NESTED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$B" +
-      ".\n\033org.apache.drill.exec.protoB\rUserBitS" +
-      "haredH\001"
+      "QueryState\022\017\n\004user\030\004 \001(\t:\001-\022\'\n\007foreman\030\005" +
+      " \001(\0132\026.exec.DrillbitEndpoint\"\272\003\n\014QueryPr" +
+      "ofile\022 \n\002id\030\001 \001(\0132\024.exec.shared.QueryId\022" +
+      "$\n\004type\030\002 \001(\0162\026.exec.shared.QueryType\022\r\n" +
+      "\005start\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t" +
+      "\022\014\n\004plan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.D" +
+      "rillbitEndpoint\0222\n\005state\030\010 \001(\0162#.exec.sh" +
+      "ared.QueryResult.QueryState\022\027\n\017total_fra" +
+      "gments\030\t \001(\005\022\032\n\022finished_fragments\030\n \001(\005" +
+      "\022;\n\020fragment_profile\030\013 \003(\0132!.exec.shared",
+      ".MajorFragmentProfile\022\017\n\004user\030\014 \001(\t:\001-\022\r" +
+      "\n\005error\030\r \001(\t\022\024\n\014verboseError\030\016 \001(\t\022\020\n\010e" +
+      "rror_id\030\017 \001(\t\022\022\n\nerror_node\030\020 \001(\t\"t\n\024Maj" +
+      "orFragmentProfile\022\031\n\021major_fragment_id\030\001" +
+      " \001(\005\022A\n\026minor_fragment_profile\030\002 \003(\0132!.e" +
+      "xec.shared.MinorFragmentProfile\"\350\002\n\024Mino" +
+      "rFragmentProfile\022)\n\005state\030\001 \001(\0162\032.exec.s" +
+      "hared.FragmentState\022(\n\005error\030\002 \001(\0132\031.exe" +
+      "c.shared.DrillPBError\022\031\n\021minor_fragment_" +
+      "id\030\003 \001(\005\0226\n\020operator_profile\030\004 \003(\0132\034.exe",
+      "c.shared.OperatorProfile\022\022\n\nstart_time\030\005" +
+      " \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memory_used\030\007 " +
+      "\001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(\n\010endpoint" +
+      "\030\t \001(\0132\026.exec.DrillbitEndpoint\022\023\n\013last_u" +
+      "pdate\030\n \001(\003\022\025\n\rlast_progress\030\013 \001(\003\"\377\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#\n\033pea" +
+      "k_local_memory_allocated\030\007 \001(\003\022(\n\006metric",
+      "\030\010 \003(\0132\030.exec.shared.MetricValue\022\022\n\nwait" +
+      "_nanos\030\t \001(\003\"B\n\rStreamProfile\022\017\n\007records" +
+      "\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003" +
+      "\"J\n\013MetricValue\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlo" +
+      "ng_value\030\002 \001(\003\022\024\n\014double_value\030\003 \001(\001*5\n\n" +
+      "RpcChannel\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020" +
+      "\001\022\010\n\004USER\020\002*/\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOG" +
+      "ICAL\020\002\022\014\n\010PHYSICAL\020\003*\207\001\n\rFragmentState\022\013" +
+      "\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007" +
+      "RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n",
+      "\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQUESTED\020\006*\335" +
+      "\005\n\020CoreOperatorType\022\021\n\rSINGLE_SENDER\020\000\022\024" +
+      "\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH" +
+      "_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOI" +
+      "N\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020" +
+      "\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030ORDERED_PARTI" +
+      "TION_SENDER\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_" +
+      "RECEIVER\020\013\022\020\n\014RANGE_SENDER\020\014\022\n\n\006SCREEN\020\r" +
+      "\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAM" +
+      "ING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTER",
+      "NAL_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\rHI" +
+      "VE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\r" +
+      "MOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017D" +
+      "IRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEX" +
+      "T_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_" +
+      "SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025" +
+      "\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020" +
+      "!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rA" +
+      "VRO_SUB_SCAN\020$B.\n\033org.apache.drill.exec.",
+      "protoB\rUserBitSharedH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -20260,7 +21042,7 @@ public final class UserBitShared {
           internal_static_exec_shared_QueryProfile_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_QueryProfile_descriptor,
-              new java.lang.String[] { "Id", "Type", "Start", "End", "Query", "Plan", "Foreman", "State", "TotalFragments", "FinishedFragments", "FragmentProfile", });
+              new java.lang.String[] { "Id", "Type", "Start", "End", "Query", "Plan", "Foreman", "State", "TotalFragments", "FinishedFragments", "FragmentProfile", "User", "Error", "VerboseError", "ErrorId", "ErrorNode", });
           internal_static_exec_shared_MajorFragmentProfile_descriptor =
             getDescriptor().getMessageTypes().get(14);
           internal_static_exec_shared_MajorFragmentProfile_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryInfo.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryInfo.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryInfo.java
index 1c86ae7..4080e26 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryInfo.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryInfo.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 
+import com.dyuproject.protostuff.ByteString;
 import com.dyuproject.protostuff.GraphIOUtil;
 import com.dyuproject.protostuff.Input;
 import com.dyuproject.protostuff.Message;
@@ -46,11 +47,12 @@ public final class QueryInfo implements Externalizable, Message<QueryInfo>, Sche
 
     static final QueryInfo DEFAULT_INSTANCE = new QueryInfo();
 
+    static final String DEFAULT_USER = ByteString.stringDefaultValue("-");
     
     private String query;
     private long start;
     private QueryResult.QueryState state;
-    private String user;
+    private String user = DEFAULT_USER;
     private DrillbitEndpoint foreman;
 
     public QueryInfo()
@@ -213,7 +215,7 @@ public final class QueryInfo implements Externalizable, Message<QueryInfo>, Sche
         if(message.state != null)
              output.writeEnum(3, message.state.number, false);
 
-        if(message.user != null)
+        if(message.user != null && message.user != DEFAULT_USER)
             output.writeString(4, message.user, false);
 
         if(message.foreman != null)

http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryProfile.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryProfile.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryProfile.java
index 235b6e6..c610a84 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryProfile.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryProfile.java
@@ -27,6 +27,7 @@ import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.dyuproject.protostuff.ByteString;
 import com.dyuproject.protostuff.GraphIOUtil;
 import com.dyuproject.protostuff.Input;
 import com.dyuproject.protostuff.Message;
@@ -48,6 +49,7 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
 
     static final QueryProfile DEFAULT_INSTANCE = new QueryProfile();
 
+    static final String DEFAULT_USER = ByteString.stringDefaultValue("-");
     
     private QueryId id;
     private QueryType type;
@@ -60,6 +62,11 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
     private int totalFragments;
     private int finishedFragments;
     private List<MajorFragmentProfile> fragmentProfile;
+    private String user = DEFAULT_USER;
+    private String error;
+    private String verboseError;
+    private String errorId;
+    private String errorNode;
 
     public QueryProfile()
     {
@@ -211,6 +218,71 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
         return this;
     }
 
+    // user
+
+    public String getUser()
+    {
+        return user;
+    }
+
+    public QueryProfile setUser(String user)
+    {
+        this.user = user;
+        return this;
+    }
+
+    // error
+
+    public String getError()
+    {
+        return error;
+    }
+
+    public QueryProfile setError(String error)
+    {
+        this.error = error;
+        return this;
+    }
+
+    // verboseError
+
+    public String getVerboseError()
+    {
+        return verboseError;
+    }
+
+    public QueryProfile setVerboseError(String verboseError)
+    {
+        this.verboseError = verboseError;
+        return this;
+    }
+
+    // errorId
+
+    public String getErrorId()
+    {
+        return errorId;
+    }
+
+    public QueryProfile setErrorId(String errorId)
+    {
+        this.errorId = errorId;
+        return this;
+    }
+
+    // errorNode
+
+    public String getErrorNode()
+    {
+        return errorNode;
+    }
+
+    public QueryProfile setErrorNode(String errorNode)
+    {
+        this.errorNode = errorNode;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -303,6 +375,21 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
                     message.fragmentProfile.add(input.mergeObject(null, MajorFragmentProfile.getSchema()));
                     break;
 
+                case 12:
+                    message.user = input.readString();
+                    break;
+                case 13:
+                    message.error = input.readString();
+                    break;
+                case 14:
+                    message.verboseError = input.readString();
+                    break;
+                case 15:
+                    message.errorId = input.readString();
+                    break;
+                case 16:
+                    message.errorNode = input.readString();
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -353,6 +440,21 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
             }
         }
 
+
+        if(message.user != null && message.user != DEFAULT_USER)
+            output.writeString(12, message.user, false);
+
+        if(message.error != null)
+            output.writeString(13, message.error, false);
+
+        if(message.verboseError != null)
+            output.writeString(14, message.verboseError, false);
+
+        if(message.errorId != null)
+            output.writeString(15, message.errorId, false);
+
+        if(message.errorNode != null)
+            output.writeString(16, message.errorNode, false);
     }
 
     public String getFieldName(int number)
@@ -370,6 +472,11 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
             case 9: return "totalFragments";
             case 10: return "finishedFragments";
             case 11: return "fragmentProfile";
+            case 12: return "user";
+            case 13: return "error";
+            case 14: return "verboseError";
+            case 15: return "errorId";
+            case 16: return "errorNode";
             default: return null;
         }
     }
@@ -394,6 +501,11 @@ public final class QueryProfile implements Externalizable, Message<QueryProfile>
         __fieldMap.put("totalFragments", 9);
         __fieldMap.put("finishedFragments", 10);
         __fieldMap.put("fragmentProfile", 11);
+        __fieldMap.put("user", 12);
+        __fieldMap.put("error", 13);
+        __fieldMap.put("verboseError", 14);
+        __fieldMap.put("errorId", 15);
+        __fieldMap.put("errorNode", 16);
     }
     
 }


[02/12] drill git commit: DRILL-3007: Disable automatic mmap threshold on Linux

Posted by ja...@apache.org.
DRILL-3007: Disable automatic mmap threshold on Linux


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

Branch: refs/heads/merge_2015_05_09
Commit: 0f424096a1d98e603d346ba4906eb7c3434855df
Parents: 42d5f81
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat May 9 17:17:56 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:24:21 2015 -0700

----------------------------------------------------------------------
 distribution/src/resources/drill-config.sh | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0f424096/distribution/src/resources/drill-config.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-config.sh b/distribution/src/resources/drill-config.sh
index c66afea..3de6be4 100644
--- a/distribution/src/resources/drill-config.sh
+++ b/distribution/src/resources/drill-config.sh
@@ -197,6 +197,13 @@ if $is_cygwin; then
   fi
 fi
 
+# make sure allocator chunks are done as mmap'd memory (and reduce arena overhead)
+export MALLOC_ARENA_MAX=4
+export MALLOC_MMAP_THRESHOLD_=131072
+export MALLOC_TRIM_THRESHOLD_=131072
+export MALLOC_TOP_PAD_=131072
+export MALLOC_MMAP_MAX_=65536
+
 # Variables exported form this script
 export HADOOP_HOME
 export is_cygwin


[03/12] drill git commit: DRILL-2981: Add queries log. Update profile to store normal and verbose exception as well as node and errorid.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/42d5f818/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index a17dbc7..e3b6168 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -174,7 +174,7 @@ message QueryInfo {
   optional string query = 1;
   optional int64 start = 2;
   optional QueryResult.QueryState state = 3;
-  optional string user = 4;
+  optional string user = 4 [default = "-"];
   optional DrillbitEndpoint foreman = 5;
 }
 
@@ -191,6 +191,11 @@ message QueryProfile {
   optional int32 total_fragments = 9;
   optional int32 finished_fragments = 10;
   repeated MajorFragmentProfile fragment_profile = 11;
+  optional string user = 12 [default = "-"];
+  optional string error = 13;
+  optional string verboseError = 14;
+  optional string error_id = 15;
+  optional string error_node = 16;
 }
 
 message MajorFragmentProfile {


[12/12] drill git commit: DRILL-2870: Part2 - Fix return type of aggregate functions to be nullable

Posted by ja...@apache.org.
DRILL-2870: Part2 - Fix return type of aggregate functions to be nullable


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

Branch: refs/heads/merge_2015_05_09
Commit: 6a7de92e3692a75ade346e87af10d8ed1705f916
Parents: ed5fa92
Author: Mehant Baid <me...@gmail.com>
Authored: Sat May 9 13:36:05 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun May 10 09:30:22 2015 -0700

----------------------------------------------------------------------
 .../codegen/data/AggrBitwiseLogicalTypes.tdd    |  64 ++++++------
 .../src/main/codegen/data/AggrTypes1.tdd        | 102 +++++++++----------
 .../src/main/codegen/data/AggrTypes2.tdd        |  20 ++--
 .../src/main/codegen/data/AggrTypes3.tdd        |  80 +++++++--------
 .../AggrBitwiseLogicalTypeFunctions.java        |  23 +++--
 .../codegen/templates/AggrTypeFunctions1.java   |  40 +++-----
 .../codegen/templates/AggrTypeFunctions2.java   |  36 +++----
 .../codegen/templates/AggrTypeFunctions3.java   |  42 ++------
 .../templates/DateIntervalAggrFunctions1.java   |  80 ++++++++-------
 .../templates/IntervalAggrFunctions2.java       |  39 ++++---
 .../templates/VarCharAggrFunctions1.java        |  53 ++++------
 .../exec/physical/impl/join/JoinUtils.java      |   2 +-
 .../exec/fn/impl/TestAggregateFunctions.java    |  71 ++++++++++++-
 .../drill/exec/physical/impl/agg/TestAgg.java   |   3 +-
 .../resources/parquet/alltypes_required.parquet | Bin 0 -> 1197 bytes
 15 files changed, 340 insertions(+), 315 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/data/AggrBitwiseLogicalTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrBitwiseLogicalTypes.tdd b/exec/java-exec/src/main/codegen/data/AggrBitwiseLogicalTypes.tdd
index 2b72abd..1070faa 100644
--- a/exec/java-exec/src/main/codegen/data/AggrBitwiseLogicalTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrBitwiseLogicalTypes.tdd
@@ -17,41 +17,41 @@
 {
   logicalAggrTypes: [
     {className: "BitwiseAnd", funcName: "bit_and", aliasName: "every", types: [
-     {inputType: "BigInt", outputType: "BigInt", maxval: "Integer"},
-      {inputType: "NullableBigInt", outputType: "BigInt", maxval: "Integer"},
-      {inputType: "Int", outputType: "Int", maxval: "Integer"},
-      {inputType: "NullableInt", outputType: "Int", maxval: "Integer"},
-      {inputType: "SmallInt", outputType: "SmallInt", maxval: "Byte", extraCast: "short"},
-      {inputType: "NullableSmallInt", outputType: "SmallInt", maxval: "Byte", extraCast: "short"},
-      {inputType: "TinyInt", outputType: "TinyInt", maxval: "Byte", extraCast: "byte"},
-      {inputType: "NullableTinyInt", outputType: "TinyInt", maxval: "Byte", extraCast: "byte"},
-      {inputType: "UInt1", outputType: "Int", maxval: "Byte"},
-      {inputType: "NullableUInt1", outputType: "Int", maxval: "Byte"},
-      {inputType: "UInt2", outputType: "Int", maxval: "Character"},
-      {inputType: "NullableUInt2", outputType: "Int", maxval: "Character"},
-      {inputType: "UInt4", outputType: "Int", maxval: "Integer"},
-      {inputType: "NullableUInt4", outputType: "Int", maxval: "Integer"},
-      {inputType: "UInt8", outputType: "UInt8", maxval: "Long"},
-      {inputType: "NullableUInt8", outputType: "UInt8", maxval: "Long"}
+     {inputType: "BigInt", outputType: "NullableBigInt", maxval: "Integer"},
+      {inputType: "NullableBigInt", outputType: "NullableBigInt", maxval: "Integer"},
+      {inputType: "Int", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "NullableInt", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "SmallInt", outputType: "NullableSmallInt", maxval: "Byte", extraCast: "short"},
+      {inputType: "NullableSmallInt", outputType: "NullableSmallInt", maxval: "Byte", extraCast: "short"},
+      {inputType: "TinyInt", outputType: "NullableTinyInt", maxval: "Byte", extraCast: "byte"},
+      {inputType: "NullableTinyInt", outputType: "NullableTinyInt", maxval: "Byte", extraCast: "byte"},
+      {inputType: "UInt1", outputType: "NullableInt", maxval: "Byte"},
+      {inputType: "NullableUInt1", outputType: "NullableInt", maxval: "Byte"},
+      {inputType: "UInt2", outputType: "NullableInt", maxval: "Character"},
+      {inputType: "NullableUInt2", outputType: "NullableInt", maxval: "Character"},
+      {inputType: "UInt4", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "NullableUInt4", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "UInt8", outputType: "NullableUInt8", maxval: "Long"},
+      {inputType: "NullableUInt8", outputType: "NullableUInt8", maxval: "Long"}
      ]
    },
    {className: "BitwiseOr", funcName: "bit_or", aliasName: "", types: [
-     {inputType: "BigInt", outputType: "BigInt", maxval: "Integer"},
-      {inputType: "NullableBigInt", outputType: "BigInt", maxval: "Integer"},
-      {inputType: "Int", outputType: "Int", maxval: "Integer"},
-      {inputType: "NullableInt", outputType: "Int", maxval: "Integer"},
-      {inputType: "SmallInt", outputType: "SmallInt", maxval: "Byte", extraCast: "short"},
-      {inputType: "NullableSmallInt", outputType: "SmallInt", maxval: "Byte", extraCast: "short"},
-      {inputType: "TinyInt", outputType: "TinyInt", maxval: "Byte", extraCast: "byte"},
-      {inputType: "NullableTinyInt", outputType: "TinyInt", maxval: "Byte", extraCast: "byte"},
-      {inputType: "UInt1", outputType: "Int", maxval: "Byte"},
-      {inputType: "NullableUInt1", outputType: "Int", maxval: "Byte"},
-      {inputType: "UInt2", outputType: "Int", maxval: "Character"},
-      {inputType: "NullableUInt2", outputType: "Int", maxval: "Character"},
-      {inputType: "UInt4", outputType: "Int", maxval: "Integer"},
-      {inputType: "NullableUInt4", outputType: "Int", maxval: "Integer"},
-      {inputType: "UInt8", outputType: "UInt8", maxval: "Long"},
-      {inputType: "NullableUInt8", outputType: "UInt8", maxval: "Long"}
+     {inputType: "BigInt", outputType: "NullableBigInt", maxval: "Integer"},
+      {inputType: "NullableBigInt", outputType: "NullableBigInt", maxval: "Integer"},
+      {inputType: "Int", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "NullableInt", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "SmallInt", outputType: "NullableSmallInt", maxval: "Byte", extraCast: "short"},
+      {inputType: "NullableSmallInt", outputType: "NullableSmallInt", maxval: "Byte", extraCast: "short"},
+      {inputType: "TinyInt", outputType: "NullableTinyInt", maxval: "Byte", extraCast: "byte"},
+      {inputType: "NullableTinyInt", outputType: "NullableTinyInt", maxval: "Byte", extraCast: "byte"},
+      {inputType: "UInt1", outputType: "NullableInt", maxval: "Byte"},
+      {inputType: "NullableUInt1", outputType: "NullableInt", maxval: "Byte"},
+      {inputType: "UInt2", outputType: "NullableInt", maxval: "Character"},
+      {inputType: "NullableUInt2", outputType: "NullableInt", maxval: "Character"},
+      {inputType: "UInt4", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "NullableUInt4", outputType: "NullableInt", maxval: "Integer"},
+      {inputType: "UInt8", outputType: "NullableUInt8", maxval: "Long"},
+      {inputType: "NullableUInt8", outputType: "NullableUInt8", maxval: "Long"}
      ]
    }
   ]

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index 1a98a07..202f539 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -17,76 +17,76 @@
 {
   aggrtypes: [
     {className: "Min", funcName: "min", types: [
-      {inputType: "Bit", outputType: "Bit", runningType: "Bit", major: "Numeric"},
-      {inputType: "Int", outputType: "Int", runningType: "Int", major: "Numeric"},
-      {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
+      {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+      {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+      {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
       {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
       {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
       {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Float4", outputType: "Float4", runningType: "Float4", major: "Numeric"},
-      {inputType: "Float8", outputType: "Float8", runningType: "Float8", major: "Numeric"},
+      {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+      {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
       {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
-      {inputType: "Date", outputType: "Date", runningType: "Date", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "NullableDate", outputType: "Date", runningType: "Date", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "TimeStamp", outputType: "TimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "NullableTimeStamp", outputType: "TimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "Time", outputType: "Time", runningType: "Time", major: "Date", initialValue: "Integer.MAX_VALUE"},
-      {inputType: "NullableTime", outputType: "Time", runningType: "Time", major: "Date", initialValue: "Integer.MAX_VALUE"},
-      {inputType: "IntervalDay", outputType: "IntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "NullableIntervalDay", outputType: "IntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "IntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MAX_VALUE"},
-      {inputType: "NullableIntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MAX_VALUE"},
-      {inputType: "Interval", outputType: "Interval", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "NullableInterval", outputType: "Interval", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
-      {inputType: "VarChar", outputType: "VarChar", runningType: "VarChar", major: "VarBytes"},
-      {inputType: "NullableVarChar", outputType: "VarChar", runningType: "VarChar", major: "VarBytes"},
-      {inputType: "VarBinary", outputType: "VarBinary", runningType: "VarBinary", major: "VarBytes"},
-      {inputType: "NullableVarBinary", outputType: "VarBinary", runningType: "VarBinary", major: "VarBytes"}
+      {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "Integer.MAX_VALUE"},
+      {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "Integer.MAX_VALUE"},
+      {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MAX_VALUE"},
+      {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MAX_VALUE"},
+      {inputType: "Interval", outputType: "NullableInterval", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "BigInt", major: "Date", initialValue: "Long.MAX_VALUE"},
+      {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes"},
+      {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes"},
+      {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
+      {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
      ]
    },
    {className: "Max", funcName: "max", types: [
-      {inputType: "Int", outputType: "Int", runningType: "Int", major: "Numeric"},
-      {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
+      {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+      {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
       {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
       {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Float4", outputType: "Float4", runningType: "Float4", major: "Numeric"},
-      {inputType: "Float8", outputType: "Float8", runningType: "Float8", major: "Numeric"},
+      {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+      {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
       {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
-      {inputType: "Date", outputType: "Date", runningType: "Date", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "NullableDate", outputType: "Date", runningType: "Date", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "TimeStamp", outputType: "TimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "NullableTimeStamp", outputType: "TimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "Time", outputType: "Time", runningType: "Time", major: "Date", initialValue: "Integer.MIN_VALUE"},
-      {inputType: "NullableTime", outputType: "Time", runningType: "Time", major: "Date", initialValue: "Integer.MIN_VALUE"},
-      {inputType: "IntervalDay", outputType: "IntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "NullableIntervalDay", outputType: "IntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "IntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MIN_VALUE"},
-      {inputType: "NullableIntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MIN_VALUE"},
-      {inputType: "Interval", outputType: "Interval", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "NullableInterval", outputType: "Interval", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
-      {inputType: "VarChar", outputType: "VarChar", runningType: "VarChar", major: "VarBytes"},
-      {inputType: "NullableVarChar", outputType: "VarChar", runningType: "VarChar", major: "VarBytes"},
-      {inputType: "VarBinary", outputType: "VarBinary", runningType: "VarBinary", major: "VarBytes"},
-      {inputType: "NullableVarBinary", outputType: "VarBinary", runningType: "VarBinary", major: "VarBytes"}
+      {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "Integer.MIN_VALUE"},
+      {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "Integer.MIN_VALUE"},
+      {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MIN_VALUE"},
+      {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "Integer.MIN_VALUE"},
+      {inputType: "Interval", outputType: "NullableInterval", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "BigInt", major: "Date", initialValue: "Long.MIN_VALUE"},
+      {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes"},
+      {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes"},
+      {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
+      {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
      ]
    },
    {className: "Sum", funcName: "sum", types: [
-      {inputType: "Int", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt", major: "Numeric"},
+      {inputType: "Int", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+      {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
       {inputType: "NullableInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
       {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
-      {inputType: "Float4", outputType: "Float8", runningType: "Float8", major: "Numeric"},
-      {inputType: "Float8", outputType: "Float8", runningType: "Float8", major: "Numeric"},
+      {inputType: "Float4", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+      {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
-      {inputType: "IntervalDay", outputType: "IntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
-      {inputType: "NullableIntervalDay", outputType: "IntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
-      {inputType: "IntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
-      {inputType: "NullableIntervalYear", outputType: "IntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
-      {inputType: "Interval", outputType: "Interval", runningType: "Interval", major: "Date", initialValue: "0"},
-      {inputType: "NullableInterval", outputType: "Interval", runningType: "Interval", major: "Date", initialValue: "0"}
+      {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+      {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+      {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+      {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+      {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+      {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"}
      ]
    }
   ]

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/data/AggrTypes2.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes2.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes2.tdd
index ee64daf..183d028 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes2.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes2.tdd
@@ -17,20 +17,20 @@
 {
   aggrtypes: [
     {className: "Avg", funcName: "avg", types: [
-      {inputType: "Int", outputType: "Float8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
-      {inputType: "BigInt", outputType: "Float8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
+      {inputType: "Int", outputType: "NullableFloat8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
+      {inputType: "BigInt", outputType: "NullableFloat8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
       {inputType: "NullableInt", outputType: "NullableFloat8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
       {inputType: "NullableBigInt", outputType: "NullableFloat8", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Numeric"},
-      {inputType: "Float4", outputType: "Float8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
-      {inputType: "Float8", outputType: "Float8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
+      {inputType: "Float4", outputType: "NullableFloat8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
+      {inputType: "Float8", outputType: "NullableFloat8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", sumRunningType: "Float8", countRunningType: "BigInt", major: "Numeric"},
-      {inputType: "IntervalDay", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
-      {inputType: "NullableIntervalDay", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
-      {inputType: "IntervalYear", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
-      {inputType: "NullableIntervalYear", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
-      {inputType: "Interval", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
-      {inputType: "NullableInterval", outputType: "Interval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"}
+      {inputType: "IntervalDay", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
+      {inputType: "NullableIntervalDay", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
+      {inputType: "IntervalYear", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
+      {inputType: "NullableIntervalYear", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
+      {inputType: "Interval", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"},
+      {inputType: "NullableInterval", outputType: "NullableInterval", sumRunningType: "BigInt", countRunningType: "BigInt", major: "Date"}
      ]
    }
   ]

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/data/AggrTypes3.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes3.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes3.tdd
index 0c3a358..7fb67cf 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes3.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes3.tdd
@@ -17,97 +17,97 @@
 {
   aggrtypes: [
     {className: "StdDevPop", funcName: "stddev_pop", aliasName: "", types: [
-      {inputType: "BigInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "BigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Int", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Int", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "SmallInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "SmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableSmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "TinyInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "TinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableTinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt1", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt2", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"}
      ]
    },
 
     {className: "VariancePop", funcName: "var_pop", aliasName: "", types: [
-      {inputType: "BigInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "BigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Int", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Int", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "SmallInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "SmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableSmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "TinyInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "TinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableTinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt1", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt2", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"}
      ]
    },
 
     {className: "StdDevSample", funcName: "stddev_samp", aliasName: "stddev", types: [
-      {inputType: "BigInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "BigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Int", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Int", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "SmallInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "SmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableSmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "TinyInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "TinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableTinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt1", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt2", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"}
      ]
    },
 
     {className: "VarianceSample", funcName: "var_samp", aliasName: "variance", types: [
-      {inputType: "BigInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "BigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Int", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Int", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "SmallInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "SmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableSmallInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "TinyInt", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "TinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableTinyInt", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt1", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt1", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt2", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt2", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "UInt8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "UInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableUInt8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float4", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat4", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
-      {inputType: "Float8", outputType: "Float8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
+      {inputType: "Float8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"},
       {inputType: "NullableFloat8", outputType: "NullableFloat8", movingAverageType: "Float8", movingDeviationType: "Float8", countRunningType: "BigInt"}
      ]
    }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
index b159421..3453058 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
@@ -72,10 +72,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Param ${type.inputType}Holder in;
   @Workspace ${type.outputType}Holder inter;
+  @Workspace BigIntHolder nonNullCount;
   @Output ${type.outputType}Holder out;
 
   public void setup() {
-  inter = new ${type.outputType}Holder();
+    inter = new ${type.outputType}Holder();
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
 
     // Initialize the workspace variables
   <#if aggrtype.funcName == "bit_and">
@@ -94,7 +97,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
      break sout;
     }
   </#if>
-
+    nonNullCount.value = 1;
   <#if aggrtype.funcName == "bit_and">
     inter.value = <#if type.extraCast ??>(${type.extraCast})</#if>(inter.value & in.value);
     <#elseif aggrtype.funcName == "bit_or">
@@ -108,15 +111,21 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-    <#if aggrtype.funcName == "bit_and">
-      out.value = inter.value;
-      <#elseif aggrtype.funcName == "bit_or">
-      out.value = inter.value;
-    </#if>
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
+      <#if aggrtype.funcName == "bit_and">
+        out.value = inter.value;
+        <#elseif aggrtype.funcName == "bit_or">
+        out.value = inter.value;
+      </#if>
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
+    nonNullCount.value = 0;
     <#if aggrtype.funcName == "bit_and">
       inter.value = ${type.maxval}.MAX_VALUE;
       <#elseif aggrtype.funcName == "bit_or">

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index 19a6d46..1360f66 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -24,12 +24,10 @@
 
 <#include "/@includes/license.ftl" />
 
-<#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
-<#-- running counter to hold the result.  This includes: MIN, MAX, SUM, COUNT. -->
+// Source code generated using FreeMarker template ${.template_name}
 
-/* 
- * This class is automatically generated from AggrTypeFunctions1.tdd using FreeMarker.
- */
+<#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
+<#-- running counter to hold the result.  This includes: MIN, MAX, SUM. -->
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
@@ -54,18 +52,14 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Param ${type.inputType}Holder in;
   @Workspace ${type.runningType}Holder value;
-  <#if type.inputType?starts_with("Nullable") && type.outputType?starts_with("Nullable")>
-    @Workspace BigIntHolder nonNullCount;
-  </#if>
+  @Workspace BigIntHolder nonNullCount;
   @Output ${type.outputType}Holder out;
 
   public void setup() {
-	value = new ${type.runningType}Holder();
-  <#if type.inputType?starts_with("Nullable") && type.outputType?starts_with("Nullable")>
+	  value = new ${type.runningType}Holder();
 	  nonNullCount = new BigIntHolder();
 	  nonNullCount.value = 0;
-	</#if>
-	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "count">
+	<#if aggrtype.funcName == "sum">
 	  value.value = 0;
 	<#elseif aggrtype.funcName == "min">
     <#if type.runningType?starts_with("Bit")>
@@ -103,12 +97,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 		    // processing nullable input and the value is null, so don't do anything...
 		    break sout;
 	    }
-      <#if type.outputType?starts_with("Nullable")>
-        else {
-  	        nonNullCount.value++;
-	      }
-      </#if>
 	  </#if>
+    nonNullCount.value = 1;
 	  <#if aggrtype.funcName == "min">
 	    value.value = Math.min(value.value, in.value);
 	  <#elseif aggrtype.funcName == "max">
@@ -127,23 +117,17 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {   
-    <#if type.inputType?starts_with("Nullable") && type.outputType?starts_with("Nullable")>
-      if (nonNullCount.value > 0) {
-        out.value = value.value;
-        out.isSet = 1;
-      } else {
-        out.isSet = 0;
-      }
-    <#else> 
+    if (nonNullCount.value > 0) {
       out.value = value.value;
-    </#if>
+      out.isSet = 1;
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
-  <#if type.inputType?starts_with("Nullable") && type.outputType?starts_with("Nullable")>
     nonNullCount.value = 0;
-  </#if>
 	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "count">
 	  value.value = 0;
 	<#elseif aggrtype.funcName == "min">

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
index 6701f09..fe76cbe 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
@@ -24,13 +24,11 @@
 
 <#include "/@includes/license.ftl" />
 
+// Source code generated using FreeMarker template ${.template_name}
+
 <#-- A utility class that is used to generate java code for aggr functions that maintain a sum -->
 <#-- and a running count.  For now, this includes: AVG. -->
 
-/* 
- * This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
- */
-
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
@@ -63,10 +61,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   public void setup() {
   	sum = new ${type.sumRunningType}Holder();  
     count = new ${type.countRunningType}Holder();  
-    <#if type.inputType?starts_with("Nullable") >
-  	  nonNullCount = new BigIntHolder();
-  	  nonNullCount.value = 0;
-  	</#if>
+  	nonNullCount = new BigIntHolder();
+  	nonNullCount.value = 0;
     sum.value = 0;
     count.value = 0;
   }
@@ -79,10 +75,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 		    // processing nullable input and the value is null, so don't do anything...
 		    break sout;
 	    } 
-      else {
-        nonNullCount.value++;
-      } 
 	  </#if>
+    nonNullCount.value = 1;
 	  <#if aggrtype.funcName == "avg">
  	    sum.value += in.value;
  	    count.value++;
@@ -96,27 +90,21 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-    <#if type.inputType?starts_with("Nullable") >
-      if (nonNullCount.value > 0) {
-        out.value = sum.value / ((double) count.value);
-        out.isSet = 1;
-      } else {
-        out.isSet = 0;
-      }
-    <#else> 
+    if (nonNullCount.value > 0) {
       out.value = sum.value / ((double) count.value);
-    </#if>
+      out.isSet = 1;
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
-    <#if type.inputType?starts_with("Nullable") >
-      nonNullCount.value = 0;
-    </#if>
+    nonNullCount.value = 0;
     sum.value = 0;
     count.value = 0;
   }
- 
+
  }
 
 </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
index c005446..ee158bd 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
@@ -24,11 +24,9 @@
 
 <#include "/@includes/license.ftl" />
 
-<#-- A utility class that is used to generate java code for aggr functions such as stddev, variance -->
+// Source code generated using FreeMarker template ${.template_name}
 
-/*
- * This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
- */
+<#-- A utility class that is used to generate java code for aggr functions such as stddev, variance -->
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
@@ -67,10 +65,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   	avg = new ${type.movingAverageType}Holder();
     dev = new ${type.movingDeviationType}Holder();
     count = new ${type.countRunningType}Holder();
-    <#if type.inputType?starts_with("Nullable") >
-  	  nonNullCount = new BigIntHolder();
-  	  nonNullCount.value = 0;
-  	</#if>
+  	nonNullCount = new BigIntHolder();
+  	nonNullCount.value = 0;
     // Initialize the workspace variables
     avg.value = 0;
     dev.value = 0;
@@ -85,11 +81,9 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 	   // processing nullable input and the value is null, so don't do anything...
 	   break sout;
 	  }
-    else {
-      nonNullCount.value++;
-    }
 	</#if>
 
+    nonNullCount.value = 1;
     // Welford's approach to compute standard deviation
     double temp = avg.value;
     avg.value += ((in.value - temp) / count.value);
@@ -103,9 +97,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-   <#if type.inputType?starts_with("Nullable") >
-     if (nonNullCount.value > 0) {
-       out.isSet = 1;
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
       <#if aggrtype.funcName == "stddev_pop">
       if (count.value > 1)
         out.value = Math.sqrt((dev.value / (count.value - 1)));
@@ -121,29 +114,12 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
       </#if>
      } else {
        out.isSet = 0;
-     }
-   <#else> 
-    <#if aggrtype.funcName == "stddev_pop">
-    if (count.value > 1)
-      out.value = Math.sqrt((dev.value / (count.value - 1)));
-    <#elseif aggrtype.funcName == "var_pop">
-    if (count.value  > 1)
-      out.value = (dev.value / (count.value - 1));
-    <#elseif aggrtype.funcName == "stddev_samp">
-    if (count.value  > 2)
-      out.value = Math.sqrt((dev.value / (count.value - 2)));
-    <#elseif aggrtype.funcName == "var_samp">
-    if (count.value > 2)
-      out.value = (dev.value / (count.value - 2));
-    </#if>
-   </#if>
+    }
   }
 
   @Override
   public void reset() {
-    <#if type.inputType?starts_with("Nullable") >
-      nonNullCount.value = 0;
-    </#if>
+    nonNullCount.value = 0;
     avg.value = 0;
     dev.value = 0;
     count.value = 1;

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
index e934167..7480842 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
@@ -24,13 +24,11 @@
 
 <#include "/@includes/license.ftl" />
 
+// Source code generated using FreeMarker template ${.template_name}
+
 <#-- A utility class that is used to generate java code for aggr functions for Date, Time, Interval types -->
 <#--  that maintain a single running counter to hold the result.  This includes: MIN, MAX, SUM, COUNT. -->
 
-/*
- * This class is automatically generated from AggrTypeFunctions1.tdd using FreeMarker.
- */
-
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
 import org.apache.drill.exec.expr.DrillAggFunc;
@@ -52,10 +50,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Param ${type.inputType}Holder in;
   @Workspace ${type.runningType}Holder value;
+  @Workspace BigIntHolder nonNullCount;
   @Output ${type.outputType}Holder out;
 
   public void setup() {
-	value = new ${type.runningType}Holder();
+	  value = new ${type.runningType}Holder();
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
     <#if type.runningType == "Interval">
     value.months = ${type.initialValue};
     value.days= ${type.initialValue};
@@ -77,10 +78,10 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 		    break sout;
 	    }
 	  </#if>
-
+    nonNullCount.value = 1;
 	  <#if aggrtype.funcName == "min">
 
-    <#if type.outputType == "Interval">
+    <#if type.outputType?ends_with("Interval")>
 
     long inMS = (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis+
                        in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
@@ -88,7 +89,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
     value.value = Math.min(value.value, inMS);
 
-    <#elseif type.outputType == "IntervalDay">
+    <#elseif type.outputType?ends_with("IntervalDay")>
     long inMS = (long) in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
                        in.milliseconds;
 
@@ -99,13 +100,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
     value.value = Math.min(value.value, in.value);
     </#if>
 	  <#elseif aggrtype.funcName == "max">
-    <#if type.outputType == "Interval">
+    <#if type.outputType?ends_with("Interval")>
     long inMS = (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis+
                        in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
                        in.milliseconds;
 
     value.value = Math.max(value.value, inMS);
-    <#elseif type.outputType == "IntervalDay">
+    <#elseif type.outputType?ends_with("IntervalDay")>
     long inMS = (long) in.days * (org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis) +
                        in.milliseconds;
 
@@ -115,11 +116,11 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
     </#if>
 
 	  <#elseif aggrtype.funcName == "sum">
-    <#if type.outputType == "Interval">
+    <#if type.outputType?ends_with("Interval")>
     value.days += in.days;
     value.months += in.months;
     value.milliseconds += in.milliseconds;
-    <#elseif type.outputType == "IntervalDay">
+    <#elseif type.outputType?ends_with("IntervalDay")>
     value.days += in.days;
     value.milliseconds += in.milliseconds;
     <#else>
@@ -137,35 +138,40 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-
-    <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
-    <#if type.outputType == "Interval">
-    out.months = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
-    value.value = value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
-    out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    <#elseif type.outputType == "IntervalDay">
-    out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    <#else>
-    out.value = value.value;
-    </#if>
-    <#else>
-    <#if type.outputType == "Interval">
-    out.months = value.months;
-    out.days = value.days;
-    out.milliseconds = value.milliseconds;
-    <#elseif type.outputType == "IntervalDay">
-    out.days = value.days;
-    out.milliseconds = value.milliseconds;
-    <#else>
-    out.value = value.value;
-    </#if>
-    </#if>
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
+      <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
+      <#if type.outputType?ends_with("Interval")>
+      out.months = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
+      value.value = value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
+      out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      <#elseif type.outputType?ends_with("IntervalDay")>
+      out.days = (int) (value.value / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.milliseconds = (int) (value.value % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      <#else>
+      out.value = value.value;
+      </#if>
+      <#else>
+      <#if type.outputType?ends_with("Interval")>
+      out.months = value.months;
+      out.days = value.days;
+      out.milliseconds = value.milliseconds;
+      <#elseif type.outputType?ends_with("IntervalDay")>
+      out.days = value.days;
+      out.milliseconds = value.milliseconds;
+      <#else>
+      out.value = value.value;
+      </#if>
+      </#if>
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
+    nonNullCount.value = 0;
     <#if type.runningType == "Interval">
     value.months = ${type.initialValue};
     value.days= ${type.initialValue};

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
index b29fa08..6745097 100644
--- a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
@@ -24,14 +24,11 @@
 
 <#include "/@includes/license.ftl" />
 
+// Source code generated using FreeMarker template ${.template_name}
+
 <#-- A utility class that is used to generate java code for aggr functions for the interval data types. It maintains a running sum  -->
 <#-- and a running count.  For now, this includes: AVG. -->
 
-/*
- * This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
- */
-
-
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
 import org.apache.drill.exec.expr.DrillAggFunc;
@@ -57,11 +54,14 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   @Param ${type.inputType}Holder in;
   @Workspace ${type.sumRunningType}Holder sum;
   @Workspace ${type.countRunningType}Holder count;
+  @Workspace BigIntHolder nonNullCount;
   @Output ${type.outputType}Holder out;
 
   public void setup() {
   	sum = new ${type.sumRunningType}Holder();
     count = new ${type.countRunningType}Holder();
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
     sum.value = 0;
     count.value = 0;
   }
@@ -75,6 +75,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 		    break sout;
 	    }
 	  </#if>
+    nonNullCount.value = 1;
 	  <#if aggrtype.funcName == "avg">
     <#if type.inputType.endsWith("Interval")>
     sum.value += (long) in.months * org.apache.drill.exec.expr.fn.impl.DateUtility.monthToStandardDays +
@@ -98,21 +99,27 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-    double millis = sum.value / ((double) count.value);
-    <#if type.inputType.endsWith("Interval") || type.inputType.endsWith("IntervalYear")>
-    out.months = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
-    millis = millis % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
-    out.days =(int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    <#elseif type.inputType.endsWith("IntervalDay")>
-    out.months = 0;
-    out.days = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
-    </#if>
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
+      double millis = sum.value / ((double) count.value);
+      <#if type.inputType.endsWith("Interval") || type.inputType.endsWith("IntervalYear")>
+      out.months = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis);
+      millis = millis % org.apache.drill.exec.expr.fn.impl.DateUtility.monthsToMillis;
+      out.days =(int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      <#elseif type.inputType.endsWith("IntervalDay")>
+      out.months = 0;
+      out.days = (int) (millis / org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      out.milliseconds = (int) (millis % org.apache.drill.exec.expr.fn.impl.DateUtility.daysToStandardMillis);
+      </#if>
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
+    nonNullCount.value = 0;
     sum.value = 0;
     count.value = 0;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index 53474ea..14cea5b 100644
--- a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
@@ -24,13 +24,11 @@
 
 <#include "/@includes/license.ftl" />
 
+// Source code generated using FreeMarker template ${.template_name}
+
 <#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
 <#-- running counter to hold the result.  This includes: MIN, MAX, COUNT. -->
 
-/*
- * This class is automatically generated from VarCharAggrFunctions1.java using FreeMarker.
- */
-
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
 <#include "/@includes/vv_imports.ftl" />
@@ -60,27 +58,20 @@ public class ${aggrtype.className}VarBytesFunctions {
 public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
 
   @Param ${type.inputType}Holder in;
-  <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
   @Workspace ObjectHolder value;
-  @Workspace UInt1Holder init; 
+  @Workspace UInt1Holder init;
+  @Workspace BigIntHolder nonNullCount;
   @Inject DrillBuf buf;
-  <#else>
-  @Workspace  ${type.runningType}Holder value;
-  </#if>
   @Output ${type.outputType}Holder out;
 
   public void setup() {
-    <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
     init = new UInt1Holder();
+    nonNullCount = new BigIntHolder();
+    nonNullCount.value = 0;
     init.value = 0;
     value = new ObjectHolder();
     org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = new org.apache.drill.exec.expr.fn.impl.DrillByteArray();
     value.obj = tmp;
-
-    <#else>
-    value = new ${type.runningType}Holder();
-    value.value = 0;
-    </#if>
   }
 
   @Override
@@ -92,7 +83,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
         break sout;
       }
     </#if>
-    <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
+    nonNullCount.value = 1;
     org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) value.obj;
     int cmp = 0;
     boolean swap = false;
@@ -123,9 +114,6 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
         tmp.setBytes(tempArray);
       }
     }
-    <#else>
-    value.value++;
-    </#if>
     <#if type.inputType?starts_with("Nullable")>
     } // end of sout block
 	  </#if>
@@ -133,27 +121,24 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-    <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
-    org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) value.obj;
-    buf = buf.reallocIfNeeded(tmp.getLength());
-    buf.setBytes(0, tmp.getBytes(), 0, tmp.getLength());
-    out.start  = 0;
-    out.end    = tmp.getLength();
-    out.buffer = buf;
-    <#else>
-    out.value = value.value;
-    </#if>
+    if (nonNullCount.value > 0) {
+      out.isSet = 1;
+      org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) value.obj;
+      buf = buf.reallocIfNeeded(tmp.getLength());
+      buf.setBytes(0, tmp.getBytes(), 0, tmp.getLength());
+      out.start  = 0;
+      out.end    = tmp.getLength();
+      out.buffer = buf;
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
   public void reset() {
-    <#if aggrtype.funcName == "max" || aggrtype.funcName == "min">
     value = new ObjectHolder();
     init.value = 0;
-    <#else>
-    value = new ${type.runningType}Holder();
-    value.value = 0;
-    </#if>
+    nonNullCount.value = 0;
   }
 }
 </#if>

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
index 3925370..d39df8f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
@@ -178,7 +178,7 @@ public class JoinUtils {
         // currently we only support implicit casts if the input types are numeric or varchar/varbinary
         if (!allowImplicitCast(rightType, leftType)) {
           throw new DrillRuntimeException(String.format("Join only supports implicit casts between " +
-              "1. Numeric data\n 2. Varchar, Varbinary data " +
+              "1. Numeric data\n 2. Varchar, Varbinary data 3. Date, Timestamp data " +
               "Left type: %s, Right type: %s. Add explicit casts to avoid this error", leftType, rightType));
         }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index 01db7c2..69a7cbf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -207,7 +207,7 @@ public class TestAggregateFunctions extends BaseTestQuery {
         .go();
   }
 
-  @Test
+  @Test // test aggregates when input is empty and data type is optional
   public void testAggregateWithEmptyInput() throws Exception {
     String query = "select " +
         "count(employee_id) col1, avg(employee_id) col2, sum(employee_id) col3 " +
@@ -220,4 +220,73 @@ public class TestAggregateFunctions extends BaseTestQuery {
         .baselineValues(0l, null, null)
         .go();
   }
+
+  @Test // test aggregates when input is empty and data type is required
+  public void testAggregateWithEmptyRequiredInput() throws Exception {
+    // test min function on required type
+    String query = "select " +
+        "min(bool_col) col1, min(int_col) col2, min(bigint_col) col3, min(float4_col) col4, min(float8_col) col5, " +
+        "min(date_col) col6, min(time_col) col7, min(timestamp_col) col8, min(interval_year_col) col9, " +
+        "min(varhcar_col) col10 " +
+        "from cp.`parquet/alltypes_required.parquet` where 1 = 0";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7", "col8", "col9", "col10")
+        .baselineValues(null, null, null, null, null, null, null, null, null, null)
+        .go();
+
+    // test max function
+    query = "select " +
+        "max(int_col) col1, max(bigint_col) col2, max(float4_col) col3, max(float8_col) col4, " +
+        "max(date_col) col5, max(time_col) col6, max(timestamp_col) col7, max(interval_year_col) col8, " +
+        "max(varhcar_col) col9 " +
+        "from cp.`parquet/alltypes_required.parquet` where 1 = 0";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("col1", "col2", "col3", "col4", "col5", "col6", "col7", "col8", "col9")
+        .baselineValues(null, null, null, null, null, null, null, null, null)
+        .go();
+
+    // test sum function
+    query = "select " +
+        "sum(int_col) col1, sum(bigint_col) col2, sum(float4_col) col3, sum(float8_col) col4, sum(interval_year_col) col5 " +
+        "from cp.`employee.json` where 1 = 0";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("col1", "col2", "col3", "col4", "col5")
+        .baselineValues(null, null, null, null, null)
+        .go();
+
+    // test avg function
+    query = "select " +
+        "avg(int_col) col1, avg(bigint_col) col2, avg(float4_col) col3, avg(float8_col) col4, avg(interval_year_col) col5 " +
+        "from cp.`employee.json` where 1 = 0";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("col1", "col2", "col3", "col4", "col5")
+        .baselineValues(null, null, null, null, null)
+        .go();
+
+    // test stddev function
+    query = "select " +
+        "stddev_pop(int_col) col1, stddev_pop(bigint_col) col2, stddev_pop(float4_col) col3, " +
+        "stddev_pop(float8_col) col4, stddev_pop(interval_year_col) col5 " +
+        "from cp.`employee.json` where 1 = 0";
+
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("col1", "col2", "col3", "col4", "col5")
+        .baselineValues(null, null, null, null, null)
+        .go();
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
index b39566a..d2616a8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
@@ -42,6 +42,7 @@ import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.junit.Test;
 
 import com.codahale.metrics.MetricRegistry;
@@ -100,7 +101,7 @@ public class TestAgg extends ExecTest {
       IntVector key1 = exec.getValueVectorById(SchemaPath.getSimplePath("key1"), IntVector.class);
       BigIntVector key2 = exec.getValueVectorById(SchemaPath.getSimplePath("key2"), BigIntVector.class);
       BigIntVector cnt = exec.getValueVectorById(SchemaPath.getSimplePath("cnt"), BigIntVector.class);
-      BigIntVector total = exec.getValueVectorById(SchemaPath.getSimplePath("total"), BigIntVector.class);
+      NullableBigIntVector total = exec.getValueVectorById(SchemaPath.getSimplePath("total"), NullableBigIntVector.class);
       Integer[] keyArr1 = {Integer.MIN_VALUE, Integer.MIN_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE};
       long[] keyArr2 = {0,1,2,0,1,2};
       long[] cntArr = {34,34,34,34,34,34};

http://git-wip-us.apache.org/repos/asf/drill/blob/6a7de92e/exec/java-exec/src/test/resources/parquet/alltypes_required.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquet/alltypes_required.parquet b/exec/java-exec/src/test/resources/parquet/alltypes_required.parquet
new file mode 100644
index 0000000..549e316
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquet/alltypes_required.parquet differ


[08/12] drill git commit: DRILL-2697: Pauses sites wait indefinitely for a resume signal DrillClient sends a resume signal to UserServer. UserServer triggers a resume call in the correct Foreman. Foreman resumes all pauses related to the query through th

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/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 da69e9e..3e4dcb2 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
@@ -25,12 +25,13 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.math3.util.Pair;
 import org.apache.drill.QueryTestUtil;
 import org.apache.drill.SingleRowListener;
 import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -60,7 +61,9 @@ import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.store.pojo.PojoRecordReader;
 import org.apache.drill.exec.testing.ControlsInjectionUtil;
+import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.Foreman;
 import org.apache.drill.exec.work.foreman.ForemanException;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
@@ -68,15 +71,14 @@ import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 
 /**
  * Test how resilient drillbits are to throwing exceptions during various phases of query
- * execution by injecting exceptions at various points. The test cases are mentioned in DRILL-2383.
+ * execution by injecting exceptions at various points and to cancellations in various phases.
+ * The test cases are mentioned in DRILL-2383.
  */
-@Ignore
 public class TestDrillbitResilience {
   private static final Logger logger = org.slf4j.LoggerFactory.getLogger(TestDrillbitResilience.class);
 
@@ -90,7 +92,6 @@ public class TestDrillbitResilience {
    * counting sys.drillbits.
    */
   private static final String TEST_QUERY = "select * from sys.memory";
-  private static final long PAUSE_TIME_MILLIS = 3000L;
 
   private static void startDrillbit(final String name, final RemoteServiceSet remoteServiceSet) {
     if (drillbits.containsKey(name)) {
@@ -184,18 +185,17 @@ public class TestDrillbitResilience {
   }
 
   /**
-   * Clear all exceptions.
+   * Clear all injections.
    */
   private static void clearAllInjections() {
-    assertTrue(drillClient != null);
+    Preconditions.checkNotNull(drillClient);
     ControlsInjectionUtil.clearControls(drillClient);
   }
 
   /**
    * Check that all the drillbits are ok.
    * <p/>
-   * <p>The current implementation does this by counting the number of drillbits using a
-   * query.
+   * <p>The current implementation does this by counting the number of drillbits using a query.
    */
   private static void assertDrillbitsOk() {
       final SingleRowListener listener = new SingleRowListener() {
@@ -245,13 +245,14 @@ public class TestDrillbitResilience {
     try {
       QueryTestUtil.testWithListener(drillClient, QueryType.SQL, "select count(*) from sys.memory", listener);
       listener.waitForCompletion();
-      assertTrue(listener.getQueryState() == QueryState.COMPLETED);
+      final QueryState state = listener.getQueryState();
+      assertTrue(String.format("QueryState should be COMPLETED (and not %s).", state), state == QueryState.COMPLETED);
     } catch (final Exception e) {
       throw new RuntimeException("Couldn't query active drillbits", e);
     }
 
     final List<DrillPBError> errorList = listener.getErrorList();
-    assertTrue(errorList.isEmpty());
+    assertTrue("There should not be any errors when checking if Drillbits are OK.", errorList.isEmpty());
   }
 
   @SuppressWarnings("static-method")
@@ -262,10 +263,10 @@ public class TestDrillbitResilience {
   }
 
   /**
-   * Set the given exceptions.
+   * Set the given controls.
    */
-  private static void setExceptions(final String controlsString) {
-    ControlsInjectionUtil.setControls(drillClient, controlsString);
+  private static void setControls(final String controls) {
+    ControlsInjectionUtil.setControls(drillClient, controls);
   }
 
   /**
@@ -332,16 +333,16 @@ public class TestDrillbitResilience {
    */
   private static void assertExceptionInjected(final Throwable throwable,
                                               final Class<? extends Throwable> exceptionClass, final String desc) {
-    assertTrue(throwable instanceof UserException);
+    assertTrue("Throwable was not of UserException type.", throwable instanceof UserException);
     final ExceptionWrapper cause = ((UserException) throwable).getOrCreatePBError(false).getException();
-    assertEquals(exceptionClass.getName(), cause.getExceptionClass());
-    assertEquals(desc, cause.getMessage());
+    assertEquals("Exception class names should match.", exceptionClass.getName(), cause.getExceptionClass());
+    assertEquals("Exception sites should match.", desc, cause.getMessage());
   }
 
   @Test
   public void settingNoopInjectionsAndQuery() {
     final String controls = createSingleExceptionOnBit(getClass(), "noop", RuntimeException.class, DRILLBIT_BETA);
-    setExceptions(controls);
+    setControls(controls);
     try {
       QueryTestUtil.test(drillClient, TEST_QUERY);
     } catch (final Exception e) {
@@ -357,7 +358,7 @@ public class TestDrillbitResilience {
    */
   private static void testForeman(final String desc) {
     final String controls = createSingleException(Foreman.class, desc, ForemanException.class);
-    setExceptions(controls);
+    setControls(controls);
     try {
       QueryTestUtil.test(drillClient, TEST_QUERY);
       fail();
@@ -372,32 +373,39 @@ public class TestDrillbitResilience {
     testForeman("run-try-beginning");
   }
 
-  /*
-   * TODO I'm beginning to think that Foreman needs to gate output to its client in a similar way
-   * that it gates input via stateListener. That could be tricky, since some results could be
-   * queued up before Foreman has gotten through it's run(), and they would all have to be sent
-   * before the gate is opened. There's also the question of what to do in case we detect failure
-   * there after some data has been sent. Right now, this test doesn't work because that's
-   * exactly what happens, and the client believes that the query succeeded, even though an exception
-   * was thrown after setup completed, but data was asynchronously sent to the client before that.
-   * This test also revealed that the QueryState never seems to make it to the client, so we can't
-   * detect the failure that way (see SingleRowListener's getQueryState(), which I originally tried
-   * to use here to detect query completion).
-   */
   @SuppressWarnings("static-method")
   @Test
   public void foreman_runTryEnd() {
     testForeman("run-try-end");
   }
 
+  /**
+   * Tests can use this listener to wait, until the submitted query completes or fails, by
+   * calling #waitForCompletion.
+   */
   private static class WaitUntilCompleteListener implements UserResultsListener {
-    protected final CountDownLatch latch;
+    private final ExtendedLatch latch = new ExtendedLatch(1); // to signal completion
     protected QueryId queryId = null;
-    protected Exception ex = null;
-    protected QueryState state = null;
+    protected volatile Pointer<Exception> ex = new Pointer<>();
+    protected volatile QueryState state = null;
+
+    /**
+     * Method that sets the exception if the condition is not met.
+     */
+    protected final void check(final boolean condition, final String format, final Object... args) {
+      if (!condition) {
+        ex.value = new IllegalStateException(String.format(format, args));
+      }
+    }
 
-    public WaitUntilCompleteListener(final int count) {
-      latch = new CountDownLatch(count);
+    /**
+     * Method that cancels and resumes the query, in order.
+     */
+    protected final void cancelAndResume() {
+      Preconditions.checkNotNull(queryId);
+      final ExtendedLatch trigger = new ExtendedLatch(1);
+      (new CancellingThread(queryId, ex, trigger)).start();
+      (new ResumingThread(queryId, ex, trigger)).start();
     }
 
     @Override
@@ -407,7 +415,7 @@ public class TestDrillbitResilience {
 
     @Override
     public void submissionFailed(final UserException ex) {
-      this.ex = ex;
+      this.ex.value = ex;
       state = QueryState.FAILED;
       latch.countDown();
     }
@@ -424,21 +432,23 @@ public class TestDrillbitResilience {
     }
 
     public final Pair<QueryState, Exception> waitForCompletion() {
-      try {
-        latch.await();
-      } catch (final InterruptedException e) {
-        return new Pair<QueryState, Exception>(state, e);
-      }
-      return new Pair<>(state, ex);
+      latch.awaitUninterruptibly();
+      return new Pair<>(state, ex.value);
     }
   }
 
+  /**
+   * Thread that cancels the given query id. After the cancel is acknowledged, the latch is counted down.
+   */
   private static class CancellingThread extends Thread {
-
     private final QueryId queryId;
+    private final Pointer<Exception> ex;
+    private final ExtendedLatch latch;
 
-    public CancellingThread(final QueryId queryId) {
+    public CancellingThread(final QueryId queryId, final Pointer<Exception> ex, final ExtendedLatch latch) {
       this.queryId = queryId;
+      this.ex = ex;
+      this.latch = latch;
     }
 
     @Override
@@ -446,139 +456,178 @@ public class TestDrillbitResilience {
       final DrillRpcFuture<Ack> cancelAck = drillClient.cancelQuery(queryId);
       try {
         cancelAck.checkedGet();
-      } catch (final RpcException e) {
-        fail(e.getMessage()); // currently this failure does not fail the test
+      } catch (final RpcException ex) {
+        this.ex.value = ex;
       }
+      latch.countDown();
+    }
+  }
+
+  /**
+   * Thread that resumes the given query id. After the latch is counted down, the resume signal is sent, until then
+   * the thread waits without interruption.
+   */
+  private static class ResumingThread extends Thread {
+    private final QueryId queryId;
+    private final Pointer<Exception> ex;
+    private final ExtendedLatch latch;
+
+    public ResumingThread(final QueryId queryId, final Pointer<Exception> ex, final ExtendedLatch latch) {
+      this.queryId = queryId;
+      this.ex = ex;
+      this.latch = latch;
+    }
+
+    @Override
+    public void run() {
+      latch.awaitUninterruptibly();
+      final DrillRpcFuture<Ack> resumeAck = drillClient.resumeQuery(queryId);
+      try {
+        resumeAck.checkedGet();
+      } catch (final RpcException ex) {
+        this.ex.value = ex;
+      }
+    }
+  }
+
+  /**
+   * Given the result of {@link WaitUntilCompleteListener#waitForCompletion}, this method fails if the state is not
+   * as expected or if an exception is thrown.
+   */
+  private static void assertCompleteState(final Pair<QueryState, Exception> result, final QueryState expectedState) {
+    final QueryState actualState = result.getFirst();
+    final Exception exception = result.getSecond();
+    if (actualState != expectedState || exception != null) {
+      fail(String.format("Query state is incorrect (expected: %s, actual: %s) AND/OR \nException thrown: %s",
+        expectedState, actualState, exception == null ? "none." : exception));
     }
   }
 
   /**
    * Given a set of controls, this method ensures that the TEST_QUERY completes with a CANCELED state.
    */
-  private static void assertCancelled(final String controls, final WaitUntilCompleteListener listener) {
-    ControlsInjectionUtil.setControls(drillClient, controls);
+  private static void assertCancelledWithoutException(final String controls, final WaitUntilCompleteListener listener) {
+    setControls(controls);
 
     QueryTestUtil.testWithListener(drillClient, QueryType.SQL, TEST_QUERY, listener);
     final Pair<QueryState, Exception> result = listener.waitForCompletion();
-    assertTrue(String.format("Expected Query Outcome of CANCELED but had Outcome of %s", result.getFirst()),
-        result.getFirst() == QueryState.CANCELED);
-    assertTrue(String.format("Expected no Exception but had Exception %s", result.getSecond()),
-        result.getSecond() == null);
+    assertCompleteState(result, QueryState.CANCELED);
   }
 
-  @Test // Cancellation TC 1
-  public void cancelBeforeAnyResultsArrive() {
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1) {
+  private static String createPauseInjection(final Class siteClass, final String siteDesc, final int nSkip) {
+    return "{\"injections\" : [{"
+      + "\"type\" : \"pause\"," +
+      "\"siteClass\" : \"" + siteClass.getName() + "\","
+      + "\"desc\" : \"" + siteDesc + "\","
+      + "\"nSkip\" : " + nSkip
+      + "}]}";
+  }
+
+  private static String createPauseInjection(final Class siteClass, final String siteDesc) {
+    return createPauseInjection(siteClass, siteDesc, 0);
+  }
 
+  @Test // To test pause and resume. Test hangs if resume did not happen.
+  public void passThrough() {
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener() {
       @Override
       public void queryIdArrived(final QueryId queryId) {
-        (new CancellingThread(queryId)).start();
+        super.queryIdArrived(queryId);
+        final ExtendedLatch trigger = new ExtendedLatch(1);
+        (new ResumingThread(queryId, ex, trigger)).start();
+        trigger.countDown();
       }
     };
 
-    final String controls = "{\"injections\":[{"
-      + "\"type\":\"pause\"," +
-      "\"siteClass\":\"" + Foreman.class.getName() + "\","
-      + "\"desc\":\"pause-run-plan\","
-      + "\"millis\":" + PAUSE_TIME_MILLIS + ","
-      + "\"nSkip\":0,"
-      + "\"nFire\":1"
-      + "}]}";
+    final String controls = createPauseInjection(PojoRecordReader.class, "read-next");
+    setControls(controls);
+
+    QueryTestUtil.testWithListener(drillClient, QueryType.SQL, TEST_QUERY, listener);
+    final Pair<QueryState, Exception> result = listener.waitForCompletion();
+    assertCompleteState(result, QueryState.COMPLETED);
+  }
+
+  @Test // Cancellation TC 1: cancel before any result set is returned
+  public void cancelBeforeAnyResultsArrive() {
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener() {
 
-    assertCancelled(controls, listener);
+      @Override
+      public void queryIdArrived(final QueryId queryId) {
+        super.queryIdArrived(queryId);
+        cancelAndResume();
+      }
+    };
+
+    final String controls = createPauseInjection(Foreman.class, "foreman-ready");
+    assertCancelledWithoutException(controls, listener);
   }
 
-  @Test // Cancellation TC 2
+  @Test // Cancellation TC 2: cancel in the middle of fetching result set
   public void cancelInMiddleOfFetchingResults() {
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1) {
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener() {
       private boolean cancelRequested = false;
 
       @Override
-      public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
-        if (! cancelRequested) {
-          assertTrue(queryId != null);
-          (new CancellingThread(queryId)).start();
+      public void dataArrived(final QueryDataBatch result, final ConnectionThrottle throttle) {
+        if (!cancelRequested) {
+          check(queryId != null, "Query id should not be null, since we have waited long enough.");
+          cancelAndResume();
           cancelRequested = true;
         }
         result.release();
       }
     };
 
-    final String controls = "{\"injections\":[{"
-      + "\"type\":\"pause\"," +
-      "\"siteClass\":\"" + ScreenCreator.class.getName() + "\","
-      + "\"desc\":\"sending-data\","
-      + "\"millis\":" + PAUSE_TIME_MILLIS + ","
-      + "\"nSkip\":0,"
-      + "\"nFire\":1"
-      + "}]}";
-
-    assertCancelled(controls, listener);
+    // skip once i.e. wait for one batch, so that #dataArrived above triggers #cancelAndResume
+    final String controls = createPauseInjection(ScreenCreator.class, "sending-data", 1);
+    assertCancelledWithoutException(controls, listener);
   }
 
 
-  @Test // Cancellation TC 3
+  @Test // Cancellation TC 3: cancel after all result set are produced but not all are fetched
   public void cancelAfterAllResultsProduced() {
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1) {
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener() {
       private int count = 0;
 
       @Override
-      public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+      public void dataArrived(final QueryDataBatch result, final ConnectionThrottle throttle) {
         if (++count == drillbits.size()) {
-          assertTrue(queryId != null);
-          (new CancellingThread(queryId)).start();
+          check(queryId != null, "Query id should not be null, since we have waited long enough.");
+          cancelAndResume();
         }
         result.release();
       }
     };
 
-    final String controls = "{\"injections\":[{"
-      + "\"type\":\"pause\"," +
-      "\"siteClass\":\"" + ScreenCreator.class.getName() + "\","
-      + "\"desc\":\"send-complete\","
-      + "\"millis\":" + PAUSE_TIME_MILLIS + ","
-      + "\"nSkip\":0,"
-      + "\"nFire\":1"
-      + "}]}";
-
-    assertCancelled(controls, listener);
+    final String controls = createPauseInjection(ScreenCreator.class, "send-complete");
+    assertCancelledWithoutException(controls, listener);
   }
 
-  @Test // Cancellation TC 4
+  @Test // Cancellation TC 4: cancel after everything is completed and fetched
   public void cancelAfterEverythingIsCompleted() {
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1) {
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener() {
       private int count = 0;
 
       @Override
-      public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
+      public void dataArrived(final QueryDataBatch result, final ConnectionThrottle throttle) {
         if (++count == drillbits.size()) {
-          assertTrue(queryId != null);
-          (new CancellingThread(queryId)).start();
+          check(queryId != null, "Query id should not be null, since we have waited long enough.");
+          cancelAndResume();
         }
         result.release();
       }
     };
 
-    final String controls = "{\"injections\":[{"
-      + "\"type\":\"pause\"," +
-      "\"siteClass\":\"" + Foreman.class.getName() + "\","
-      + "\"desc\":\"foreman-cleanup\","
-      + "\"millis\":" + PAUSE_TIME_MILLIS + ","
-      + "\"nSkip\":0,"
-      + "\"nFire\":1"
-      + "}]}";
-
-    assertCancelled(controls, listener);
+    final String controls = createPauseInjection(Foreman.class, "foreman-cleanup");
+    assertCancelledWithoutException(controls, listener);
   }
 
-  @Test // Completion TC 1
+  @Test // Completion TC 1: success
   public void successfullyCompletes() {
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1);
-    QueryTestUtil.testWithListener(
-      drillClient, QueryType.SQL, TEST_QUERY, listener);
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener();
+    QueryTestUtil.testWithListener(drillClient, QueryType.SQL, TEST_QUERY, listener);
     final Pair<QueryState, Exception> result = listener.waitForCompletion();
-    assertTrue(result.getFirst() == QueryState.COMPLETED);
-    assertTrue(result.getSecond() == null);
+    assertCompleteState(result, QueryState.COMPLETED);
   }
 
   /**
@@ -586,16 +635,16 @@ public class TestDrillbitResilience {
    */
   private static void assertFailsWithException(final String controls, final Class<? extends Throwable> exceptionClass,
                                                final String exceptionDesc) {
-    setExceptions(controls);
-    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener(1);
-    QueryTestUtil.testWithListener(drillClient, QueryType.SQL,  TEST_QUERY, listener);
+    setControls(controls);
+    final WaitUntilCompleteListener listener = new WaitUntilCompleteListener();
+    QueryTestUtil.testWithListener(drillClient, QueryType.SQL, TEST_QUERY, listener);
     final Pair<QueryState, Exception> result = listener.waitForCompletion();
-    assertTrue(result.getFirst() == QueryState.FAILED);
-    final Exception e = result.getSecond();
-    assertExceptionInjected(e, exceptionClass, exceptionDesc);
+    final QueryState state = result.getFirst();
+    assertTrue(String.format("Query state should be FAILED (and not %s).", state), state == QueryState.FAILED);
+    assertExceptionInjected(result.getSecond(), exceptionClass, exceptionDesc);
   }
 
-  @Test // Completion TC 2
+  @Test // Completion TC 2: failed query - before query is executed - while sql parsing
   public void failsWhenParsing() {
     final String exceptionDesc = "sql-parsing";
     final Class<? extends Throwable> exceptionClass = ForemanSetupException.class;
@@ -603,7 +652,7 @@ public class TestDrillbitResilience {
     assertFailsWithException(controls, exceptionClass, exceptionDesc);
   }
 
-  @Test // Completion TC 3
+  @Test // Completion TC 3: failed query - before query is executed - while sending fragments to other drillbits
   public void failsWhenSendingFragments() {
     final String exceptionDesc = "send-fragments";
     final Class<? extends Throwable> exceptionClass = ForemanException.class;
@@ -611,7 +660,7 @@ public class TestDrillbitResilience {
     assertFailsWithException(controls, exceptionClass, exceptionDesc);
   }
 
-  @Test // Completion TC 4
+  @Test // Completion TC 4: failed query - during query execution
   public void failsDuringExecution() {
     final String exceptionDesc = "fragment-execution";
     final Class<? extends Throwable> exceptionClass = IOException.class;

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
new file mode 100644
index 0000000..c98f54c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
@@ -0,0 +1,155 @@
+/**
+ * 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.testing;
+
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.concurrent.ExtendedLatch;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
+import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.util.Pointer;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestCountDownLatchInjection extends BaseTestQuery {
+
+  private static final UserSession session = UserSession.Builder.newBuilder()
+    .withCredentials(UserCredentials.newBuilder()
+      .setUserName("foo")
+      .build())
+    .withUserProperties(UserProperties.getDefaultInstance())
+    .withOptionManager(bits[0].getContext().getOptionManager())
+    .build();
+
+  /**
+   * Class whose methods we want to simulate count down latches at run-time for testing
+   * purposes. The class must have access to {@link org.apache.drill.exec.ops.QueryContext} or
+   * {@link org.apache.drill.exec.ops.FragmentContext}.
+   */
+  private static class DummyClass {
+    private static final ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DummyClass.class);
+
+    private final QueryContext context;
+    private final CountDownLatch latch;
+    private final int count;
+
+    public DummyClass(final QueryContext context, final CountDownLatch latch, final int count) {
+      this.context = context;
+      this.latch = latch;
+      this.count = count;
+    }
+
+    public static final String LATCH_NAME = "<<latch>>";
+
+    /**
+     * Method that initializes and waits for "count" number of count down (from those many threads)
+     */
+    public long initAndWait() throws InterruptedException {
+      // ... code ...
+
+      injector.getLatch(context.getExecutionControls(), LATCH_NAME).initialize(count);
+
+      // ... code ...
+      latch.countDown(); // trigger threads spawn
+
+      final long startTime = System.currentTimeMillis();
+      // simulated wait for "count" threads to count down on the same latch
+      injector.getLatch(context.getExecutionControls(), LATCH_NAME).await();
+      final long endTime = System.currentTimeMillis();
+      // ... code ...
+      return (endTime - startTime);
+    }
+
+    public void countDown() {
+      // ... code ...
+      injector.getLatch(context.getExecutionControls(), LATCH_NAME).countDown();
+      // ... code ...
+    }
+  }
+
+  private static class ThreadCreator extends Thread {
+
+    private final DummyClass dummyClass;
+    private final ExtendedLatch latch;
+    private final int count;
+    private final Pointer<Long> countingDownTime;
+
+    public ThreadCreator(final DummyClass dummyClass, final ExtendedLatch latch, final int count,
+                         final Pointer<Long> countingDownTime) {
+      this.dummyClass = dummyClass;
+      this.latch = latch;
+      this.count = count;
+      this.countingDownTime = countingDownTime;
+    }
+
+    @Override
+    public void run() {
+      latch.awaitUninterruptibly();
+      final long startTime = System.currentTimeMillis();
+      for (int i = 0; i < count; i++) {
+        (new Thread() {
+          @Override
+          public void run() {
+            dummyClass.countDown();
+          }
+        }).start();
+      }
+      final long endTime = System.currentTimeMillis();
+      countingDownTime.value = (endTime - startTime);
+    }
+  }
+
+  @Test // test would hang if the correct init, wait and countdowns did not happen, and the test timeout mechanism will
+  // catch that case
+  public void latchInjected() {
+    final int threads = 10;
+    final ExtendedLatch trigger = new ExtendedLatch(1);
+    final Pointer<Long> countingDownTime = new Pointer<>();
+
+    final String jsonString = "{\"injections\":[{"
+      + "\"type\":\"latch\"," +
+      "\"siteClass\":\"org.apache.drill.exec.testing.TestCountDownLatchInjection$DummyClass\","
+      + "\"desc\":\"" + DummyClass.LATCH_NAME + "\""
+      + "}]}";
+
+    ControlsInjectionUtil.setControls(session, jsonString);
+
+    final QueryContext queryContext = new QueryContext(session, bits[0].getContext());
+
+    final DummyClass dummyClass = new DummyClass(queryContext, trigger, threads);
+    (new ThreadCreator(dummyClass, trigger, threads, countingDownTime)).start();
+    final long timeSpentWaiting;
+    try {
+      timeSpentWaiting = dummyClass.initAndWait();
+    } catch (final InterruptedException e) {
+      fail("Thread should not be interrupted; there is no deliberate attempt.");
+      return;
+    }
+    assertTrue(timeSpentWaiting >= countingDownTime.value);
+    try {
+      queryContext.close();
+    } catch (final Exception e) {
+      fail("Failed to close query context: " + e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
index 5fa2b3f..ba29c58 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
@@ -18,20 +18,33 @@
 package org.apache.drill.exec.testing;
 
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.concurrent.ExtendedLatch;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ZookeeperHelper;
+import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.util.Pointer;
 import org.junit.Test;
 import org.slf4j.Logger;
 
+import java.util.concurrent.CountDownLatch;
+
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TestPauseInjection extends BaseTestQuery {
 
   private static final UserSession session = UserSession.Builder.newBuilder()
-      .withCredentials(UserBitShared.UserCredentials.newBuilder().setUserName("foo").build())
+      .withCredentials(UserCredentials.newBuilder()
+        .setUserName("foo")
+        .build())
       .withUserProperties(UserProperties.getDefaultInstance())
       .withOptionManager(bits[0].getContext().getOptionManager())
       .build();
@@ -46,9 +59,11 @@ public class TestPauseInjection extends BaseTestQuery {
     private static final ExecutionControlsInjector injector = ExecutionControlsInjector.getInjector(DummyClass.class);
 
     private final QueryContext context;
+    private final CountDownLatch latch;
 
-    public DummyClass(final QueryContext context) {
+    public DummyClass(final QueryContext context, final CountDownLatch latch) {
       this.context = context;
+      this.latch = latch;
     }
 
     public static final String PAUSES = "<<pauses>>";
@@ -61,6 +76,7 @@ public class TestPauseInjection extends BaseTestQuery {
     public long pauses() {
       // ... code ...
 
+      latch.countDown();
       final long startTime = System.currentTimeMillis();
       // simulated pause
       injector.injectPause(context.getExecutionControls(), PAUSES, logger);
@@ -71,30 +87,136 @@ public class TestPauseInjection extends BaseTestQuery {
     }
   }
 
+  private static class ResumingThread extends Thread {
+
+    private final QueryContext context;
+    private final ExtendedLatch latch;
+    private final Pointer<Exception> ex;
+    private final long millis;
+
+    public ResumingThread(final QueryContext context, final ExtendedLatch latch, final Pointer<Exception> ex,
+                          final long millis) {
+      this.context = context;
+      this.latch = latch;
+      this.ex = ex;
+      this.millis = millis;
+    }
+
+    @Override
+    public void run() {
+      latch.awaitUninterruptibly();
+      try {
+        Thread.sleep(millis);
+      } catch (final InterruptedException ex) {
+        this.ex.value = ex;
+      }
+      context.getExecutionControls().unpauseAll();
+    }
+  }
+
   @Test
   public void pauseInjected() {
-    final long pauseMillis = 1000L;
+    final long expectedDuration = 1000L;
+    final ExtendedLatch trigger = new ExtendedLatch(1);
+    final Pointer<Exception> ex = new Pointer<>();
+
     final String jsonString = "{\"injections\":[{"
       + "\"type\":\"pause\"," +
       "\"siteClass\":\"org.apache.drill.exec.testing.TestPauseInjection$DummyClass\","
       + "\"desc\":\"" + DummyClass.PAUSES + "\","
-      + "\"millis\":" + pauseMillis + ","
-      + "\"nSkip\":0,"
-      + "\"nFire\":1"
+      + "\"nSkip\":0"
       + "}]}";
 
     ControlsInjectionUtil.setControls(session, jsonString);
 
     final QueryContext queryContext = new QueryContext(session, bits[0].getContext());
 
+    (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
+
     // test that the pause happens
-    final DummyClass dummyClass = new DummyClass(queryContext);
-    final long time = dummyClass.pauses();
-    assertTrue((time >= pauseMillis));
+    final DummyClass dummyClass = new DummyClass(queryContext, trigger);
+    final long actualDuration = dummyClass.pauses();
+    assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration),
+      expectedDuration <= actualDuration);
+    assertTrue("No exception should be thrown.", ex.value == null);
     try {
       queryContext.close();
-    } catch (Exception e) {
-      fail();
+    } catch (final Exception e) {
+      fail("Failed to close query context: " + e);
+    }
+  }
+
+  @Test
+  public void pauseOnSpecificBit() {
+    final RemoteServiceSet remoteServiceSet = RemoteServiceSet.getLocalServiceSet();
+    final ZookeeperHelper zkHelper = new ZookeeperHelper();
+    zkHelper.startZookeeper(1);
+
+    // Creating two drillbits
+    final Drillbit drillbit1, drillbit2;
+    final DrillConfig drillConfig = zkHelper.getConfig();
+    try {
+      drillbit1 = Drillbit.start(drillConfig, remoteServiceSet);
+      drillbit2 = Drillbit.start(drillConfig, remoteServiceSet);
+    } catch (final DrillbitStartupException e) {
+      throw new RuntimeException("Failed to start two drillbits.", e);
+    }
+
+    final DrillbitContext drillbitContext1 = drillbit1.getContext();
+    final DrillbitContext drillbitContext2 = drillbit2.getContext();
+
+    final UserSession session = UserSession.Builder.newBuilder()
+      .withCredentials(UserCredentials.newBuilder()
+        .setUserName("foo")
+        .build())
+      .withUserProperties(UserProperties.getDefaultInstance())
+      .withOptionManager(drillbitContext1.getOptionManager())
+      .build();
+
+    final DrillbitEndpoint drillbitEndpoint1 = drillbitContext1.getEndpoint();
+    final String jsonString = "{\"injections\":[{"
+      + "\"type\" : \"pause\"," +
+      "\"siteClass\" : \"org.apache.drill.exec.testing.TestPauseInjection$DummyClass\","
+      + "\"desc\" : \"" + DummyClass.PAUSES + "\","
+      + "\"nSkip\" : 0, "
+      + "\"address\" : \"" + drillbitEndpoint1.getAddress() + "\","
+      + "\"port\" : " + drillbitEndpoint1.getUserPort()
+      + "}]}";
+
+    ControlsInjectionUtil.setControls(session, jsonString);
+
+    {
+      final long expectedDuration = 1000L;
+      final ExtendedLatch trigger = new ExtendedLatch(1);
+      final Pointer<Exception> ex = new Pointer<>();
+      final QueryContext queryContext = new QueryContext(session, drillbitContext1);
+      (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
+
+      // test that the pause happens
+      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
+      final long actualDuration = dummyClass.pauses();
+      assertTrue(String.format("Test should stop for at least %d milliseconds.", expectedDuration),
+        expectedDuration <= actualDuration);
+      assertTrue("No exception should be thrown.", ex.value == null);
+      try {
+        queryContext.close();
+      } catch (final Exception e) {
+        fail("Failed to close query context: " + e);
+      }
+    }
+
+    {
+      final ExtendedLatch trigger = new ExtendedLatch(1);
+      final QueryContext queryContext = new QueryContext(session, drillbitContext2);
+
+      // if the resume did not happen, the test would hang
+      final DummyClass dummyClass = new DummyClass(queryContext, trigger);
+      dummyClass.pauses();
+      try {
+        queryContext.close();
+      } catch (final Exception e) {
+        fail("Failed to close query context: " + e);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
index 470e976..b428337 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -47,13 +47,13 @@ public final class BitControl {
      */
     GOODBYE(2, 2),
     /**
-     * <code>REQ_INIATILIZE_FRAGMENTS = 3;</code>
+     * <code>REQ_INITIALIZE_FRAGMENTS = 3;</code>
      *
      * <pre>
      * bit requests
      * </pre>
      */
-    REQ_INIATILIZE_FRAGMENTS(3, 3),
+    REQ_INITIALIZE_FRAGMENTS(3, 3),
     /**
      * <code>REQ_CANCEL_FRAGMENT = 6;</code>
      *
@@ -91,25 +91,33 @@ public final class BitControl {
      */
     REQ_QUERY_CANCEL(9, 15),
     /**
+     * <code>REQ_UNPAUSE_FRAGMENT = 16;</code>
+     *
+     * <pre>
+     * send a resume message for a fragment, returns Ack
+     * </pre>
+     */
+    REQ_UNPAUSE_FRAGMENT(10, 16),
+    /**
      * <code>RESP_FRAGMENT_HANDLE = 11;</code>
      *
      * <pre>
      * bit responses
      * </pre>
      */
-    RESP_FRAGMENT_HANDLE(10, 11),
+    RESP_FRAGMENT_HANDLE(11, 11),
     /**
      * <code>RESP_FRAGMENT_STATUS = 12;</code>
      */
-    RESP_FRAGMENT_STATUS(11, 12),
+    RESP_FRAGMENT_STATUS(12, 12),
     /**
      * <code>RESP_BIT_STATUS = 13;</code>
      */
-    RESP_BIT_STATUS(12, 13),
+    RESP_BIT_STATUS(13, 13),
     /**
      * <code>RESP_QUERY_STATUS = 14;</code>
      */
-    RESP_QUERY_STATUS(13, 14),
+    RESP_QUERY_STATUS(14, 14),
     ;
 
     /**
@@ -125,13 +133,13 @@ public final class BitControl {
      */
     public static final int GOODBYE_VALUE = 2;
     /**
-     * <code>REQ_INIATILIZE_FRAGMENTS = 3;</code>
+     * <code>REQ_INITIALIZE_FRAGMENTS = 3;</code>
      *
      * <pre>
      * bit requests
      * </pre>
      */
-    public static final int REQ_INIATILIZE_FRAGMENTS_VALUE = 3;
+    public static final int REQ_INITIALIZE_FRAGMENTS_VALUE = 3;
     /**
      * <code>REQ_CANCEL_FRAGMENT = 6;</code>
      *
@@ -169,6 +177,14 @@ public final class BitControl {
      */
     public static final int REQ_QUERY_CANCEL_VALUE = 15;
     /**
+     * <code>REQ_UNPAUSE_FRAGMENT = 16;</code>
+     *
+     * <pre>
+     * send a resume message for a fragment, returns Ack
+     * </pre>
+     */
+    public static final int REQ_UNPAUSE_FRAGMENT_VALUE = 16;
+    /**
      * <code>RESP_FRAGMENT_HANDLE = 11;</code>
      *
      * <pre>
@@ -197,13 +213,14 @@ public final class BitControl {
         case 0: return HANDSHAKE;
         case 1: return ACK;
         case 2: return GOODBYE;
-        case 3: return REQ_INIATILIZE_FRAGMENTS;
+        case 3: return REQ_INITIALIZE_FRAGMENTS;
         case 6: return REQ_CANCEL_FRAGMENT;
         case 7: return REQ_RECEIVER_FINISHED;
         case 8: return REQ_FRAGMENT_STATUS;
         case 9: return REQ_BIT_STATUS;
         case 10: return REQ_QUERY_STATUS;
         case 15: return REQ_QUERY_CANCEL;
+        case 16: return REQ_UNPAUSE_FRAGMENT;
         case 11: return RESP_FRAGMENT_HANDLE;
         case 12: return RESP_FRAGMENT_STATUS;
         case 13: return RESP_BIT_STATUS;
@@ -7395,16 +7412,17 @@ public final class BitControl {
       "oint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report_tim" +
       "e\030\003 \001(\003\"h\n\020FinishedReceiver\022*\n\010receiver\030",
       "\001 \001(\0132\030.exec.bit.FragmentHandle\022(\n\006sende" +
-      "r\030\002 \001(\0132\030.exec.bit.FragmentHandle*\271\002\n\007Rp" +
+      "r\030\002 \001(\0132\030.exec.bit.FragmentHandle*\323\002\n\007Rp" +
       "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
-      "\020\002\022\034\n\030REQ_INIATILIZE_FRAGMENTS\020\003\022\027\n\023REQ_" +
+      "\020\002\022\034\n\030REQ_INITIALIZE_FRAGMENTS\020\003\022\027\n\023REQ_" +
       "CANCEL_FRAGMENT\020\006\022\031\n\025REQ_RECEIVER_FINISH" +
       "ED\020\007\022\027\n\023REQ_FRAGMENT_STATUS\020\010\022\022\n\016REQ_BIT" +
       "_STATUS\020\t\022\024\n\020REQ_QUERY_STATUS\020\n\022\024\n\020REQ_Q" +
-      "UERY_CANCEL\020\017\022\030\n\024RESP_FRAGMENT_HANDLE\020\013\022" +
-      "\030\n\024RESP_FRAGMENT_STATUS\020\014\022\023\n\017RESP_BIT_ST" +
-      "ATUS\020\r\022\025\n\021RESP_QUERY_STATUS\020\016B+\n\033org.apa",
-      "che.drill.exec.protoB\nBitControlH\001"
+      "UERY_CANCEL\020\017\022\030\n\024REQ_UNPAUSE_FRAGMENT\020\020\022" +
+      "\030\n\024RESP_FRAGMENT_HANDLE\020\013\022\030\n\024RESP_FRAGME" +
+      "NT_STATUS\020\014\022\023\n\017RESP_BIT_STATUS\020\r\022\025\n\021RESP",
+      "_QUERY_STATUS\020\016B+\n\033org.apache.drill.exec" +
+      ".protoB\nBitControlH\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/52dcd7e4/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 c3ff58b..afe8bfe 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
@@ -67,25 +67,33 @@ public final class UserProtos {
      */
     REQUEST_RESULTS(5, 5),
     /**
+     * <code>RESUME_PAUSED_QUERY = 11;</code>
+     *
+     * <pre>
+     * user is sending a query resume request to the drillbit
+     * </pre>
+     */
+    RESUME_PAUSED_QUERY(6, 11),
+    /**
      * <code>QUERY_DATA = 6;</code>
      *
      * <pre>
      * bit to user
      * </pre>
      */
-    QUERY_DATA(6, 6),
+    QUERY_DATA(7, 6),
     /**
      * <code>QUERY_HANDLE = 7;</code>
      */
-    QUERY_HANDLE(7, 7),
+    QUERY_HANDLE(8, 7),
     /**
      * <code>REQ_META_FUNCTIONS = 8;</code>
      */
-    REQ_META_FUNCTIONS(8, 8),
+    REQ_META_FUNCTIONS(9, 8),
     /**
      * <code>RESP_FUNCTION_LIST = 9;</code>
      */
-    RESP_FUNCTION_LIST(9, 9),
+    RESP_FUNCTION_LIST(10, 9),
     /**
      * <code>QUERY_RESULT = 10;</code>
      *
@@ -93,7 +101,7 @@ public final class UserProtos {
      * drillbit is reporting a query status change, most likely a terminal message, to the user
      * </pre>
      */
-    QUERY_RESULT(10, 10),
+    QUERY_RESULT(11, 10),
     ;
 
     /**
@@ -129,6 +137,14 @@ public final class UserProtos {
      */
     public static final int REQUEST_RESULTS_VALUE = 5;
     /**
+     * <code>RESUME_PAUSED_QUERY = 11;</code>
+     *
+     * <pre>
+     * user is sending a query resume request to the drillbit
+     * </pre>
+     */
+    public static final int RESUME_PAUSED_QUERY_VALUE = 11;
+    /**
      * <code>QUERY_DATA = 6;</code>
      *
      * <pre>
@@ -168,6 +184,7 @@ public final class UserProtos {
         case 3: return RUN_QUERY;
         case 4: return CANCEL_QUERY;
         case 5: return REQUEST_RESULTS;
+        case 11: return RESUME_PAUSED_QUERY;
         case 6: return QUERY_DATA;
         case 7: return QUERY_HANDLE;
         case 8: return REQ_META_FUNCTIONS;
@@ -4986,16 +5003,17 @@ public final class UserProtos {
       "n\030\003 \001(\t\"|\n\022BitToUserHandshake\022\023\n\013rpc_ver" +
       "sion\030\002 \001(\005\022*\n\006status\030\003 \001(\0162\032.exec.user.H" +
       "andshakeStatus\022\017\n\007errorId\030\004 \001(\t\022\024\n\014error" +
-      "Message\030\005 \001(\t*\310\001\n\007RpcType\022\r\n\tHANDSHAKE\020\000",
+      "Message\030\005 \001(\t*\341\001\n\007RpcType\022\r\n\tHANDSHAKE\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\n" +
-      "QUERY_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n\022REQ_ME" +
-      "TA_FUNCTIONS\020\010\022\026\n\022RESP_FUNCTION_LIST\020\t\022\020" +
-      "\n\014QUERY_RESULT\020\n*#\n\020QueryResultsMode\022\017\n\013" +
-      "STREAM_FULL\020\001*^\n\017HandshakeStatus\022\013\n\007SUCC" +
-      "ESS\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.apa" +
-      "che.drill.exec.protoB\nUserProtosH\001"
+      "\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020\005\022\027\n\023" +
+      "RESUME_PAUSED_QUERY\020\013\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\020QueryResultsMode\022\017\n\013STREAM_FULL\020\001*^" +
+      "\n\017HandshakeStatus\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_VE" +
+      "RSION_MISMATCH\020\002\022\017\n\013AUTH_FAILED\020\003\022\023\n\017UNK" +
+      "NOWN_FAILURE\020\004B+\n\033org.apache.drill.exec." +
+      "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/52dcd7e4/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 4d03073..6687a86 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,6 +28,7 @@ public enum RpcType implements com.dyuproject.protostuff.EnumLite<RpcType>
     RUN_QUERY(3),
     CANCEL_QUERY(4),
     REQUEST_RESULTS(5),
+    RESUME_PAUSED_QUERY(11),
     QUERY_DATA(6),
     QUERY_HANDLE(7),
     REQ_META_FUNCTIONS(8),
@@ -61,6 +62,7 @@ public enum RpcType implements com.dyuproject.protostuff.EnumLite<RpcType>
             case 8: return REQ_META_FUNCTIONS;
             case 9: return RESP_FUNCTION_LIST;
             case 10: return QUERY_RESULT;
+            case 11: return RESUME_PAUSED_QUERY;
             default: return null;
         }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/protocol/src/main/protobuf/BitControl.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
index 93bc33c..c9295f0 100644
--- a/protocol/src/main/protobuf/BitControl.proto
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -14,25 +14,25 @@ enum RpcType {
   HANDSHAKE = 0;
   ACK = 1;
   GOODBYE = 2;
-    
+
   // bit requests
-  REQ_INIATILIZE_FRAGMENTS = 3; // Returns Handle
-    
+  REQ_INITIALIZE_FRAGMENTS = 3; // Returns Handle
+
   REQ_CANCEL_FRAGMENT = 6; // send a cancellation message for a fragment, returns Ack
   REQ_RECEIVER_FINISHED = 7;
   REQ_FRAGMENT_STATUS = 8; // send a fragment status, return Ack
   REQ_BIT_STATUS = 9; // get bit status.
   REQ_QUERY_STATUS = 10;
   REQ_QUERY_CANCEL = 15;
-      
-    // bit responses
+  REQ_UNPAUSE_FRAGMENT = 16; // send a resume message for a fragment, returns Ack
+
+  // bit responses
   RESP_FRAGMENT_HANDLE = 11;
   RESP_FRAGMENT_STATUS = 12;
   RESP_BIT_STATUS = 13;
   RESP_QUERY_STATUS = 14;
 }
 
-
 message BitControlHandshake{
   optional int32 rpc_version = 1;
   optional exec.shared.RpcChannel channel = 2 [default = BIT_CONTROL];

http://git-wip-us.apache.org/repos/asf/drill/blob/52dcd7e4/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index 185a646..ceed3d8 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -17,6 +17,7 @@ enum RpcType {
   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;
+  RESUME_PAUSED_QUERY = 11; // user is sending a query resume request to the drillbit
 
   // bit to user
   QUERY_DATA = 6; // drillbit is sending a query result data batch to the user