You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ad...@apache.org on 2015/09/05 00:23:12 UTC

drill git commit: DRILL-3455: If fragments on unregistered Drillbits finished successfully, do not fail the query

Repository: drill
Updated Branches:
  refs/heads/master cc7bc9535 -> 57c5d15d6


DRILL-3455: If fragments on unregistered Drillbits finished successfully, do not fail the query

+ DRILL-3448: Flipped the atLeastOneFailure condition in QueryManager
+ fixes in DrillbitStatusListener interface
+ logs from implementations of DrillbitStatusListener


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

Branch: refs/heads/master
Commit: 57c5d15d617ba93fc65949350c49778ec63e6986
Parents: cc7bc95
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Wed Jul 22 17:16:29 2015 -0700
Committer: adeneche <ad...@gmail.com>
Committed: Fri Sep 4 11:39:41 2015 -0700

----------------------------------------------------------------------
 .../work/foreman/DrillbitStatusListener.java    |  6 +--
 .../drill/exec/work/foreman/QueryManager.java   | 42 ++++++++++++--------
 2 files changed, 28 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/57c5d15d/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java
index 80f2ca1..2a6d412 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/DrillbitStatusListener.java
@@ -26,19 +26,17 @@ import java.util.Set;
  * Interface to define the listener to take actions when the set of active drillbits is changed.
  */
 public interface DrillbitStatusListener {
-  // TODO this doesn't belong here
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitStatusListener.class);
 
   /**
    * The action to taken when a set of drillbits are unregistered from the cluster.
    * @param  unregisteredDrillbits the set of newly unregistered drillbits.
    */
-  public void drillbitUnregistered(Set<CoordinationProtos.DrillbitEndpoint> unregisteredDrillbits);
+  void drillbitUnregistered(Set<CoordinationProtos.DrillbitEndpoint> unregisteredDrillbits);
 
   /**
    * The action to taken when a set of new drillbits are registered to the cluster.
    * @param  registeredDrillbits the set of newly registered drillbits. Note: the complete set of currently registered bits could be different.
    */
-  public void drillbitRegistered(Set<CoordinationProtos.DrillbitEndpoint> registeredDrillbits);
+  void drillbitRegistered(Set<CoordinationProtos.DrillbitEndpoint> registeredDrillbits);
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/57c5d15d/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 554a279..60173e2 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
@@ -420,11 +420,18 @@ public class QueryManager {
      * the internal fragmentComplete() method so whether we have failure or success, the nodeComplete event will only
      * occur once. (Two threads could be decrementing the fragment at the same time since this will likely come from an
      * external event).
+     *
+     * @return true if the node has fragments that are pending (non-terminal state); false if all fragments running on
+     * this node have already terminated.
      */
-    public void nodeDead() {
+    public boolean nodeDead() {
+      if (completedFragments.get() == totalFragments.get()) {
+        return false;
+      }
       while (completedFragments.get() < totalFragments.get()) {
         fragmentComplete();
       }
+      return true;
     }
 
   }
@@ -492,26 +499,29 @@ public class QueryManager {
       final StringBuilder failedNodeList = new StringBuilder();
       boolean atLeastOneFailure = false;
 
-      for(final DrillbitEndpoint ep : unregisteredDrillbits) {
+      for (final DrillbitEndpoint ep : unregisteredDrillbits) {
         final NodeTracker tracker = nodeMap.get(ep);
-        if (tracker != null) {
-          // mark node as dead.
-          tracker.nodeDead();
-
-          // capture node name for exception or logging message
-          if (atLeastOneFailure) {
-            failedNodeList.append(", ");
-          }else{
-            atLeastOneFailure = true;
-          }
-          failedNodeList.append(ep.getAddress());
-          failedNodeList.append(":");
-          failedNodeList.append(ep.getUserPort());
+        if (tracker == null) {
+          continue; // fragments were not assigned to this Drillbit
+        }
+
+        // mark node as dead.
+        if (!tracker.nodeDead()) {
+          continue; // fragments assigned to this Drillbit completed
+        }
 
+        // fragments were running on the Drillbit, capture node name for exception or logging message
+        if (atLeastOneFailure) {
+          failedNodeList.append(", ");
+        } else {
+          atLeastOneFailure = true;
         }
+        failedNodeList.append(ep.getAddress());
+        failedNodeList.append(":");
+        failedNodeList.append(ep.getUserPort());
       }
 
-      if (!atLeastOneFailure) {
+      if (atLeastOneFailure) {
         logger.warn("Drillbits [{}] no longer registered in cluster.  Canceling query {}",
             failedNodeList, QueryIdHelper.getQueryId(queryId));
         stateListener.moveToState(QueryState.FAILED,