[ 
https://issues.apache.org/jira/browse/DRILL-6255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16475040#comment-16475040
 ] 

ASF GitHub Bot commented on DRILL-6255:
---------------------------------------

sohami commented on a change in pull request #1253: DRILL-6255: Drillbit while 
sending control message to itself creates …
URL: https://github.com/apache/drill/pull/1253#discussion_r188090731
 
 

 ##########
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java
 ##########
 @@ -180,44 +170,38 @@ private void setupNonRootFragments(final 
Collection<PlanFragment> fragments) thr
       foreman.getQueryManager().addFragmentStatusTracker(planFragment, false);
 
       if (planFragment.getLeafFragment()) {
-        updateFragmentCollection(planFragment, localDrillbitEndpoint, 
localLeafFragmentList, remoteLeafFragmentMap);
+        leafFragmentMap.put(planFragment.getAssignment(), planFragment);
       } else {
-        updateFragmentCollection(planFragment, localDrillbitEndpoint, 
localIntFragmentList, remoteIntFragmentMap);
+        intFragmentMap.put(planFragment.getAssignment(), planFragment);
       }
     }
 
     /*
      * We need to wait for the intermediates to be sent so that they'll be set 
up by the time
-     * the leaves start producing data. We'll use this latch to wait for the 
responses.
+     * the leaves start producing data. We'll use this latch to wait for the 
responses. All the local intermediate
+     * fragments are submitted locally without creating any actual control 
connection to itself.
      *
      * However, in order not to hang the process if any of the RPC requests 
fails, we always
      * count down (see FragmentSubmitFailures), but we count the number of 
failures so that we'll
      * know if any submissions did fail.
      */
-    scheduleRemoteIntermediateFragments(remoteIntFragmentMap);
-
-    // Setup local intermediate fragments
-    for (final PlanFragment fragment : localIntFragmentList) {
-      startLocalFragment(fragment);
-    }
+    scheduleRemoteIntermediateFragments(intFragmentMap);
 
     injector.injectChecked(foreman.getQueryContext().getExecutionControls(), 
"send-fragments", ForemanException.class);
     /*
      * Send the remote (leaf) fragments; we don't wait for these. Any problems 
will come in through
-     * the regular sendListener event delivery.
+     * the regular sendListener event delivery˚. All the local leaf fragments 
are submitted locally without creating
+     * any actual control connection to itself.
      */
-    for (final DrillbitEndpoint ep : remoteLeafFragmentMap.keySet()) {
-      sendRemoteFragments(ep, remoteLeafFragmentMap.get(ep), null, null);
-    }
-
-    // Setup local leaf fragments
-    for (final PlanFragment fragment : localLeafFragmentList) {
-      startLocalFragment(fragment);
+    for (final DrillbitEndpoint ep : leafFragmentMap.keySet()) {
+      sendRemoteFragments(ep, leafFragmentMap.get(ep), null, null);
     }
   }
 
   /**
-   * Send all the remote fragments belonging to a single target drillbit in 
one request.
+   * Send all the remote fragments belonging to a single target drillbit in 
one request. If the assignment
 
 Review comment:
   renamed `scheduleRemoteIntermediateFragments` to 
`scheduleIntermediateFragments` but kept `sendRemoteFragments` as is since it's 
used to send intermediate fragments as well not just leaf fragments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Drillbit while sending control message to itself creates a connection instead 
> of submitting locally
> ---------------------------------------------------------------------------------------------------
>
>                 Key: DRILL-6255
>                 URL: https://issues.apache.org/jira/browse/DRILL-6255
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Flow
>    Affects Versions: 1.12.0
>            Reporter: Sorabh Hamirwasia
>            Assignee: Sorabh Hamirwasia
>            Priority: Major
>             Fix For: 1.14.0
>
>
> With the new shutdown feature introduced in 1.12, there is a state introduced 
> in DrillbitEndpoint. Due to this the equality check happening 
> [here|https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java#L256]
>  will result in false and hence the fragments supposed to be scheduled on 
> Foreman will be treated as remote fragments and a connection will be created 
> to schedule it. The equality check is false because localEndpoint state is 
> STARTUP whereas state in assigned Drillbit is ONLINE.
> I guess now we should update the equality check to verify just for address 
> and control port to be same between assigned and local Drillbit endpoint. A 
> test can be added for this based on _sys.memory_ table since that guarantees 
> scheduling minor fragments on each Drillbit node.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to